You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemds.apache.org by mb...@apache.org on 2021/09/19 14:24:18 UTC

[systemds] branch master updated: [SYSTEMDS-3136/3137] IOGEN: Generating custom readers by example

This is an automated email from the ASF dual-hosted git repository.

mboehm7 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/systemds.git


The following commit(s) were added to refs/heads/master by this push:
     new 205f57f  [SYSTEMDS-3136/3137] IOGEN: Generating custom readers by example
205f57f is described below

commit 205f57f841e34b3ba47d5a60a4476124089fbee4
Author: Saeed Fathollahzadeh <s....@gmail.com>
AuthorDate: Sun Sep 19 15:26:01 2021 +0200

    [SYSTEMDS-3136/3137] IOGEN: Generating custom readers by example
    
    Initial commit of a framework for generating custom readers by example
    (given raw sample, and mapped sample - generate generally applicable
    reader).
    
    Closes #1369.
---
 .github/workflows/functionsTests.yml               |   2 +-
 .../sysds/runtime/iogen/CustomProperties.java      | 135 ++++
 .../sysds/runtime/iogen/FastStringTokenizer.java   |  95 +++
 .../sysds/runtime/iogen/FrameGenerateReader.java   | 292 +++++++
 .../apache/sysds/runtime/iogen/GenerateReader.java | 131 ++++
 .../sysds/runtime/iogen/MatrixGenerateReader.java  | 298 +++++++
 .../org/apache/sysds/runtime/iogen/RawRow.java     | 558 +++++++++++++
 .../apache/sysds/runtime/iogen/ReaderMapping.java  | 860 +++++++++++++++++++++
 .../sysds/runtime/iogen/SampleProperties.java      |  79 ++
 .../sysds/runtime/iogen/ValueTrimFormat.java       | 228 ++++++
 .../apache/sysds/runtime/util/DataConverter.java   | 280 +++----
 .../iogen/FrameGenerateReaderCSVTest.java          | 120 +++
 .../iogen/FrameGenerateReaderLibSVMTest.java       | 142 ++++
 .../iogen/FrameGenerateReaderMatrixMarketTest.java | 100 +++
 .../test/functions/iogen/GenerateRandomFrame.java  | 313 ++++++++
 .../test/functions/iogen/GenerateRandomMatrix.java | 319 ++++++++
 .../functions/iogen/GenerateReaderFrameTest.java   | 193 +++++
 .../functions/iogen/GenerateReaderMatrixTest.java  | 107 +++
 .../iogen/MatrixGenerateReaderCSVTest.java         | 151 ++++
 .../iogen/MatrixGenerateReaderLibSVMTest.java      | 169 ++++
 .../MatrixGenerateReaderMatrixMarketTest.java      | 238 ++++++
 21 files changed, 4669 insertions(+), 141 deletions(-)

diff --git a/.github/workflows/functionsTests.yml b/.github/workflows/functionsTests.yml
index ae77dd4..f94316f 100644
--- a/.github/workflows/functionsTests.yml
+++ b/.github/workflows/functionsTests.yml
@@ -47,7 +47,7 @@ jobs:
           "**.functions.federated.io.**,**.functions.federated.paramserv.**,**.functions.federated.primitives.**,**.functions.federated.transform.**",
           "**.functions.codegenalg.partone.**",
           "**.functions.builtin.**",
-          "**.functions.frame.**,**.functions.indexing.**,**.functions.io.**,**.functions.jmlc.**,**.functions.lineage.**",
+          "**.functions.frame.**,**.functions.indexing.**,**.functions.io.**,**.functions.iogen.**,**.functions.jmlc.**,**.functions.lineage.**",
           "**.functions.dnn.**,**.functions.paramserv.**",
           "**.functions.misc.**,**.functions.mlcontext.**",
           "**.functions.nary.**,**.functions.quaternary.**",
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/CustomProperties.java b/src/main/java/org/apache/sysds/runtime/iogen/CustomProperties.java
new file mode 100644
index 0000000..05c9222
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/CustomProperties.java
@@ -0,0 +1,135 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.io.FileFormatProperties;
+
+import java.io.Serializable;
+import java.util.HashSet;
+
+public class CustomProperties extends FileFormatProperties implements Serializable {
+	protected static final Log LOG = LogFactory.getLog(CustomProperties.class.getName());
+	private static final long serialVersionUID = -4447926749068752721L;
+
+	private String delim;
+	private String indexDelim;
+	private HashSet<String> naStrings;
+	private int firstColIndex;
+	private int firstRowIndex;
+
+	protected enum GRPattern {
+		Regular, Irregular;
+
+		@Override
+		public String toString() {
+			return this.name().toLowerCase();
+		}
+	}
+
+	protected enum GRSymmetry {
+		GENERAL, SYMMETRIC, SKEW_SYMMETRIC;
+
+		@Override
+		public String toString() {
+			return this.name().toLowerCase().replaceAll("_", "-");
+		}
+	}
+
+	private GRPattern rowPattern;
+	private GRPattern colPattern;
+	private GRSymmetry grSymmetry;
+
+	public CustomProperties() {
+	}
+
+	// Row & Col Regular Format
+	public CustomProperties(GRPattern rowPattern, String delim, HashSet<String> naStrings) {
+		this.delim = delim;
+		this.naStrings = naStrings;
+		this.rowPattern = rowPattern;
+		this.colPattern = GRPattern.Regular;
+		this.grSymmetry = GRSymmetry.GENERAL;
+		this.firstRowIndex = 0;
+		this.firstColIndex = 0;
+	}
+
+	// Row Regular & Col Irregular Format
+	public CustomProperties(GRPattern rowPattern, String delim, String indexDelim, int firstColIndex) {
+		this.delim = delim;
+		this.indexDelim = indexDelim;
+		this.rowPattern = rowPattern;
+		this.colPattern = GRPattern.Irregular;
+		this.grSymmetry = GRSymmetry.GENERAL;
+		this.firstColIndex = firstColIndex;
+		this.firstRowIndex = 0;
+	}
+
+	// Row Irregular format
+	public CustomProperties(GRSymmetry grSymmetry, String delim, int firstRowIndex, int firstColIndex) {
+		this.delim = delim;
+		this.grSymmetry = grSymmetry;
+		this.colPattern = GRPattern.Regular;
+		this.rowPattern = GRPattern.Irregular;
+		this.firstColIndex = firstColIndex;
+		this.firstRowIndex = firstRowIndex;
+	}
+
+	public String getDelim() {
+		return delim;
+	}
+
+	public String getIndexDelim() {
+		return indexDelim;
+	}
+
+	public HashSet<String> getNaStrings() {
+		return naStrings;
+	}
+
+	public GRPattern getRowPattern() {
+		return rowPattern;
+	}
+
+	public GRPattern getColPattern() {
+		return colPattern;
+	}
+
+	public GRSymmetry getGrSymmetry() {
+		return grSymmetry;
+	}
+
+	public int getFirstColIndex() {
+		return firstColIndex;
+	}
+
+	public void setFirstColIndex(int firstColIndex) {
+		this.firstColIndex = firstColIndex;
+	}
+
+	public int getFirstRowIndex() {
+		return firstRowIndex;
+	}
+
+	public void setFirstRowIndex(int firstRowIndex) {
+		this.firstRowIndex = firstRowIndex;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/FastStringTokenizer.java b/src/main/java/org/apache/sysds/runtime/iogen/FastStringTokenizer.java
new file mode 100644
index 0000000..42bfbe4
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/FastStringTokenizer.java
@@ -0,0 +1,95 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+import java.io.Serializable;
+import java.util.HashSet;
+
+public class FastStringTokenizer implements Serializable {
+
+	private static final long serialVersionUID = -4698672725609750097L;
+	private String _string = null;
+	private String _del = "";
+	private int _pos = -1;
+	private int _index = 0;
+	private HashSet<String> naStrings = null;
+
+	public FastStringTokenizer(String delimiter) {
+		_del = delimiter;
+		reset(null);
+	}
+
+	public void reset(String string) {
+		_string = string;
+		_pos = 0;
+		_index = 0;
+	}
+
+	public String nextToken() {
+		int len = _string.length();
+		int start = _pos;
+
+		if(_pos == -1) {
+			_index = -1;
+			return "0";
+		}
+		//find start (skip over leading delimiters)
+		while(start < len && _del.equals(_string.substring(start, Math.min(start + _del.length(), _string.length())))) {
+			start += _del.length();
+			_index++;
+		}
+
+		//find end (next delimiter) and return
+		if(start < len) {
+			_pos = _string.indexOf(_del, start);
+			if(start < _pos && _pos < len)
+				return _string.substring(start, _pos);
+			else
+				return _string.substring(start);
+		}
+		//no next token
+		_index = -1;
+		return null;
+	}
+
+	public int nextInt() {
+		return Integer.parseInt(nextToken());
+	}
+
+	public long nextLong() {
+		return Long.parseLong(nextToken());
+	}
+
+	public double nextDouble() {
+		String nt = nextToken();
+		if((naStrings != null && naStrings.contains(nt)) || nt == null)
+			return 0;
+		else
+			return Double.parseDouble(nt);
+	}
+
+	public int getIndex() {
+		return _index;
+	}
+
+	public void setNaStrings(HashSet<String> naStrings) {
+		this.naStrings = naStrings;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/FrameGenerateReader.java b/src/main/java/org/apache/sysds/runtime/iogen/FrameGenerateReader.java
new file mode 100644
index 0000000..5820e08
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/FrameGenerateReader.java
@@ -0,0 +1,292 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+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.*;
+import org.apache.sysds.common.Types;
+import org.apache.sysds.conf.ConfigurationManager;
+import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.io.FrameReader;
+import org.apache.sysds.runtime.io.IOUtilFunctions;
+import org.apache.sysds.runtime.matrix.data.FrameBlock;
+import org.apache.sysds.runtime.util.InputStreamInputFormat;
+import org.apache.sysds.runtime.util.UtilFunctions;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+public abstract class FrameGenerateReader extends FrameReader {
+
+	protected CustomProperties _props;
+	protected final FastStringTokenizer fastStringTokenizerDelim;
+
+	public FrameGenerateReader(CustomProperties _props) {
+		this._props = _props;
+		fastStringTokenizerDelim = new FastStringTokenizer(_props.getDelim());
+	}
+
+	private int getNumRows(List<Path> files, FileSystem fs) throws IOException, DMLRuntimeException {
+		int rows = 0;
+		String value;
+		for(int fileNo = 0; fileNo < files.size(); fileNo++) {
+			BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(files.get(fileNo))));
+			try {
+				// Row Regular
+				if(_props.getRowPattern().equals(CustomProperties.GRPattern.Regular)) {
+					// TODO: check the file has header?
+					while(br.readLine() != null)
+						rows++;
+				}
+				// Row Irregular
+				else {
+					FastStringTokenizer st = new FastStringTokenizer(_props.getDelim());
+					while((value = br.readLine()) != null) {
+						st.reset(value);
+						int row = st.nextInt();
+						rows = Math.max(rows, row);
+					}
+					rows++;
+				}
+			}
+			finally {
+				IOUtilFunctions.closeSilently(br);
+			}
+		}
+		return rows;
+	}
+
+	@Override
+	public FrameBlock readFrameFromHDFS(String fname, Types.ValueType[] schema, String[] names, long rlen,
+		long clen) throws IOException, DMLRuntimeException {
+
+		// prepare file access
+		JobConf job = new JobConf(ConfigurationManager.getCachedJobConf());
+		Path path = new Path(fname);
+		FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
+		FileInputFormat.addInputPath(job, path);
+
+		// check existence and non-empty file
+		checkValidInputFile(fs, path);
+
+		// compute size if necessary
+		if(rlen <= 0) {
+			ArrayList<Path> paths = new ArrayList<>();
+			paths.add(path);
+			rlen = getNumRows(paths, fs);
+		}
+
+		// allocate output frame block
+		Types.ValueType[] lschema = createOutputSchema(schema, clen);
+		String[] lnames = createOutputNames(names, clen);
+		FrameBlock ret = createOutputFrameBlock(lschema, lnames, rlen);
+
+		// core read (sequential/parallel)
+		readFrameFromHDFS(path, job, fs, ret, lschema, lnames, rlen, clen);
+
+		return ret;
+
+	}
+
+	@Override
+	public FrameBlock readFrameFromInputStream(InputStream is, Types.ValueType[] schema, String[] names,
+		long rlen, long clen) throws IOException, DMLRuntimeException {
+
+		// allocate output frame block
+		Types.ValueType[] lschema = createOutputSchema(schema, clen);
+		String[] lnames = createOutputNames(names, clen);
+		FrameBlock ret = createOutputFrameBlock(lschema, lnames, rlen);
+
+		// core read (sequential/parallel)
+		InputStreamInputFormat informat = new InputStreamInputFormat(is);
+		InputSplit split = informat.getSplits(null, 1)[0];
+		readFrameFromInputSplit(split, informat, null, ret, schema, names, rlen, clen, 0, true);
+
+		return ret;
+	}
+
+	protected void readFrameFromHDFS(Path path, JobConf job, FileSystem fs, FrameBlock dest, Types.ValueType[] schema,
+		String[] names, long rlen, long clen) throws IOException {
+
+		TextInputFormat informat = new TextInputFormat();
+		informat.configure(job);
+		InputSplit[] splits = informat.getSplits(job, 1);
+		splits = IOUtilFunctions.sortInputSplits(splits);
+		for(int i = 0, rpos = 0; i < splits.length; i++)
+			rpos = readFrameFromInputSplit(splits[i], informat, job, dest, schema, names, rlen, clen, rpos, i == 0);
+	}
+
+	protected abstract int readFrameFromInputSplit(InputSplit split, InputFormat<LongWritable, Text> informat,
+		JobConf job, FrameBlock dest, Types.ValueType[] schema, String[] names, long rlen, long clen, int rl,
+		boolean first) throws IOException;
+
+	public static class FrameReaderRowRegularColRegular extends FrameGenerateReader {
+
+		public FrameReaderRowRegularColRegular(CustomProperties _props) {
+			super(_props);
+		}
+
+		@Override
+		protected int readFrameFromInputSplit(InputSplit split, InputFormat<LongWritable, Text> informat,
+			JobConf job, FrameBlock dest, Types.ValueType[] schema, String[] names, long rlen, long clen, int rl,
+			boolean first) throws IOException {
+
+			String cellValue;
+			fastStringTokenizerDelim.setNaStrings(_props.getNaStrings());
+
+			// create record reader
+			RecordReader<LongWritable, Text> reader = informat.getRecordReader(split, job, Reporter.NULL);
+			LongWritable key = new LongWritable();
+			Text value = new Text();
+			int row = rl;
+			int col = 0;
+			Set<String> naValues = _props.getNaStrings();
+
+			// Read the data
+			try {
+				while(reader.next(key, value)) // foreach line
+				{
+					String cellStr = value.toString();
+					fastStringTokenizerDelim.reset(cellStr);
+					while(col != -1) {
+						cellValue = fastStringTokenizerDelim.nextToken();
+						col = fastStringTokenizerDelim.getIndex();
+						if(col != -1 && cellValue != null && (naValues == null || !naValues.contains(cellValue))) {
+							dest.set(row, col, UtilFunctions.stringToObject(schema[col], cellValue));
+						}
+					}
+					row++;
+					col = 0;
+				}
+			}
+			finally {
+				IOUtilFunctions.closeSilently(reader);
+			}
+			return row;
+		}
+	}
+
+	public static class FrameReaderRowRegularColIrregular extends FrameGenerateReader {
+
+		public FrameReaderRowRegularColIrregular(CustomProperties _props) {
+			super(_props);
+		}
+
+		@Override
+		protected int readFrameFromInputSplit(InputSplit split, InputFormat<LongWritable, Text> informat,
+			JobConf job, FrameBlock dest, Types.ValueType[] schema, String[] names, long rlen, long clen, int rl,
+			boolean first) throws IOException {
+
+			String cellValue;
+			FastStringTokenizer fastStringTokenizerIndexDelim = new FastStringTokenizer(_props.getIndexDelim());
+
+			// create record reader
+			RecordReader<LongWritable, Text> reader = informat.getRecordReader(split, job, Reporter.NULL);
+			LongWritable key = new LongWritable();
+			Text value = new Text();
+			int row = rl;
+			int col = 0;
+
+			// Read the data
+			try {
+				while(reader.next(key, value)) // foreach line
+				{
+					String cellStr = value.toString();
+					fastStringTokenizerDelim.reset(cellStr);
+					String cellValueString = fastStringTokenizerDelim.nextToken();
+					dest.set(row, (int) clen - 1 - _props.getFirstColIndex(),
+						UtilFunctions.stringToObject(schema[(int) clen - 1 - _props.getFirstColIndex()], cellValueString));
+
+					while(col != -1) {
+						String nt = fastStringTokenizerDelim.nextToken();
+						if(fastStringTokenizerDelim.getIndex() == -1)
+							break;
+						fastStringTokenizerIndexDelim.reset(nt);
+						col = fastStringTokenizerIndexDelim.nextInt();
+						cellValue = fastStringTokenizerIndexDelim.nextToken();
+						if(col != -1 && cellValue != null) {
+							dest.set(row, col - _props.getFirstColIndex(),
+								UtilFunctions.stringToObject(schema[col - _props.getFirstColIndex()], cellValue));
+						}
+					}
+					row++;
+					col = 0;
+				}
+			}
+			finally {
+				IOUtilFunctions.closeSilently(reader);
+			}
+			return row;
+		}
+	}
+
+	public static class FrameReaderRowIrregular extends FrameGenerateReader {
+
+		public FrameReaderRowIrregular(CustomProperties _props) {
+			super(_props);
+		}
+
+		@Override
+		protected int readFrameFromInputSplit(InputSplit split, InputFormat<LongWritable, Text> informat,
+			JobConf job, FrameBlock dest, Types.ValueType[] schema, String[] names, long rlen, long clen, int rl,
+			boolean first) throws IOException {
+
+			String cellValue;
+			fastStringTokenizerDelim.setNaStrings(_props.getNaStrings());
+
+			// create record reader
+			RecordReader<LongWritable, Text> reader = informat.getRecordReader(split, job, Reporter.NULL);
+			LongWritable key = new LongWritable();
+			Text value = new Text();
+			int row = rl;
+			int col = 0;
+
+			// Read the data
+			try {
+				while(reader.next(key, value)) // foreach line
+				{
+					String cellStr = value.toString();
+					fastStringTokenizerDelim.reset(cellStr);
+					int ri = fastStringTokenizerDelim.nextInt();
+					col = fastStringTokenizerDelim.nextInt();
+					cellValue = fastStringTokenizerDelim.nextToken();
+
+					if(col != -1 && cellValue != null) {
+						dest.set(ri-_props.getFirstRowIndex(), col - _props.getFirstColIndex(),
+							UtilFunctions.stringToObject(schema[col - _props.getFirstColIndex()], cellValue));
+					}
+					row = Math.max(row, ri);
+				}
+			}
+			finally {
+				IOUtilFunctions.closeSilently(reader);
+			}
+			return row;
+		}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/GenerateReader.java b/src/main/java/org/apache/sysds/runtime/iogen/GenerateReader.java
new file mode 100644
index 0000000..0e9b1cd
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/GenerateReader.java
@@ -0,0 +1,131 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.io.MatrixReader;
+import org.apache.sysds.runtime.io.FrameReader;
+import org.apache.sysds.runtime.matrix.data.FrameBlock;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+
+/*
+   Generate Reader has two steps:
+      1. Identify file format and extract the properties of it based on the Sample Matrix.
+      The ReaderMapping class tries to map the Sample Matrix on the Sample Raw Matrix.
+      The result of a ReaderMapping is a FileFormatProperties object.
+
+      2. Generate a reader based on inferred properties.
+
+    Note. Base on this implementation, it is possible to generate a reader 
+    base on Sample Matrix and generate a reader for a frame or vice versa.
+*/
+public abstract class GenerateReader {
+
+	protected static final Log LOG = LogFactory.getLog(GenerateReader.class.getName());
+
+	protected static ReaderMapping readerMapping;
+
+	public GenerateReader(SampleProperties sampleProperties) throws Exception {
+
+		readerMapping = sampleProperties.getDataType().isMatrix() ? new ReaderMapping.MatrixReaderMapping(
+			sampleProperties.getSampleRaw(), sampleProperties.getSampleMatrix()) : new ReaderMapping.FrameReaderMapping(
+			sampleProperties.getSampleRaw(), sampleProperties.getSampleFrame());
+	}
+
+	// Generate Reader for Matrix
+	public static class GenerateReaderMatrix extends GenerateReader {
+
+		private MatrixReader matrixReader;
+
+		public GenerateReaderMatrix(SampleProperties sampleProperties) throws Exception {
+			super(sampleProperties);
+		}
+
+		public GenerateReaderMatrix(String sampleRaw, MatrixBlock sampleMatrix) throws Exception {
+			super(new SampleProperties(sampleRaw, sampleMatrix));
+		}
+
+		public MatrixReader getReader() throws Exception {
+
+			boolean isMapped = readerMapping != null && readerMapping.isMapped();
+			if(!isMapped) {
+				throw new Exception("Sample raw data and sample matrix don't match !!");
+			}
+			CustomProperties ffp = readerMapping.getFormatProperties();
+			if(ffp == null) {
+				throw new Exception("The file format couldn't recognize!!");
+			}
+			// 2. Generate a Matrix Reader:
+			if(ffp.getRowPattern().equals(CustomProperties.GRPattern.Regular)) {
+				if(ffp.getColPattern().equals(CustomProperties.GRPattern.Regular)) {
+					matrixReader = new MatrixGenerateReader.MatrixReaderRowRegularColRegular(ffp);
+				}
+				else {
+					matrixReader = new MatrixGenerateReader.MatrixReaderRowRegularColIrregular(ffp);
+				}
+			}
+			else {
+				matrixReader = new MatrixGenerateReader.MatrixReaderRowIrregular(ffp);
+			}
+			return matrixReader;
+		}
+
+	}
+
+	// Generate Reader for Frame
+	public static class GenerateReaderFrame extends GenerateReader {
+
+		private FrameReader frameReader;
+
+		public GenerateReaderFrame(SampleProperties sampleProperties) throws Exception {
+			super(sampleProperties);
+		}
+
+		public GenerateReaderFrame(String sampleRaw, FrameBlock sampleFrame) throws Exception {
+			super(new SampleProperties(sampleRaw, sampleFrame));
+		}
+
+		public FrameReader getReader() throws Exception {
+
+			boolean isMapped = readerMapping != null && readerMapping.isMapped();
+			if(!isMapped) {
+				throw new Exception("Sample raw data and sample frame don't match !!");
+			}
+			CustomProperties ffp = readerMapping.getFormatProperties();
+			if(ffp == null) {
+				throw new Exception("The file format couldn't recognize!!");
+			}
+			// 2. Generate a Frame Reader:
+			if(ffp.getRowPattern().equals(CustomProperties.GRPattern.Regular)) {
+				if(ffp.getColPattern().equals(CustomProperties.GRPattern.Regular)) {
+					frameReader = new FrameGenerateReader.FrameReaderRowRegularColRegular(ffp);
+				}
+				else {
+					frameReader = new FrameGenerateReader.FrameReaderRowRegularColIrregular(ffp);
+				}
+			}
+			else {
+				frameReader = new FrameGenerateReader.FrameReaderRowIrregular(ffp);
+			}
+			return frameReader;
+		}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/MatrixGenerateReader.java b/src/main/java/org/apache/sysds/runtime/iogen/MatrixGenerateReader.java
new file mode 100644
index 0000000..770c155
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/MatrixGenerateReader.java
@@ -0,0 +1,298 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+import org.apache.commons.lang.mutable.MutableInt;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.sysds.conf.ConfigurationManager;
+import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.sysds.runtime.io.IOUtilFunctions;
+import org.apache.sysds.runtime.io.MatrixReader;
+import org.apache.sysds.runtime.util.UtilFunctions;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public abstract class MatrixGenerateReader extends MatrixReader {
+
+	protected static CustomProperties _props;
+	protected final FastStringTokenizer fastStringTokenizerDelim;
+
+	public MatrixGenerateReader(CustomProperties _props) {
+		MatrixGenerateReader._props = _props;
+		fastStringTokenizerDelim = new FastStringTokenizer(_props.getDelim());
+	}
+
+	protected MatrixBlock computeSize(List<Path> files, FileSystem fs, long rlen, long clen)
+		throws IOException, DMLRuntimeException {
+		// allocate target matrix block based on given size;
+		return new MatrixBlock(getNumRows(files, fs), (int) clen, rlen * clen);
+	}
+
+	private static int getNumRows(List<Path> files, FileSystem fs) throws IOException, DMLRuntimeException {
+		int rows = 0;
+		String value;
+		for(int fileNo = 0; fileNo < files.size(); fileNo++) {
+			BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(files.get(fileNo))));
+			try {
+				// Row Regular
+				if(_props.getRowPattern().equals(CustomProperties.GRPattern.Regular)) {
+					// TODO: check the file has header?
+					while(br.readLine() != null)
+						rows++;
+				}
+				// Row Irregular
+				else {
+					FastStringTokenizer st = new FastStringTokenizer(_props.getDelim());
+					while((value = br.readLine()) != null) {
+						st.reset(value);
+						int row = st.nextInt();
+						rows = Math.max(rows, row);
+					}
+					rows++;
+				}
+			}
+			finally {
+				IOUtilFunctions.closeSilently(br);
+			}
+		}
+		return rows;
+	}
+
+	@Override
+	public MatrixBlock readMatrixFromHDFS(String fname, long rlen, long clen, int blen, long estnnz)
+		throws IOException, DMLRuntimeException {
+
+		MatrixBlock ret = null;
+		if(rlen >= 0 && clen >= 0) //otherwise allocated on read
+			ret = createOutputMatrixBlock(rlen, clen, (int) rlen, estnnz, true, false);
+
+		//prepare file access
+		JobConf job = new JobConf(ConfigurationManager.getCachedJobConf());
+		Path path = new Path(fname);
+		FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
+
+		//core read
+		ret = readMatrixFromHDFS(path, job, fs, ret, rlen, clen, blen);
+
+		return ret;
+	}
+
+	@Override
+	public MatrixBlock readMatrixFromInputStream(InputStream is, long rlen, long clen, int blen, long estnnz)
+		throws IOException, DMLRuntimeException {
+
+		MatrixBlock ret = null;
+		if(rlen >= 0 && clen >= 0) //otherwise allocated on read
+			ret = createOutputMatrixBlock(rlen, clen, (int) rlen, estnnz, true, false);
+
+		return ret;
+	}
+
+	@SuppressWarnings("unchecked")
+	private MatrixBlock readMatrixFromHDFS(Path path, JobConf job, FileSystem fs, MatrixBlock dest, long rlen,
+		long clen, int blen) throws IOException, DMLRuntimeException {
+		//prepare file paths in alphanumeric order
+		ArrayList<Path> files = new ArrayList<>();
+		if(fs.isDirectory(path)) {
+			for(FileStatus stat : fs.listStatus(path, IOUtilFunctions.hiddenFileFilter))
+				files.add(stat.getPath());
+			Collections.sort(files);
+		}
+		else
+			files.add(path);
+
+		//determine matrix size via additional pass if required
+		if(dest == null) {
+			dest = computeSize(files, fs, rlen, clen);
+			rlen = dest.getNumRows();
+			//clen = dest.getNumColumns();
+		}
+
+		//actual read of individual files
+		long lnnz = 0;
+		MutableInt row = new MutableInt(0);
+		for(int fileNo = 0; fileNo < files.size(); fileNo++) {
+			lnnz += readMatrixFromInputStream(fs.open(files.get(fileNo)), path.toString(), dest, row, rlen, clen, blen);
+		}
+
+		//post processing
+		dest.setNonZeros(lnnz);
+
+		return dest;
+	}
+
+	protected abstract long readMatrixFromInputStream(InputStream is, String srcInfo, MatrixBlock dest,
+		MutableInt rowPos, long rlen, long clen, int blen) throws IOException;
+
+	public static class MatrixReaderRowRegularColRegular extends MatrixGenerateReader {
+
+		public MatrixReaderRowRegularColRegular(CustomProperties _props) {
+			super(_props);
+		}
+
+		@Override
+		protected long readMatrixFromInputStream(InputStream is, String srcInfo, MatrixBlock dest,
+			MutableInt rowPos, long rlen, long clen, int blen) throws IOException {
+
+			String value = null;
+			int row = rowPos.intValue();
+			double cellValue = 0;
+			int col = 0;
+			long lnnz = 0;
+			fastStringTokenizerDelim.setNaStrings(_props.getNaStrings());
+
+			BufferedReader br = new BufferedReader(new InputStreamReader(is));
+
+			//TODO: separate implementation for Sparse and Dens Matrix Blocks
+
+			// Read the data
+			try {
+				while((value = br.readLine()) != null) //foreach line
+				{
+					fastStringTokenizerDelim.reset(value);
+					while(col != -1) {
+						cellValue = fastStringTokenizerDelim.nextDouble();
+						col = fastStringTokenizerDelim.getIndex();
+						if(cellValue != 0) {
+							dest.appendValue(row, col, cellValue);
+							lnnz++;
+						}
+					}
+					row++;
+					col = 0;
+				}
+			}
+			finally {
+				IOUtilFunctions.closeSilently(br);
+			}
+
+			rowPos.setValue(row);
+			return lnnz;
+		}
+	}
+
+	public static class MatrixReaderRowRegularColIrregular extends MatrixGenerateReader {
+
+		public MatrixReaderRowRegularColIrregular(CustomProperties _props) {
+			super(_props);
+		}
+
+		@Override
+		protected long readMatrixFromInputStream(InputStream is, String srcInfo, MatrixBlock dest,
+			MutableInt rowPos, long rlen, long clen, int blen) throws IOException {
+
+			String value = null;
+			int row = rowPos.intValue();
+			double cellValue = 0;
+			int col = 0;
+			long lnnz = 0;
+
+			final FastStringTokenizer fastStringTokenizerIndexDelim = new FastStringTokenizer(_props.getIndexDelim());
+			BufferedReader br = new BufferedReader(new InputStreamReader(is));
+
+			//TODO: separate implementation for Sparse and Dens Matrix Blocks
+
+			// Read the data
+			try {
+				while((value = br.readLine()) != null) //foreach line
+				{
+					fastStringTokenizerDelim.reset(value);
+					String cellValueString = fastStringTokenizerDelim.nextToken();
+					cellValue = UtilFunctions.parseToDouble(cellValueString, null);
+					dest.appendValue(row, (int) clen-_props.getFirstColIndex()-1, cellValue);
+
+					while(col != -1) {
+						String nt = fastStringTokenizerDelim.nextToken();
+						if(fastStringTokenizerDelim.getIndex() == -1)
+							break;
+						fastStringTokenizerIndexDelim.reset(nt);
+						col = fastStringTokenizerIndexDelim.nextInt();
+						cellValue = fastStringTokenizerIndexDelim.nextDouble();
+						if(cellValue != 0) {
+							dest.appendValue(row, col-_props.getFirstColIndex(), cellValue);
+							lnnz++;
+						}
+					}
+					row++;
+					col = 0;
+				}
+			}
+			finally {
+				IOUtilFunctions.closeSilently(br);
+			}
+
+			rowPos.setValue(row);
+			return lnnz;
+		}
+	}
+
+	public static class MatrixReaderRowIrregular extends MatrixGenerateReader {
+
+		public MatrixReaderRowIrregular(CustomProperties _props) {
+			super(_props);
+		}
+
+		@Override
+		protected long readMatrixFromInputStream(InputStream is, String srcInfo, MatrixBlock dest,
+			MutableInt rowPos, long rlen, long clen, int blen) throws IOException {
+			String value = null;
+			int row = rowPos.intValue();
+			double cellValue = 0;
+			int col = 0;
+			long lnnz = 0;
+
+			BufferedReader br = new BufferedReader(new InputStreamReader(is));
+
+			//TODO: separate implementation for Sparse and Dens Matrix Blocks
+
+			// Read the data
+			try {
+				while((value = br.readLine()) != null) //foreach line
+				{
+					fastStringTokenizerDelim.reset(value);
+					int ri = fastStringTokenizerDelim.nextInt();
+					col = fastStringTokenizerDelim.nextInt();
+					cellValue = fastStringTokenizerDelim.nextDouble();
+
+					if(cellValue != 0) {
+						dest.appendValue(ri-_props.getFirstColIndex(), col-_props.getFirstColIndex(), cellValue);
+						lnnz++;
+					}
+					row = Math.max(row, ri);
+				}
+			}
+			finally {
+				IOUtilFunctions.closeSilently(br);
+			}
+			rowPos.setValue(row);
+			return lnnz;
+		}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/RawRow.java b/src/main/java/org/apache/sysds/runtime/iogen/RawRow.java
new file mode 100644
index 0000000..6ef8222
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/RawRow.java
@@ -0,0 +1,558 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+import org.apache.sysds.common.Types;
+import org.apache.sysds.runtime.matrix.data.Pair;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashSet;
+
+public class RawRow {
+	private final String raw;
+	private ArrayList<Integer> numericPositions = new ArrayList<>();
+	private final BitSet numericReserved;
+	private final String numericRaw;
+	private final BitSet reserved;
+	private int numericLastIndex;
+	private int rawLastIndex;
+
+	private Pair<Integer, Integer> resultNumeric;
+
+	public RawRow(String raw, ArrayList<Integer> numericPositions, String numericRaw) {
+		this.raw = raw;
+		this.numericReserved = new BitSet(numericRaw.length());
+		this.numericRaw = numericRaw;
+		this.reserved = new BitSet(numericRaw.length());
+		this.numericPositions = numericPositions;
+
+	}
+
+	public RawRow(String raw) {
+		this.raw = raw;
+		char[] rawChars = raw.toCharArray();
+		StringBuilder sbNumericRaw = new StringBuilder();
+		for(int i = 0; i < rawChars.length; i++) {
+			char ch = rawChars[i];
+			if(Character.isDigit(ch)) {
+				sbNumericRaw.append(ch);
+				numericPositions.add(i);
+			}
+		}
+		numericReserved = new BitSet(numericPositions.size());
+		numericRaw = sbNumericRaw.toString();
+		reserved = new BitSet(raw.length());
+		numericLastIndex = 0;
+		rawLastIndex = 0;
+	}
+
+	public Pair<Integer, Integer> findValue(ValueTrimFormat vtf, boolean forward, boolean update) {
+		Types.ValueType vt = vtf.getValueType();
+		if(vt.isNumeric())
+			return findNumericValue(vtf, forward, update);
+
+		else if(vt == Types.ValueType.STRING)
+			return findStringValue(vtf, forward, update);
+		else if(vt == Types.ValueType.BOOLEAN) {
+			ValueTrimFormat vtfb = new ValueTrimFormat(vtf.getStringOfActualValue());
+			return findStringValue(vtfb, forward, update);
+		}
+		return null;
+	}
+
+	public Pair<Integer, Integer> findValue(ValueTrimFormat vtf, boolean forward) {
+		return findValue(vtf, forward, true);
+	}
+
+	public Pair<Integer, Integer> findSequenceValues(ArrayList<ValueTrimFormat> vtfs, int startIndex, boolean update) {
+		int currentNumericLastIndex = numericLastIndex;
+		int currentRawLastIndex = rawLastIndex;
+		Pair<Integer, Integer> spair = null;
+		Pair<Integer, Integer> epair = null;
+		ValueTrimFormat snode = vtfs.get(0);
+		rawLastIndex = 0;
+		numericLastIndex = 0;
+
+		do {
+			spair = findValue(snode, true, false);
+			if(spair.getKey() != -1) {
+				for(int i = 1; i < vtfs.size(); i++) {
+					epair = findAtValue(vtfs.get(i), rawLastIndex, numericLastIndex, false);
+					if(epair.getKey() == -1)
+						break;
+				}
+				if(epair != null && epair.getKey() != -1)
+					break;
+			}
+			else
+				break;
+		}
+		while(true);
+		if(update && epair != null && epair.getKey() != -1) {
+			reserved.set(spair.getKey(), epair.getKey() + epair.getValue(), true);
+		}
+		else {
+			numericLastIndex = currentNumericLastIndex;
+			rawLastIndex = currentRawLastIndex;
+		}
+
+		if(epair != null && epair.getKey() != -1) {
+			spair.set(spair.getKey(), epair.getKey() + epair.getValue());
+
+		}
+		else
+			spair.set(-1, 0);
+
+		return spair;
+	}
+
+	public Pair<Integer, Integer> findAtValue(ValueTrimFormat vtf, int rawIndex, int numericIndex, boolean update) {
+		if(vtf.getValueType() == Types.ValueType.STRING)
+			return findAtStringValue(vtf, rawIndex, update);
+		else if(vtf.getValueType().isNumeric())
+			return findAtNumericValue(vtf, rawIndex, numericIndex, update);
+		else if(vtf.getValueType() == Types.ValueType.BOOLEAN) {
+			ValueTrimFormat vtfb = new ValueTrimFormat(vtf.getStringOfActualValue());
+			return findAtStringValue(vtfb, rawIndex, update);
+		}
+		else
+			throw new RuntimeException("FindAt just work for fixed length of values!");
+	}
+
+	public Pair<Integer, Integer> findAtValue(ValueTrimFormat vtf, int rawIndex, int numericIndex) {
+		return findAtValue(vtf, rawIndex, numericIndex, true);
+	}
+
+	private Pair<Integer, Integer> findAtStringValue(ValueTrimFormat stf, int index, boolean update) {
+		Pair<Integer, Integer> result = new Pair<>(-1, 0);
+		int length = stf.getStringOfActualValue().length();
+		if(index + length > raw.length() || index <= 0)
+			return result;
+
+		if(reserved.get(index, index + length).isEmpty()) {
+			if(raw.substring(index, index + length).equalsIgnoreCase(stf.getStringOfActualValue())) {
+				result.set(index, length);
+				rawLastIndex = result.getKey() + result.getValue();
+			}
+		}
+		if(result.getKey() != -1 && update) {
+			reserved.set(result.getKey(), result.getKey() + result.getValue(), true);
+		}
+		return result;
+	}
+
+	private Pair<Integer, Integer> findAtNumericValue(ValueTrimFormat ntf, int rawStart, int numericStart,
+		boolean update) {
+		Pair<Integer, Integer> result = new Pair<>(-1, 0);
+		int end = rawStart;
+
+		for(int i = rawStart; i < raw.length(); i++) {
+			if(!reserved.get(i))
+				end++;
+			else
+				break;
+		}
+		boolean flagD = false;
+		StringBuilder sb = new StringBuilder();
+		for(int i = rawStart; i < end; i++) {
+			char ch = raw.charAt(i);
+			if(ch == 'E' || ch == 'e' || ch == '+' || ch == '-') {
+				sb.append(ch);
+			}
+			else if(!flagD && ch == '.') {
+				sb.append(ch);
+				flagD = true;
+			}
+			else if(Character.isDigit(ch))
+				sb.append(ch);
+			else
+				break;
+		}
+		Double value = tryParse(sb.toString());
+		if(value != null) {
+			if(value == ntf.getDoubleActualValue()) {
+				result.setKey(rawStart);
+				result.setValue(sb.length());
+			}
+		}
+
+		if(result.getKey() != -1) {
+			if(update) {
+				for(int i = resultNumeric.getKey() - 1; i >= 0; i--) {
+					if(numericPositions.get(i) >= result.getKey())
+						numericReserved.set(i);
+					else
+						break;
+				}
+
+				for(int i = resultNumeric.getKey() + 1; i < numericPositions.size(); i++) {
+					if(numericPositions.get(i) <= result.getKey() + result.getValue()) {
+						numericReserved.set(i);
+						numericLastIndex = i;
+					}
+					else
+						break;
+				}
+				numericReserved.set(resultNumeric.getKey(), resultNumeric.getKey() + resultNumeric.getValue(), true);
+				reserved.set(result.getKey(), result.getKey() + result.getValue(), true);
+			}
+			else {
+				for(int i = resultNumeric.getKey() + 1; i < numericPositions.size(); i++) {
+					if(numericPositions.get(i) <= result.getKey() + result.getValue()) {
+						numericLastIndex = i;
+					}
+					else
+						break;
+				}
+			}
+			numericLastIndex = Math.max(numericLastIndex, resultNumeric.getKey() + resultNumeric.getValue());
+			rawLastIndex = result.getKey() + result.getValue();
+		}
+		return result;
+	}
+
+	private Pair<Integer, Integer> findStringValue(ValueTrimFormat stf, boolean forward, boolean update) {
+		ArrayList<Pair<Integer, Integer>> unreserved = getRawUnreservedPositions(forward);
+		Pair<Integer, Integer> result = new Pair<>(-1, 0);
+		for(Pair<Integer, Integer> p : unreserved) {
+			int start = p.getKey();
+			int end = p.getValue();
+			String ntfString = stf.getStringOfActualValue();
+			int length = ntfString.length();
+			int index = raw.indexOf(ntfString, start);
+			if(index != -1 && (index <= end - length + 1)) {
+				result.setKey(index);
+				result.setValue(length);
+				rawLastIndex = index + length;
+				if(update)
+					reserved.set(result.getKey(), result.getKey() + result.getValue(), true);
+				break;
+			}
+		}
+		return result;
+	}
+
+	private Pair<Integer, Integer> findNumericValue(ValueTrimFormat ntf, boolean forward, boolean update) {
+		ArrayList<Pair<Integer, Integer>> unreserved = getUnreservedPositions(forward);
+		Pair<Integer, Integer> result = new Pair<>(-1, 0);
+		resultNumeric = new Pair<>(-1, 0);
+		for(Pair<Integer, Integer> p : unreserved) {
+			int start = p.getKey();
+			int end = p.getValue();
+			for(int s = start; s <= end && result.getKey() == -1; ) {
+				String ntfString = ntf.getNString();
+				int length = ntfString.length();
+				int index = numericRaw.indexOf(ntfString, s);
+				if(index == -1 || index > end - length + 1)
+					break;
+				s = index + 1;
+
+				resultNumeric.setValue(length);
+				resultNumeric.setKey(index);
+				int startPos = numericPositions.get(index);
+				int endPos = numericPositions.get(index + length - 1);
+				ntfString = raw.substring(startPos, endPos + 1);
+				Double value = tryParse(ntfString);
+				if(value == null)
+					continue;
+
+				// Choose range of string
+				boolean flagD = false;
+
+				// 1. the range contain '.'
+				// 2. the range contain none numeric chars. In this condition we should terminate checking
+				int d = endPos - startPos - length + 1;
+				if(d == 1) {
+					for(int i = startPos; i <= endPos; i++) {
+						if(raw.charAt(i) == '.') {
+							flagD = true;
+							break;
+						}
+					}
+					if(!flagD)
+						continue;
+					// Check mapping
+					ntfString = raw.substring(startPos, endPos + 1);
+				}
+				else if(d > 1)
+					continue;
+
+				StringBuilder sb = new StringBuilder();
+
+				/* 3. add extra chars if the value at the middle of another value
+					Example: target value= 123
+					source text: 1.123E12,123
+					second "123" value should report
+				*/
+
+				boolean flagPrefix = true;
+				for(int i = startPos - 1; i >= 0 && flagPrefix; i--) {
+					char ch = raw.charAt(i);
+					if(Character.isDigit(ch) && ch != '0')
+						flagPrefix = false;
+					else if(ch == '0')
+						sb.append('0');
+					else if(!flagD && ch == '.') {
+						sb.append(ch);
+						flagD = true;
+					}
+					else if(ch == '+' || ch == '-') {
+						sb.append(ch);
+						break;
+					}
+					else {
+						break;
+					}
+				}
+				if(!flagPrefix)
+					continue;
+
+				sb = sb.reverse();
+				startPos -= sb.length();
+				sb.append(ntfString);
+
+				for(int i = endPos + 1; i < raw.length(); i++) {
+					char ch = raw.charAt(i);
+					if(ch == 'E' || ch == 'e' || ch == '+' || ch == '-') {
+						sb.append(ch);
+					}
+					else if(!flagD && ch == '.') {
+						sb.append(ch);
+						flagD = true;
+					}
+					else if(Character.isDigit(ch))
+						sb.append(ch);
+					else
+						break;
+				}
+				value = tryParse(sb.toString());
+				if(value != null) {
+					if(value == ntf.getDoubleActualValue()) {
+						result.setKey(startPos);
+						result.setValue(sb.length());
+					}
+				}
+			}
+			if(result.getKey() != -1) {
+				break;
+			}
+		}
+		if(result.getKey() != -1) {
+			if(update) {
+				for(int i = resultNumeric.getKey() - 1; i >= 0; i--) {
+					if(numericPositions.get(i) >= result.getKey())
+						numericReserved.set(i);
+					else
+						break;
+				}
+
+				for(int i = resultNumeric.getKey() + 1; i < numericPositions.size(); i++) {
+					if(numericPositions.get(i) <= result.getKey() + result.getValue()) {
+						numericReserved.set(i);
+						numericLastIndex = i;
+					}
+					else
+						break;
+				}
+				numericReserved.set(resultNumeric.getKey(), resultNumeric.getKey() + resultNumeric.getValue(), true);
+				reserved.set(result.getKey(), result.getKey() + result.getValue(), true);
+			}
+			else {
+				for(int i = resultNumeric.getKey() + 1; i < numericPositions.size(); i++) {
+					if(numericPositions.get(i) <= result.getKey() + result.getValue()) {
+						numericLastIndex = i;
+					}
+					else
+						break;
+				}
+			}
+			numericLastIndex = Math.max(numericLastIndex, resultNumeric.getKey() + resultNumeric.getValue());
+			rawLastIndex = result.getKey() + result.getValue();
+		}
+		return result;
+	}
+
+	private ArrayList<Pair<Integer, Integer>> getUnreservedPositions(boolean forward) {
+		ArrayList<Pair<Integer, Integer>> result = new ArrayList<>();
+		int sIndex, eIndex;
+		int size = numericPositions.size();
+		int[] start = {numericLastIndex, 0};
+		int[] end = {size, numericLastIndex};
+		int psize = (forward || rawLastIndex == 0) ? 1 : 2;
+
+		for(int p = 0; p < psize; p++) {
+			for(int i = start[p]; i < end[p]; ) {
+				// skip all reserved indexes
+				for(int j = i; j < end[p]; j++) {
+					if(numericReserved.get(j))
+						i++;
+					else
+						break;
+				}
+				sIndex = i;
+				// Extract unreserved position
+				for(int j = i; j < end[p]; j++) {
+					if(!numericReserved.get(j))
+						i++;
+					else
+						break;
+				}
+				eIndex = i;
+				if(sIndex < eIndex)
+					result.add(new Pair<>(sIndex, eIndex - 1));
+			}
+		}
+		return result;
+	}
+
+	private ArrayList<Pair<Integer, Integer>> getRawUnreservedPositions(boolean forward) {
+		ArrayList<Pair<Integer, Integer>> result = new ArrayList<>();
+		int sIndex, eIndex;
+		int size = raw.length();
+		int[] start = {rawLastIndex, 0};
+		int[] end = {size, rawLastIndex};
+
+		int psize = (forward || rawLastIndex == 0) ? 1 : 2;
+		for(int p = 0; p < psize; p++) {
+
+			for(int i = start[p]; i < end[p]; ) {
+				// skip all reserved indexes
+				for(int j = i; j < end[p]; j++) {
+					if(reserved.get(j))
+						i++;
+					else
+						break;
+				}
+				sIndex = i;
+				// Extract unreserved position
+				for(int j = i; j < end[p]; j++) {
+					if(!reserved.get(j))
+						i++;
+					else
+						break;
+				}
+				eIndex = i;
+				if(sIndex < eIndex)
+					result.add(new Pair<>(sIndex, eIndex - 1));
+			}
+		}
+		return result;
+	}
+
+	private static Double tryParse(String input) {
+		try {
+			return Double.parseDouble(input);
+		}
+		catch(Exception ex) {
+			return null;
+		}
+	}
+
+	public Pair<String, String> getDelims() {
+		Pair<String, String> result = new Pair<>("", "");
+
+		StringBuilder sbAll = new StringBuilder();
+		StringBuilder sbPart = new StringBuilder();
+		String minToken = "";
+		for(int i = 0; i < raw.length(); i++) {
+			if(!reserved.get(i)) {
+				char ch = raw.charAt(i);
+				sbAll.append(ch);
+				sbPart.append(ch);
+			}
+			else {
+				if(sbPart.length() == 0)
+					continue;
+
+				if(minToken.length() == 0 || minToken.length() > sbPart.length())
+					minToken = sbPart.toString();
+
+				sbPart = new StringBuilder();
+			}
+		}
+		result.set(minToken, sbAll.toString());
+		return result;
+	}
+
+	public void resetReserved() {
+		reserved.set(0, raw.length(), false);
+		numericReserved.set(0, numericPositions.size(), false);
+		numericLastIndex = 0;
+		rawLastIndex = 0;
+	}
+
+	public Pair<HashSet<String>, Integer> getDelimsSet() {
+		Pair<HashSet<String>, Integer> result = new Pair<>();
+		StringBuilder sb = new StringBuilder();
+		int minSize = -1;
+		HashSet<String> set = new HashSet<>();
+		for(int i = 0; i < raw.length(); i++) {
+			if(!reserved.get(i)) {
+				char ch = raw.charAt(i);
+				sb.append(ch);
+			}
+			else {
+				if(sb.length() > 0) {
+					set.add(sb.toString());
+					minSize = minSize == -1 ? sb.length() : Math.min(minSize, sb.length());
+				}
+				sb = new StringBuilder();
+			}
+		}
+		result.set(set, minSize);
+		return result;
+	}
+
+	public String getRaw() {
+		return raw;
+	}
+
+	public void setNumericLastIndex(int numericLastIndex) {
+		this.numericLastIndex = numericLastIndex;
+	}
+
+	public void setRawLastIndex(int rawLastIndex) {
+		this.rawLastIndex = rawLastIndex;
+	}
+
+	public RawRow getResetClone() {
+		RawRow clone = new RawRow(raw, numericPositions, numericRaw);
+		clone.setRawLastIndex(0);
+		clone.setNumericLastIndex(0);
+		return clone;
+	}
+
+	public void setLastIndex(int lastIndex) {
+		this.numericLastIndex = lastIndex;
+	}
+
+	public int getNumericLastIndex() {
+		return numericLastIndex;
+	}
+
+	public int getRawLastIndex() {
+		return rawLastIndex;
+	}
+
+	public boolean isMarked() {
+		return !reserved.isEmpty();
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/ReaderMapping.java b/src/main/java/org/apache/sysds/runtime/iogen/ReaderMapping.java
new file mode 100644
index 0000000..c227c52
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/ReaderMapping.java
@@ -0,0 +1,860 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+import org.apache.sysds.common.Types;
+import org.apache.sysds.runtime.io.IOUtilFunctions;
+import org.apache.sysds.runtime.matrix.data.FrameBlock;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.matrix.data.Pair;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+
+public abstract class ReaderMapping {
+
+	protected int[][] mapRow;
+	protected int[][] mapCol;
+	protected boolean symmetric;
+	protected boolean skewSymmetric;
+	protected boolean isUpperTriangular;
+	protected int skewCoefficient;
+	protected final ArrayList<RawRow> sampleRawRows;
+
+	protected boolean mapped;
+	protected static int nrows;
+	protected static int ncols;
+	protected final int nlines;
+	protected int firstRowIndex;
+	protected int firstColIndex;
+
+	protected ValueTrimFormat[][] VTF;
+	protected ValueTrimFormat[][] VTFClone = null;
+
+	public ReaderMapping(String raw) throws Exception {
+		InputStream is = IOUtilFunctions.toInputStream(raw);
+		BufferedReader br = new BufferedReader(new InputStreamReader(is));
+		String value;
+		int nlines = 0;
+		sampleRawRows = new ArrayList<>();
+		while((value = br.readLine()) != null) {
+			sampleRawRows.add(new RawRow(value));
+			nlines++;
+		}
+		this.nlines = nlines;
+		firstColIndex = 0;
+		firstRowIndex = 0;
+	}
+
+	protected abstract boolean isSchemaNumeric();
+
+	protected void cloneSample() {
+		if(VTFClone == null) {
+			VTFClone = new ValueTrimFormat[nrows][ncols];
+			for(int r = 0; r < nrows; r++)
+				for(int c = 0; c < ncols; c++)
+					VTFClone[r][c] = VTF[r][c].getACopy();
+		}
+	}
+
+	protected void retrieveSample() {
+		for(int r = 0; r < nrows; r++)
+			for(int c = 0; c < ncols; c++)
+				VTF[r][c] = VTFClone[r][c].getACopy();
+	}
+
+	protected void transferSampleTriangular(boolean isUpper) throws Exception {
+		if(nrows != ncols)
+			throw new Exception("For upper triangular both Row and Col should be same!");
+
+		for(int r = 0; r < nrows; r++) {
+			if(isUpper) {
+				for(int c = 0; c < r; c++) {
+					VTF[r][c].setNoSet();
+				}
+			}
+			else {
+				for(int c = r + 1; c < ncols; c++) {
+					VTF[r][c].setNoSet();
+				}
+			}
+		}
+	}
+
+	protected void transferSampleSkew(int coefficient) throws Exception {
+		if(coefficient != 1 && coefficient != -1)
+			throw new Exception("The value of Coefficient have to be 1 or -1!");
+
+		for(int r = 0; r < nrows; r++)
+			for(int c = 0; c < ncols; c++) {
+				if(!VTF[r][c].isNotSet() && VTF[r][c].getValueType().isNumeric())
+					VTF[r][c] = new ValueTrimFormat(VTF[r][c].getColIndex(), VTF[r][c].getValueType(),
+						VTF[r][c].getDoubleActualValue() * coefficient);
+			}
+	}
+
+	protected abstract ValueTrimFormat[][] convertSampleTOValueTrimFormat();
+
+	// Matrix Reader Mapping
+	public static class MatrixReaderMapping extends ReaderMapping {
+
+		private MatrixBlock sampleMatrix;
+
+		public MatrixReaderMapping(String raw, MatrixBlock matrix) throws Exception {
+			super(raw);
+			this.sampleMatrix = matrix;
+			nrows = sampleMatrix.getNumRows();
+			ncols = sampleMatrix.getNumColumns();
+			VTF = convertSampleTOValueTrimFormat();
+			runMapping();
+		}
+
+		// Convert: convert each value of a sample matrix to NumberTrimFormat
+		@Override
+		protected ValueTrimFormat[][] convertSampleTOValueTrimFormat() {
+			ValueTrimFormat[][] result = new ValueTrimFormat[nrows][ncols];
+			for(int r = 0; r < nrows; r++)
+				for(int c = 0; c < ncols; c++) {
+					result[r][c] = new ValueTrimFormat(c, Types.ValueType.FP64, sampleMatrix.getValue(r, c));
+				}
+			return result;
+		}
+
+		@Override
+		protected boolean isSchemaNumeric() {
+			return true;
+		}
+
+	}
+
+	// Frame Reader Mapping
+	public static class FrameReaderMapping extends ReaderMapping {
+
+		private FrameBlock sampleFrame;
+		private Types.ValueType[] schema;
+
+		public FrameReaderMapping(String raw, FrameBlock frame) throws Exception {
+			super(raw);
+			this.sampleFrame = frame;
+			nrows = sampleFrame.getNumRows();
+			ncols = sampleFrame.getNumColumns();
+			schema = sampleFrame.getSchema();
+			VTF = convertSampleTOValueTrimFormat();
+			//TODO: set NNZ for Frame !!??
+			runMapping();
+		}
+
+		// Convert: convert each value of a sample Frame to ValueTrimFormat(Number, String, and Boolean)
+		@Override
+		protected ValueTrimFormat[][] convertSampleTOValueTrimFormat() {
+			ValueTrimFormat[][] result = new ValueTrimFormat[nrows][ncols];
+			for(int r = 0; r < nrows; r++)
+				for(int c = 0; c < ncols; c++) {
+					result[r][c] = new ValueTrimFormat(c, schema[c], sampleFrame.get(r, c));
+				}
+			return result;
+		}
+
+		@Override
+		protected boolean isSchemaNumeric() {
+			boolean result = true;
+			for(Types.ValueType vt : schema)
+				result &= vt.isNumeric();
+			return result;
+		}
+	}
+
+	public void runMapping() throws Exception {
+
+		mapped = findMapping();
+		boolean schemaNumeric = isSchemaNumeric();
+		if(!mapped) {
+			// Clone Sample Matrix/Frame
+			cloneSample();
+
+			// Symmetric and Skew-Symmetric check:
+			symmetric = nrows == ncols;
+			skewSymmetric = nrows == ncols && schemaNumeric;
+
+			for(int r = 0; r < nrows; r++) {
+				for(int c = 0; c < ncols; c++) {
+					if(symmetric)
+						symmetric = VTF[r][c].isEqual(VTF[c][r]);
+
+					if(skewSymmetric) {
+						if(r != c)
+							skewSymmetric = VTF[r][c].getDoubleActualValue() == VTF[c][r].getDoubleActualValue() * -1;
+						else
+							skewSymmetric = VTF[r][c].isNotSet();
+					}
+				}
+			}
+
+			boolean isRR = isRowRegular();
+			if(symmetric) {
+				// Lower Triangular
+				isUpperTriangular = false;
+				transferSampleTriangular(isUpperTriangular);
+				mapped = isRR ? findMapping() : findMapping() && verifyRISymmetricMapping(isUpperTriangular);
+
+				// Upper Triangular
+				if(!mapped) {
+					isUpperTriangular = true;
+					retrieveSample();
+					transferSampleTriangular(isUpperTriangular);
+					mapped = isRR ? findMapping() : findMapping() && verifyRISymmetricMapping(isUpperTriangular);
+				}
+			}
+			// Skew-Symmetric check:
+			else if(skewSymmetric) {
+				// Lower Triangular
+				isUpperTriangular = false;
+				transferSampleTriangular(isUpperTriangular);
+				mapped = isRR ? findMapping() : findMapping() && verifyRISymmetricMapping(isUpperTriangular);
+
+				// Lower Triangular Skew
+				if(!mapped) {
+					skewCoefficient = -1;
+					transferSampleSkew(skewCoefficient);
+					mapped = isRR ? findMapping() : findMapping() && verifyRISymmetricMapping(isUpperTriangular);
+				}
+
+				// Upper Triangular
+				if(!mapped) {
+					isUpperTriangular = true;
+					skewCoefficient = 1;
+					retrieveSample();
+					transferSampleTriangular(isUpperTriangular);
+					mapped = isRR ? findMapping() : findMapping() && verifyRISymmetricMapping(isUpperTriangular);
+				}
+				// Upper Triangular Skew
+				if(!mapped) {
+					skewCoefficient = -1;
+					transferSampleSkew(skewCoefficient);
+					mapped = isRR ? findMapping() : findMapping() && verifyRISymmetricMapping(isUpperTriangular);
+				}
+			}
+		}
+	}
+
+	protected boolean findMapping() {
+		mapRow = new int[nrows][ncols];
+		mapCol = new int[nrows][ncols];
+
+		// Set "-1" as default value for all defined matrix
+		for(int r = 0; r < nrows; r++)
+			for(int c = 0; c < ncols; c++)
+				mapRow[r][c] = mapCol[r][c] = -1;
+
+		for(int i = 0; i < nlines; i++) {
+			sampleRawRows.get(i).resetReserved();
+		}
+		int itRow = 0;
+		for(int r = 0; r < nrows; r++) {
+			ArrayList<ValueTrimFormat> vtfRow = new ArrayList<>();
+			for(int i = 0; i < ncols; i++) {
+				if(!VTF[r][i].isNotSet())
+					vtfRow.add(VTF[r][i]);
+			}
+			Collections.sort(vtfRow);
+
+			for(ValueTrimFormat vtf : vtfRow) {
+				int c = vtf.getColIndex();
+				HashSet<Integer> checkedLines = new HashSet<>();
+				while(checkedLines.size() < nlines) {
+					RawRow row = sampleRawRows.get(itRow);
+					Pair<Integer, Integer> mi = row.findValue(vtf, false);
+					if(mi.getKey() != -1) {
+						mapRow[r][c] = itRow;
+						mapCol[r][c] = mi.getKey();
+						break;
+					}
+					else {
+						checkedLines.add(itRow);
+						itRow++;
+						if(itRow == nlines)
+							itRow = 0;
+					}
+				}
+			}
+		}
+		boolean flagMap = true;
+		for(int r = 0; r < nrows && flagMap; r++)
+			for(int c = 0; c < ncols && flagMap; c++)
+				if(mapRow[r][c] == -1 && !VTF[r][c].isNotSet()) {
+					flagMap = false;
+				}
+		return flagMap;
+	}
+
+	private boolean verifyRISymmetricMapping(boolean upperTriangular) {
+
+		boolean result = false;
+		int[] rowIndex = {0, 1, 0, 1};
+		int[] colIndex = {0, 1, 1, 0};
+		for(int i = 0; i < rowIndex.length && !result; i++) {
+			result = verifyRISymmetricMapping(upperTriangular, rowIndex[i], colIndex[i]);
+			if(result) {
+				firstRowIndex = rowIndex[i];
+				firstColIndex = colIndex[i];
+			}
+		}
+		return result;
+	}
+
+	private boolean verifyRISymmetricMapping(boolean upperTriangular, int firstRowIndex, int firstColIndex) {
+
+		HashSet<Integer> checkedRow = new HashSet<>();
+		boolean rcvMapped = true;
+		int selectedIndex;
+
+		for(int r = nrows - 2; r >= 0 && rcvMapped; r--) {
+			selectedIndex = upperTriangular ? Math.min(r + 1, nrows - 1) : Math.max(r - 1, 0);
+			if(r == selectedIndex)
+				break;
+			int lindeIndex = 0;
+			rcvMapped = false;
+			do {
+				if(checkedRow.contains(lindeIndex) || VTF[r][selectedIndex].isNotSet())
+					continue;
+				RawRow row = sampleRawRows.get(lindeIndex).getResetClone();
+				if(isMapRowColValue(row, r + firstRowIndex, selectedIndex + firstColIndex, VTF[r][selectedIndex])) {
+					checkedRow.add(lindeIndex);
+					rcvMapped = true;
+				}
+			}
+			while(++lindeIndex < nlines && !rcvMapped);
+		}
+		return rcvMapped;
+	}
+
+	public final CustomProperties getFormatProperties() throws Exception {
+		CustomProperties ffp;
+		if(isRowRegular()) {
+			ffp = getFileFormatPropertiesOfRRCRMapping();
+			if(ffp == null) {
+				ffp = getFileFormatPropertiesOfRRCIMapping();
+			}
+		}
+		else {
+			ffp = getFileFormatPropertiesOfRIMapping();
+		}
+		return ffp;
+	}
+
+	public final boolean isRowRegular() {
+		int nrows = mapRow.length;
+		int ncols = mapRow[0].length;
+		boolean result = true;
+		int rValue = -1;
+		for(int c = 0; c < ncols; c++) {
+			if(mapRow[0][c] != -1) {
+				rValue = mapRow[0][c];
+				break;
+			}
+		}
+
+		for(int r = 0; r < nrows && result; r++) {
+			for(int c = 0; c < ncols && result; c++) {
+				if(mapRow[r][c] != -1 && mapRow[r][c] != rValue + r) {
+					result = false;
+				}
+			}
+		}
+		return result;
+	}
+
+	/* Get delimiters between two indexes.
+		 Row String:     1,2,3,4,5
+		 Sample Matrix: [1 2 3 4 5 ]
+		 Map Col:       [0 2 4 6 8 ]
+		 result:        ["," "," "," "," ","]
+		*/
+	public final CustomProperties getFileFormatPropertiesOfRRCRMapping() {
+
+		ArrayList<String> rowDelims = new ArrayList<>();
+		HashSet<String> naString = new HashSet<>();
+		String stringToken = null;
+
+		// append all delimiters as a string and then tokenize it
+		for(int r = 0; r < nrows; r++) {
+			RawRow rr = sampleRawRows.get(r);
+			Pair<String, String> pair = rr.getDelims();
+			rowDelims.add(pair.getValue());
+			if(stringToken == null || (pair.getKey().length() > 0 && stringToken.length() > pair.getKey().length()))
+				stringToken = pair.getKey();
+		}
+		if(stringToken.length() == 0)
+			stringToken = rowDelims.get(0);
+		String uniqueDelimiter = null;
+		StringBuilder token = new StringBuilder();
+
+		FastStringTokenizer fastStringTokenizer;
+
+		for(Character ch : stringToken.toCharArray()) {
+			token.append(ch);
+			boolean flagCurrToken = true;
+			HashSet<String> ns = new HashSet<>();
+			fastStringTokenizer = new FastStringTokenizer(token.toString());
+			for(int r = 0; r < nrows; r++) {
+				String row = rowDelims.get(r);
+				fastStringTokenizer.reset(row);
+				ArrayList<String> delimsOfToken = fastStringTokenizer.getTokens();
+
+				// remove numeric NA Strings
+				// This case can appear in Frame DataType
+				for(String s : delimsOfToken) {
+					try {
+						Double.parseDouble(s);
+					}
+					catch(Exception ex) {
+						ns.add(s);
+					}
+				}
+				if(fastStringTokenizer._count != ncols - 1) {
+					flagCurrToken = false;
+					break;
+				}
+			}
+			if(flagCurrToken) {
+				uniqueDelimiter = token.toString();
+				naString = ns;
+			}
+		}
+		if(uniqueDelimiter != null) {
+			CustomProperties ffpgr = new CustomProperties(CustomProperties.GRPattern.Regular, uniqueDelimiter,
+				naString);
+			ffpgr.setDescription("CSV Format Recognized");
+			return ffpgr;
+		}
+		else
+			return null;
+	}
+
+	private static class FastStringTokenizer implements Serializable {
+		private static final long serialVersionUID = -4698672725609750097L;
+		private String _string = null;
+		private String _del = "";
+		private int _pos = -1;
+		private int _count = 0;
+
+		public FastStringTokenizer(String delimiter) {
+			_del = delimiter;
+			reset(null);
+		}
+
+		public void reset(String string) {
+			_string = string;
+			_pos = 0;
+			_count = 0;
+		}
+
+		private String nextToken() {
+			int len = _string.length();
+			int start = _pos;
+
+			//find start (skip over leading delimiters)
+			while(start != -1 && start < len && _del
+				.equals(_string.substring(start, Math.min(start + _del.length(), _string.length())))) {
+				start += _del.length();
+				_count++;
+			}
+
+			//find end (next delimiter) and return
+			if(start < len && start != -1) {
+				_pos = _string.indexOf(_del, start);
+				if(start < _pos && _pos < len) {
+					return _string.substring(start, _pos);
+				}
+				else
+					return _string.substring(start);
+			}
+			//no next token
+			return null;
+		}
+
+		public ArrayList<String> getTokens() {
+			ArrayList<String> tokens = new ArrayList<>();
+			tokens.add("");
+			String token;
+			do {
+				token = nextToken();
+				if(token != null) {
+					tokens.add(token);
+				}
+			}
+			while(token != null);
+			return tokens;
+		}
+	}
+
+	private CustomProperties getFileFormatPropertiesOfRIMapping() {
+
+		int[] rowIndex = {0, 1, 0, 1};
+		int[] colIndex = {0, 1, 1, 0};
+		CustomProperties ffp = null;
+		for(int i = 0; i < rowIndex.length && ffp == null; i++) {
+			ffp = getDelimsOfMapping(rowIndex[i], colIndex[i]);
+			if(ffp != null) {
+				firstRowIndex = rowIndex[i];
+				firstColIndex = colIndex[i];
+			}
+		}
+
+		if(ffp != null) {
+			ffp.setFirstColIndex(firstColIndex);
+			ffp.setFirstRowIndex(firstRowIndex);
+			ffp.setDescription(
+				"Market Matrix Format Recognized: FirstRowIndex: " + firstRowIndex + " and  FirstColIndex: " + firstColIndex);
+		}
+		return ffp;
+	}
+
+	private CustomProperties getDelimsOfMapping(int firstRowIndex, int firstColIndex) {
+
+		//HashSet<Integer> checkedRow = new HashSet<>();
+		HashSet<String> delims = new HashSet<>();
+		int minDelimLength = -1;
+		boolean rcvMapped = false;
+		int selectedRowIndex = nrows - 2;
+		int selectedColIndex = ncols - 1;
+		// select maximum none zero col index
+		for(int c = ncols - 1; c >= 0; c--) {
+			if(!VTF[selectedRowIndex][c].isNotSet()) {
+				selectedColIndex = c;
+				break;
+			}
+		}
+		int lindeIndex = 0;
+		do {
+			RawRow row = sampleRawRows.get(lindeIndex).getResetClone();
+			if(isMapRowColValue(row, selectedRowIndex + firstRowIndex, selectedColIndex + firstColIndex,
+				VTF[selectedRowIndex][selectedColIndex])) {
+				rcvMapped = true;
+
+				Pair<HashSet<String>, Integer> pair = row.getDelimsSet();
+				delims.addAll(pair.getKey());
+				minDelimLength = minDelimLength == -1 ? pair.getValue() : Math.min(minDelimLength, pair.getValue());
+			}
+		}
+		while(++lindeIndex < nlines && !rcvMapped);
+
+		if(!rcvMapped) {
+			return null;
+		}
+		else {
+
+			String uniqueDelim = null;
+			for(int l = 1; l < minDelimLength + 1; l++) {
+				boolean flagToken = true;
+				HashSet<String> token = new HashSet<>();
+				for(String delim : delims) {
+					if(delim.length() % l != 0) {
+						flagToken = false;
+						break;
+					}
+					for(int i = 0; i <= delim.length() - l; i += l)
+						token.add(delim.substring(i, i + l));
+					if(token.size() > 1) {
+						flagToken = false;
+						break;
+					}
+				}
+				if(flagToken) {
+					if(token.size() > 0)
+						uniqueDelim = token.iterator().next();
+					break;
+				}
+			}
+
+			if(uniqueDelim != null) {
+				CustomProperties.GRSymmetry symmetry;
+				if(symmetric)
+					symmetry = CustomProperties.GRSymmetry.SYMMETRIC;
+				else if(skewSymmetric)
+					symmetry = CustomProperties.GRSymmetry.SKEW_SYMMETRIC;
+				else
+					symmetry = CustomProperties.GRSymmetry.GENERAL;
+
+				return new CustomProperties(symmetry, uniqueDelim, firstRowIndex, firstColIndex);
+			}
+			else
+				return null;
+		}
+	}
+
+	public CustomProperties getFileFormatPropertiesOfRRCIMapping() {
+
+		CustomProperties ffplibsvm;
+		int firstColIndex = 0;
+
+		// FirstColIndex = 0
+		ffplibsvm = getDelimsOfRRCIMapping(firstColIndex);
+
+		// FirstColIndex = 1
+		if(ffplibsvm == null) {
+			firstColIndex = 1;
+			ffplibsvm = getDelimsOfRRCIMapping(firstColIndex);
+		}
+
+		if(ffplibsvm != null) {
+			ffplibsvm.setDescription("LibSVM Format Recognized: First Index Started From " + firstColIndex);
+			ffplibsvm.setFirstColIndex(firstColIndex);
+		}
+		return ffplibsvm;
+	}
+
+	private CustomProperties getDelimsOfRRCIMapping(int firstColIndex) {
+		HashMap<String, HashSet<String>> tokens = new HashMap<>();
+		HashSet<String> allTokens = new HashSet<>();
+		int maxNNZCount = 0;
+		int selectedRowIndex = 0;
+		for(int r = 0; r < nrows; r++) {
+			int rnnz = 0;
+			for(int c = 0; c < ncols; c++)
+				if(!VTF[r][c].isNotSet())
+					rnnz++;
+			if(maxNNZCount < rnnz) {
+				maxNNZCount = rnnz;
+				selectedRowIndex = r;
+			}
+		}
+
+		RawRow row = sampleRawRows.get(selectedRowIndex);
+		// For find index delimiter, we need to find all possible "Index Delim Value" tokens
+		for(int c = ncols - 1; c >= 0; c--) {
+			ValueTrimFormat v = VTF[selectedRowIndex][c];
+			if(v.isNotSet())
+				continue;
+
+			String key = (c + firstColIndex) + "," + v.getStringOfActualValue();
+			HashSet<String> token = tokens.computeIfAbsent(key, k -> new HashSet<>());
+			token.addAll(getColIndexValueMappedTokens(row, c + firstColIndex, v));
+			allTokens.addAll(token);
+		}
+
+		//After find all tokens the intersection of tokens is a good candidate for "Index delimiter"
+		// This part of code try to find the intersection of tokens
+		// In some cases like LobSVM label value don't have Index Delim token,
+		// So, we ignored this condition for some values
+		ArrayList<String> missedKeys = new ArrayList<>();
+		HashSet<Integer> labelIndex = new HashSet<>();
+		ArrayList<String> selectedTokens = new ArrayList<>();
+
+		for(String key : tokens.keySet()) {
+			if(tokens.get(key).size() == 0)
+				missedKeys.add(key);
+		}
+		if(missedKeys.size() > 1)
+			return null;
+		else {
+			for(String t : allTokens) {
+				missedKeys.clear();
+				for(String key : tokens.keySet()) {
+					if(!tokens.get(key).contains(t)) {
+						missedKeys.add(key);
+					}
+				}
+				if(missedKeys.size() == 1) {
+					int li = Integer.parseInt(missedKeys.iterator().next().split(",")[0]);
+					labelIndex.add(li);
+					selectedTokens.add(t);
+				}
+			}
+		}
+
+		/* After find index delim token, the next step is find Item Separator
+		 The algorithm for find separator, mark all Indexes, Values and Index Delim on the raw string
+		 Finally the reminder of the text is separator. In some cases(i.e., duplicated values)
+		 there are more than on position for value and this cause wrong matching and finally wrong value
+		 for separator. To avoid this type of problems, first looked for biggest char base size values
+		 (for example a= 123.45 b= 1000000 a will match first because based on VariableTrimFormat algorithm
+		 "a" have 5 char ad the length is 5, but b have 1 char and the length is one).
+		 */
+		String separator = null;
+		String indexSeparator = null;
+		boolean isVerify = false;
+
+		// Just one row of the sample raw is enough for finding item separator. "selectedRowIndex" mentioned
+		// first row of sample raw data
+
+		for(int i = 0; i < selectedTokens.size() && !isVerify; i++) {
+			isVerify = true;
+			indexSeparator = selectedTokens.get(i);
+
+			row = sampleRawRows.get(selectedRowIndex).getResetClone();
+			// find all values
+			ArrayList<ValueTrimFormat> vtfValueList = new ArrayList<>();
+			ValueTrimFormat vtfIndexDelim = new ValueTrimFormat(indexSeparator);
+			for(int c = 0; c < ncols; c++) {
+				if(!VTF[selectedRowIndex][c].isNotSet() && !labelIndex.contains(c + firstColIndex)) {
+					vtfValueList.add(VTF[selectedRowIndex][c].getACopy());
+				}
+			}
+			Collections.sort(vtfValueList);
+
+			for(ValueTrimFormat vtf : vtfValueList) {
+				ArrayList<ValueTrimFormat> indexDelimValue = new ArrayList<>();
+				ValueTrimFormat vtfColIndex = new ValueTrimFormat(vtf.getColIndex() + firstColIndex);
+				indexDelimValue.add(vtfColIndex);
+				indexDelimValue.add(vtfIndexDelim);
+				indexDelimValue.add(vtf);
+				row.findSequenceValues(indexDelimValue, 0, true);
+			}
+			for(Integer li : labelIndex) {
+				row.findValue(VTF[selectedRowIndex][li - firstColIndex], false);
+			}
+			//row.print();
+			separator = row.getDelims().getKey();
+			if(separator == null) {
+				isVerify = false;
+				break;
+			}
+		}
+		if(isVerify) {
+			return new CustomProperties(CustomProperties.GRPattern.Regular, separator, indexSeparator, firstColIndex);
+		}
+		else
+			return null;
+	}
+
+	private static boolean isMapRowColValue(RawRow rawrow, int row, int col, ValueTrimFormat value) {
+		ValueTrimFormat vtfRow = new ValueTrimFormat(row);
+		ValueTrimFormat vtfCol = new ValueTrimFormat(col);
+		ValueTrimFormat vtfValue = value.getACopy();
+		boolean mapped = true;
+
+		byte hasZero = 0b000;
+		if(vtfRow.isNotSet())
+			hasZero |= 0b100;
+
+		if(vtfCol.isNotSet())
+			hasZero |= 0b010;
+
+		if(vtfValue.isNotSet())
+			hasZero |= 0b001;
+
+		ValueTrimFormat[] order = new ValueTrimFormat[3];
+		/* valid formats:
+		   Row, Col, Value
+		1. 0  , 0  , Value  >> 110 -> 6
+		2. 0  , col, Value  >> 100 -> 4
+		3. row, 0  , value  >> 010 -> 2
+		4. row, col, value  >> 000 -> 0
+		-----------------   >> otherwise the value is not set.
+		 */
+		switch(hasZero) {
+			case 0:
+				order[0] = vtfRow;
+				order[1] = vtfCol;
+				order[2] = vtfValue;
+				break;
+
+			case 2:
+				order[0] = vtfRow;
+				order[1] = vtfValue;
+				order[2] = vtfCol;
+				break;
+
+			case 4:
+				order[0] = vtfCol;
+				order[1] = vtfValue;
+				order[2] = vtfRow;
+				break;
+
+			case 6:
+				order[0] = vtfValue;
+				order[1] = vtfRow;
+				order[2] = vtfCol;
+				break;
+			default:
+				throw new RuntimeException("Not set values can't be find on a string");
+		}
+		for(ValueTrimFormat vtf : order) {
+			if(rawrow.findValue(vtf, false).getKey() == -1) {
+				mapped = false;
+				break;
+			}
+		}
+		return mapped;
+
+	}
+
+	private static HashSet<String> getColIndexValueMappedTokens(RawRow rawrow, int col, ValueTrimFormat value) {
+		ValueTrimFormat vtfColIndex = new ValueTrimFormat(col);
+		ValueTrimFormat vtfColValue = value.getACopy();
+		Pair<Integer, Integer> pairCol;
+		Pair<Integer, Integer> pairValue;
+		HashSet<String> tokens = new HashSet<>();
+		RawRow row = rawrow.getResetClone();
+		int lastIndex = 0;
+		int lastTokenStart = -1;
+		int lastTokenEnd = -1;
+		int lastTokenID = -1;
+		do {
+			row.resetReserved();
+			row.setLastIndex(lastIndex);
+			pairCol = row.findValue(vtfColIndex, true);
+			if(pairCol.getKey() == -1)
+				break;
+
+			lastIndex = row.getNumericLastIndex();
+
+			pairValue = row.findValue(vtfColValue, true);
+			if(pairValue.getKey() == -1)
+				break;
+
+			int tl = pairValue.getKey() - pairCol.getKey() + pairCol.getValue();
+			if(tl > 0) {
+
+				if(lastTokenID == -1)
+					lastTokenID = pairValue.getKey();
+
+				if(lastTokenID != pairValue.getKey()) {
+					String token = row.getRaw().substring(lastTokenStart, lastTokenEnd);
+					tokens.add(token);
+				}
+
+				lastTokenStart = pairCol.getKey() + pairCol.getValue();
+				lastTokenEnd = pairValue.getKey();
+			}
+		}
+		while(true);
+		if(lastTokenEnd - lastTokenStart > 0) {
+			String token = row.getRaw().substring(lastTokenStart, lastTokenEnd);
+			tokens.add(token);
+		}
+		return tokens;
+	}
+
+	public boolean isSymmetric() {
+		return symmetric;
+	}
+
+	public boolean isMapped() {
+		return mapped;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/SampleProperties.java b/src/main/java/org/apache/sysds/runtime/iogen/SampleProperties.java
new file mode 100644
index 0000000..06b1afe
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/SampleProperties.java
@@ -0,0 +1,79 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.common.Types;
+import org.apache.sysds.runtime.io.FileFormatProperties;
+import org.apache.sysds.runtime.matrix.data.FrameBlock;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+
+public class SampleProperties extends FileFormatProperties {
+
+	protected static final Log LOG = LogFactory.getLog(CustomProperties.class.getName());
+
+	private String sampleRaw;
+	private MatrixBlock sampleMatrix;
+	private FrameBlock sampleFrame;
+	private Types.DataType dataType;
+
+	public SampleProperties(String sampleRaw) {
+		this.sampleRaw = sampleRaw;
+	}
+
+	public SampleProperties(String sampleRaw, MatrixBlock sampleMatrix) {
+		this.sampleRaw = sampleRaw;
+		this.sampleMatrix = sampleMatrix;
+		this.dataType = Types.DataType.MATRIX;
+	}
+
+	public SampleProperties(String sampleRaw, FrameBlock sampleFrame) {
+		this.sampleRaw = sampleRaw;
+		this.sampleFrame = sampleFrame;
+		this.dataType = Types.DataType.FRAME;
+	}
+
+	public String getSampleRaw() {
+		return sampleRaw;
+	}
+
+	public MatrixBlock getSampleMatrix() {
+		return sampleMatrix;
+	}
+
+	public FrameBlock getSampleFrame() {
+		return sampleFrame;
+	}
+
+	public Types.DataType getDataType() {
+		return dataType;
+	}
+
+	public void setSampleMatrix(MatrixBlock sampleMatrix) {
+		this.sampleMatrix = sampleMatrix;
+		dataType = Types.DataType.MATRIX;
+	}
+
+	public void setSampleFrame(FrameBlock sampleFrame) {
+		this.sampleFrame = sampleFrame;
+		dataType = Types.DataType.FRAME;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/iogen/ValueTrimFormat.java b/src/main/java/org/apache/sysds/runtime/iogen/ValueTrimFormat.java
new file mode 100644
index 0000000..5f5f20a
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/iogen/ValueTrimFormat.java
@@ -0,0 +1,228 @@
+/*
+ * 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.sysds.runtime.iogen;
+
+import org.apache.sysds.common.Types;
+import org.apache.sysds.runtime.util.UtilFunctions;
+
+import java.text.DecimalFormat;
+
+public class ValueTrimFormat implements Comparable<ValueTrimFormat> {
+
+	// save the col index of the value on the Matrix.
+	// We need this value when we want to reorder matrix cols
+	private final int colIndex;
+	private Object actualValue;
+
+	// Convert all numeric values(i.e., double, float, int, long, ...) to number trim format
+	public char S; // signe of value "+" or "-"
+	private char[] N; // array of none zero chars. Example: value = 0.00012345, N = [1,2,3,4,5]
+	private String NString;
+	private Types.ValueType valueType;
+
+	public ValueTrimFormat(int actualValue) {
+		this(-1, Types.ValueType.INT32, actualValue);
+	}
+
+	public ValueTrimFormat(String actualValue) {
+		this.valueType = Types.ValueType.STRING;
+		this.actualValue = actualValue;
+		this.colIndex = -1;
+	}
+
+	public ValueTrimFormat(int colIndex, Types.ValueType vt, Object o) {
+		this.valueType = vt;
+		this.colIndex = colIndex;
+		this.actualValue = o;
+		if(vt.isNumeric()) {
+			double value = UtilFunctions.getDouble(o);
+
+			// remove scientific format
+			DecimalFormat decimalFormat = new DecimalFormat("0.000000000000000000000000000000");
+			String stringValue = decimalFormat.format(value);
+			if(value == 0) {
+				S = '+';
+				N = new char[] {'0'};
+			}
+			else {
+				S = (value < 0) ? '-' : '+';
+				if((o instanceof Long || o instanceof Integer) && stringValue.contains(".")) {
+					stringValue = stringValue.substring(0, stringValue.indexOf("."));
+				}
+				numberTrimFormat(stringValue);
+			}
+			StringBuilder s = new StringBuilder();
+			for(Character c : N)
+				s.append(c);
+			NString = s.toString();
+		}
+		else if(vt != Types.ValueType.STRING && vt != Types.ValueType.BOOLEAN) {
+			throw new RuntimeException("Don't support  value type format!");
+		}
+	}
+
+	private void numberTrimFormat(String stringValue) {
+		if(stringValue.charAt(0) == '+' || stringValue.charAt(0) == '-')
+			stringValue = stringValue.substring(1);
+
+		int length = stringValue.length();
+		int firstNZ = -1;
+		int lastNZ = -1;
+		for(int i = 0; i < length; i++) {
+			char fChar = stringValue.charAt(i);
+			char lChar = stringValue.charAt(length - i - 1);
+			if(Character.isDigit(fChar) && fChar != '0' && firstNZ == -1)
+				firstNZ = i;
+
+			if(Character.isDigit(lChar) && lChar != '0' && lastNZ == -1)
+				lastNZ = length - i;
+
+			if(firstNZ > 0 && lastNZ > 0)
+				break;
+		}
+		String subValue = stringValue.substring(firstNZ, lastNZ);
+		int dotLength = subValue.contains(".") ? 1 : 0;
+		N = new char[lastNZ - firstNZ - dotLength];
+		int index = 0;
+		for(Character c : subValue.toCharArray()) {
+			if(c != '.')
+				N[index++] = c;
+		}
+	}
+
+	public double getDoubleActualValue() {
+		return UtilFunctions.getDouble(actualValue);
+	}
+
+	// Get a copy of value
+	public ValueTrimFormat getACopy() {
+		ValueTrimFormat copy = null;
+		if(valueType.isNumeric()) {
+			copy = new ValueTrimFormat(colIndex, valueType, getDoubleActualValue());
+			copy.S = S;
+			copy.N = N;
+		}
+		else {
+			copy = new ValueTrimFormat(colIndex, valueType, actualValue);
+		}
+		return copy;
+	}
+
+	// Check the value is a not set value
+	public boolean isNotSet() {
+
+		if(this.valueType == Types.ValueType.STRING)
+			return actualValue == null || ((String) actualValue).length() == 0;
+		else if(this.valueType.isNumeric())
+			return getDoubleActualValue() == 0;
+		else if(this.valueType == Types.ValueType.BOOLEAN)
+			return actualValue == null || !((Boolean) actualValue);
+		return true;
+	}
+
+	// Set as NoSet
+	public void setNoSet() {
+		if(this.valueType == Types.ValueType.STRING)
+			actualValue = "";
+		else if(this.valueType.isNumeric()) {
+			actualValue = (double) 0;
+			S = '+';
+			N = new char[] {'0'};
+			NString = null;
+		}
+		else if(this.valueType == Types.ValueType.BOOLEAN)
+			actualValue = null;
+	}
+
+	// Get String of actual value
+	public String getStringOfActualValue() {
+		return UtilFunctions.objectToString(actualValue);
+	}
+
+	public boolean isEqual(ValueTrimFormat vtf) {
+		if(vtf.getValueType() != this.getValueType())
+			return false;
+		else if(vtf.getValueType() == Types.ValueType.FP32)
+			return ((Float) this.actualValue).compareTo((Float) vtf.actualValue) == 0;
+		return UtilFunctions.compareTo(valueType, this.actualValue, vtf.actualValue) == 0;
+	}
+
+	public int getColIndex() {
+		return colIndex;
+	}
+
+	private static int getLength(ValueTrimFormat vtf) {
+		Types.ValueType vt = vtf.valueType;
+		int len = -1;
+		if(vt == Types.ValueType.STRING )
+			len = vtf.getStringOfActualValue().length();
+		else if(vt == Types.ValueType.BOOLEAN)
+			len = 1;
+		return len;
+	}
+
+	@Override
+	public int compareTo(ValueTrimFormat vtf) {
+		Types.ValueType vt = vtf.valueType;
+		if(vt.isNumeric() && this.valueType.isNumeric()) {
+			return compareNumericVTF(vtf, this);
+		}
+		else if(vt.isNumeric() && this.valueType == Types.ValueType.STRING) {
+			return -1;
+		}
+		else if(vt == Types.ValueType.STRING && this.valueType.isNumeric()) {
+			try {
+				Double d = Double.parseDouble(vtf.getStringOfActualValue());
+				ValueTrimFormat vtfs = new ValueTrimFormat(-1,Types.ValueType.FP64,d);
+				return compareNumericVTF(vtfs, this);
+			}
+			catch(Exception exception){
+				return 1;
+			}
+		}
+		else
+			return Integer.compare(getLength(vtf), getLength(this));
+	}
+
+	private static int compareNumericVTF(ValueTrimFormat vtf1, ValueTrimFormat vtf2){
+		double dv1 = vtf1.getDoubleActualValue();
+		double dv2 = vtf2.getDoubleActualValue();
+		int vc = Double.compare(dv1, dv2);
+
+		if(vc == 0)
+			return 0;
+
+		int s1 = dv1 >= 0 ? 0 : 1;
+		int s2 = dv2 >= 0 ? 0 : 1;
+		int nc = Integer.compare(vtf1.N.length + s1, vtf2.N.length + s2);
+		if(nc == 0)
+			return Double.compare(Math.abs(dv1), Math.abs(dv2));
+		else
+			return nc;
+	}
+
+	public String getNString() {
+		return NString;
+	}
+
+	public Types.ValueType getValueType() {
+		return valueType;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/util/DataConverter.java b/src/main/java/org/apache/sysds/runtime/util/DataConverter.java
index 841c6d7..6850a97 100644
--- a/src/main/java/org/apache/sysds/runtime/util/DataConverter.java
+++ b/src/main/java/org/apache/sysds/runtime/util/DataConverter.java
@@ -76,12 +76,12 @@ import org.apache.sysds.runtime.meta.DataCharacteristics;
  * This class provides methods to read and write matrix blocks from to HDFS using different data formats.
  * Those functionalities are used especially for CP read/write and exporting in-memory matrices to HDFS
  * (before executing MR jobs).
- * 
+ *
  */
 public class DataConverter {
 	// private static final Log LOG = LogFactory.getLog(DataConverter.class.getName());
 	private static final String DELIM = " ";
-	
+
 	//////////////
 	// READING and WRITING of matrix blocks to/from HDFS
 	// (textcell, binarycell, binaryblock)
@@ -96,7 +96,7 @@ public class DataConverter {
 		throws IOException {
 		writeMatrixToHDFS(mat, dir, fmt, dc, -1, null, false);
 	}
-	
+
 	public static void writeMatrixToHDFS(MatrixBlock mat, String dir, FileFormat fmt, DataCharacteristics dc, int replication, FileFormatProperties formatProperties, boolean diag)
 		throws IOException {
 		MatrixWriter writer = MatrixWriterFactory.createMatrixWriter( fmt, replication, formatProperties );
@@ -104,7 +104,7 @@ public class DataConverter {
 	}
 
 	public static void writeTensorToHDFS(TensorBlock tensor, String dir, FileFormat fmt, DataCharacteristics dc)
-			throws IOException {
+		throws IOException {
 		TensorWriter writer = TensorWriterFactory.createTensorWriter(fmt);
 		int blen = dc.getBlocksize();
 		writer.writeTensorToHDFS(tensor, dir, blen);
@@ -112,30 +112,30 @@ public class DataConverter {
 
 	public static MatrixBlock readMatrixFromHDFS(String dir, FileFormat fmt, long rlen, long clen, int blen, boolean localFS)
 		throws IOException
-	{	
+	{
 		ReadProperties prop = new ReadProperties();
-		
+
 		prop.path = dir;
 		prop.fmt = fmt;
 		prop.rlen = rlen;
 		prop.clen = clen;
 		prop.blen = blen;
 		prop.localFS = localFS;
-		
+
 		return readMatrixFromHDFS(prop);
 	}
 
 	public static MatrixBlock readMatrixFromHDFS(String dir, FileFormat fmt, long rlen, long clen, int blen)
 		throws IOException
-	{	
+	{
 		ReadProperties prop = new ReadProperties();
-		
+
 		prop.path = dir;
 		prop.fmt = fmt;
 		prop.rlen = rlen;
 		prop.clen = clen;
 		prop.blen = blen;
-		
+
 		return readMatrixFromHDFS(prop);
 	}
 
@@ -143,23 +143,23 @@ public class DataConverter {
 		throws IOException
 	{
 		ReadProperties prop = new ReadProperties();
-		
+
 		prop.path = dir;
 		prop.fmt = fmt;
 		prop.rlen = rlen;
 		prop.clen = clen;
 		prop.blen = blen;
 		prop.expectedNnz = expectedNnz;
-		
+
 		return readMatrixFromHDFS(prop);
 	}
 
-	public static MatrixBlock readMatrixFromHDFS(String dir, FileFormat fmt, long rlen, long clen, 
-			int blen, long expectedNnz, boolean localFS)
+	public static MatrixBlock readMatrixFromHDFS(String dir, FileFormat fmt, long rlen, long clen,
+		int blen, long expectedNnz, boolean localFS)
 		throws IOException
 	{
 		ReadProperties prop = new ReadProperties();
-		
+
 		prop.path = dir;
 		prop.fmt = fmt;
 		prop.rlen = rlen;
@@ -167,16 +167,16 @@ public class DataConverter {
 		prop.blen = blen;
 		prop.expectedNnz = expectedNnz;
 		prop.localFS = localFS;
-		
+
 		return readMatrixFromHDFS(prop);
 	}
 
-	public static MatrixBlock readMatrixFromHDFS(String dir, FileFormat fmt, long rlen, long clen, 
-			int blen, long expectedNnz, FileFormatProperties formatProperties)
-	throws IOException
+	public static MatrixBlock readMatrixFromHDFS(String dir, FileFormat fmt, long rlen, long clen,
+		int blen, long expectedNnz, FileFormatProperties formatProperties)
+		throws IOException
 	{
 		ReadProperties prop = new ReadProperties();
-		
+
 		prop.path = dir;
 		prop.fmt = fmt;
 		prop.rlen = rlen;
@@ -184,12 +184,12 @@ public class DataConverter {
 		prop.blen = blen;
 		prop.expectedNnz = expectedNnz;
 		prop.formatProperties = formatProperties;
-		
+
 		return readMatrixFromHDFS(prop);
 	}
 
 	public static TensorBlock readTensorFromHDFS(String dir, FileFormat fmt, long[] dims, int blen,
-			ValueType[] schema) throws IOException {
+		ValueType[] schema) throws IOException {
 		TensorBlock ret;
 		try {
 			TensorReader reader = TensorReaderFactory.createTensorReader(fmt);
@@ -201,34 +201,34 @@ public class DataConverter {
 		}
 		return ret;
 	}
-	
+
 	/**
 	 * Core method for reading matrices in format textcell, matrixmarket, binarycell, or binaryblock 
 	 * from HDFS into main memory. For expected dense matrices we directly copy value- or block-at-a-time 
 	 * into the target matrix. In contrast, for sparse matrices, we append (column-value)-pairs and do a 
 	 * final sort if required in order to prevent large reorg overheads and increased memory consumption 
 	 * in case of unordered inputs.  
-	 * 
+	 *
 	 * DENSE MxN input:
 	 *  * best/average/worst: O(M*N)
 	 * SPARSE MxN input
 	 *  * best (ordered, or binary block w/ clen&lt;=blen): O(M*N)
 	 *  * average (unordered): O(M*N*log(N))
 	 *  * worst (descending order per row): O(M * N^2)
-	 * 
+	 *
 	 * NOTE: providing an exact estimate of 'expected sparsity' can prevent a full copy of the result
 	 * matrix block (required for changing sparse-&gt;dense, or vice versa)
-	 * 
+	 *
 	 * @param prop read properties
 	 * @return matrix block
 	 * @throws IOException if IOException occurs
 	 */
-	public static MatrixBlock readMatrixFromHDFS(ReadProperties prop) 
+	public static MatrixBlock readMatrixFromHDFS(ReadProperties prop)
 		throws IOException
-	{	
+	{
 		//Timing time = new Timing(true);
-		
-		//core matrix reading 
+
+		//core matrix reading
 		MatrixBlock ret = null;
 		try {
 			MatrixReader reader = MatrixReaderFactory.createMatrixReader(prop);
@@ -237,21 +237,21 @@ public class DataConverter {
 		catch(DMLRuntimeException rex)
 		{
 			throw new IOException(rex);
-		}	
-		
+		}
+
 		//System.out.println("read matrix ("+prop.rlen+","+prop.clen+","+ret.getNonZeros()+") in "+time.stop());
-				
+
 		return ret;
 	}
 
-	
+
 	//////////////
-	// Utils for CREATING and COPYING matrix blocks 
+	// Utils for CREATING and COPYING matrix blocks
 	///////
-	
+
 	/**
-	 * Creates a two-dimensional double matrix of the input matrix block. 
-	 * 
+	 * Creates a two-dimensional double matrix of the input matrix block.
+	 *
 	 * @param mb matrix block
 	 * @return 2d double array
 	 */
@@ -284,8 +284,8 @@ public class DataConverter {
 	{
 		int rows = mb.getNumRows();
 		int cols = mb.getNumColumns();
-		boolean[] ret = new boolean[rows*cols]; //false-initialized 
-		
+		boolean[] ret = new boolean[rows*cols]; //false-initialized
+
 		if( mb.getNonZeros() > 0 )
 		{
 			if( mb.isInSparseFormat() )
@@ -303,7 +303,7 @@ public class DataConverter {
 						ret[cix] = (mb.getValueDenseUnsafe(i, j) != 0.0);
 			}
 		}
-		
+
 		return ret;
 	}
 
@@ -311,7 +311,7 @@ public class DataConverter {
 		int rows = mb.getNumRows();
 		int cols = mb.getNumColumns();
 		int[] ret = new int[rows*cols]; //0-initialized
-		if( mb.isEmptyBlock(false) ) 
+		if( mb.isEmptyBlock(false) )
 			return ret;
 		if( mb.isInSparseFormat() ) {
 			Iterator<IJV> iter = mb.getSparseBlockIterator();
@@ -328,12 +328,12 @@ public class DataConverter {
 		}
 		return ret;
 	}
-	
+
 	public static long[] convertToLongVector( MatrixBlock mb) {
 		int rows = mb.getNumRows();
 		int cols = mb.getNumColumns();
 		long[] ret = new long[rows*cols]; //0-initialized
-		if( mb.isEmptyBlock(false) ) 
+		if( mb.isEmptyBlock(false) )
 			return ret;
 		if( mb.isInSparseFormat() ) {
 			Iterator<IJV> iter = mb.getSparseBlockIterator();
@@ -350,17 +350,17 @@ public class DataConverter {
 		}
 		return ret;
 	}
-	
+
 	public static DenseBlock convertToDenseBlock(MatrixBlock mb) {
 		return convertToDenseBlock(mb, true);
 	}
-	
+
 	public static DenseBlock convertToDenseBlock(MatrixBlock mb, boolean deep) {
 		int rows = mb.getNumRows();
 		int cols = mb.getNumColumns();
-		DenseBlock ret = (!mb.isInSparseFormat() && mb.isAllocated() && !deep) ? 
+		DenseBlock ret = (!mb.isInSparseFormat() && mb.isAllocated() && !deep) ?
 			mb.getDenseBlock() : DenseBlockFactory.createDenseBlock(rows, cols); //0-initialized
-		
+
 		if( !mb.isEmptyBlock(false) ) {
 			if( mb.isInSparseFormat() ) {
 				Iterator<IJV> iter = mb.getSparseBlockIterator();
@@ -373,28 +373,28 @@ public class DataConverter {
 				ret.set(mb.getDenseBlock());
 			}
 		}
-		
+
 		return ret;
 	}
 
 	public static double[] convertToDoubleVector(MatrixBlock mb) {
 		return convertToDoubleVector(mb, true);
 	}
-	
+
 	public static double[] convertToDoubleVector( MatrixBlock mb, boolean deep ) {
 		return convertToDoubleVector(mb, deep, false);
 	}
-	
+
 	public static double[] convertToDoubleVector( MatrixBlock mb, boolean deep, boolean allowNull )
 	{
 		if( mb.isEmpty() && allowNull )
 			return null;
-		
+
 		int rows = mb.getNumRows();
 		int cols = mb.getNumColumns();
-		double[] ret = (!mb.isInSparseFormat() && mb.isAllocated() && !deep) ? 
+		double[] ret = (!mb.isInSparseFormat() && mb.isAllocated() && !deep) ?
 			mb.getDenseBlockValues() : new double[rows*cols]; //0-initialized
-		
+
 		if( !mb.isEmptyBlock(false) ) {
 			if( mb.isInSparseFormat() ) {
 				Iterator<IJV> iter = mb.getSparseBlockIterator();
@@ -408,7 +408,7 @@ public class DataConverter {
 				System.arraycopy(mb.getDenseBlockValues(), 0, ret, 0, rows*cols);
 			}
 		}
-		
+
 		return ret;
 	}
 
@@ -418,7 +418,7 @@ public class DataConverter {
 		int cols = mb.getNumColumns();
 		long nnz = mb.getNonZeros();
 		ArrayList<Double> ret = new ArrayList<>();
-		
+
 		if( mb.isInSparseFormat() )
 		{
 			Iterator<IJV> iter = mb.getSparseBlockIterator();
@@ -435,13 +435,13 @@ public class DataConverter {
 				for( int j=0; j<cols; j++ )
 					ret.add( mb.getValueDenseUnsafe(i, j) );
 		}
-				
+
 		return ret;
 	}
-	
+
 	/**
 	 * Creates a dense Matrix Block and copies the given double matrix into it.
-	 * 
+	 *
 	 * @param data 2d double array
 	 * @return matrix block
 	 */
@@ -450,21 +450,21 @@ public class DataConverter {
 		int cols = (rows > 0)? data[0].length : 0;
 		MatrixBlock mb = new MatrixBlock(rows, cols, false);
 		try
-		{ 
+		{
 			//copy data to mb (can be used because we create a dense matrix)
 			mb.init( data, rows, cols );
-		} 
+		}
 		catch (Exception e){} //can never happen
-		
+
 		//check and convert internal representation
 		mb.examSparsity();
-		
+
 		return mb;
 	}
 
 	/**
 	 * Converts an Integer matrix to an MatrixBlock
-	 * 
+	 *
 	 * @param data Int matrix input that is converted to double MatrixBlock
 	 * @return The matrixBlock constructed.
 	 */
@@ -484,7 +484,7 @@ public class DataConverter {
 
 	/**
 	 * Creates a dense Matrix Block and copies the given double vector into it.
-	 * 
+	 *
 	 * @param data double array
 	 * @param columnVector if true, create matrix with single column. if false, create matrix with single row
 	 * @return matrix block
@@ -507,14 +507,14 @@ public class DataConverter {
 			nrows = Math.max( nrows, index.getRowIndex() );
 			ncols = Math.max( ncols, index.getColumnIndex() );
 		}
-		
+
 		// convert to matrix block
 		return convertToMatrixBlock(map, (int)nrows, (int)ncols);
 	}
-	
+
 	/**
 	 * NOTE: this method also ensures the specified matrix dimensions
-	 * 
+	 *
 	 * @param map map of matrix index keys and double values
 	 * @param rlen number of rows
 	 * @param clen number of columns
@@ -525,7 +525,7 @@ public class DataConverter {
 		int nnz = map.size();
 		boolean sparse = MatrixBlock.evalSparseFormatInMemory(rlen, clen, nnz);
 		MatrixBlock mb = new MatrixBlock(rlen, clen, sparse, nnz);
-		
+
 		// copy map values into new block
 		if( sparse ) //SPARSE <- cells
 		{
@@ -539,14 +539,14 @@ public class DataConverter {
 				if( value != 0 && rix<=rlen && cix<=clen )
 					mb.appendValue( rix-1, cix-1, value );
 			}
-			
+
 			//sort sparse target representation
 			mb.sortSparseRows();
 		}
 		else  //DENSE <- cells
 		{
-			//directly insert cells into dense target 
-			for( Entry<MatrixIndexes,Double> e : map.entrySet() ) 
+			//directly insert cells into dense target
+			for( Entry<MatrixIndexes,Double> e : map.entrySet() )
 			{
 				MatrixIndexes index = e.getKey();
 				double value = e.getValue();
@@ -556,7 +556,7 @@ public class DataConverter {
 					mb.quickSetValue( rix-1, cix-1, value );
 			}
 		}
-		
+
 		return mb;
 	}
 
@@ -565,14 +565,14 @@ public class DataConverter {
 		// compute dimensions from the map
 		int nrows = (int)map.getMaxRow();
 		int ncols = (int)map.getMaxColumn();
-		
+
 		// convert to matrix block
 		return convertToMatrixBlock(map, nrows, ncols);
 	}
-	
+
 	/**
 	 * NOTE: this method also ensures the specified matrix dimensions
-	 * 
+	 *
 	 * @param map ?
 	 * @param rlen number of rows
 	 * @param clen number of columns
@@ -582,28 +582,28 @@ public class DataConverter {
 	{
 		return map.toMatrixBlock(rlen, clen);
 	}
-	
+
 	/**
-	 * Converts a frame block with arbitrary schema into a matrix block. 
-	 * Since matrix block only supports value type double, we do a best 
-	 * effort conversion of non-double types which might result in errors 
+	 * Converts a frame block with arbitrary schema into a matrix block.
+	 * Since matrix block only supports value type double, we do a best
+	 * effort conversion of non-double types which might result in errors
 	 * for non-numerical data.
-	 * 
+	 *
 	 * @param frame frame block
 	 * @return matrix block
 	 */
-	public static MatrixBlock convertToMatrixBlock(FrameBlock frame) 
+	public static MatrixBlock convertToMatrixBlock(FrameBlock frame)
 	{
 		int m = frame.getNumRows();
 		int n = frame.getNumColumns();
 		MatrixBlock mb = new MatrixBlock(m, n, false);
 		mb.allocateDenseBlock();
-		
+
 		ValueType[] schema = frame.getSchema();
 		int dFreq = UtilFunctions.frequency(schema, ValueType.FP64);
-		
+
 		if( dFreq == schema.length ) {
-			// special case double schema (without cell-object creation, 
+			// special case double schema (without cell-object creation,
 			// cache-friendly row-column copy)
 			double[][] a = new double[n][];
 			for( int j=0; j<n; j++ )
@@ -637,29 +637,29 @@ public class DataConverter {
 			}
 			mb.setNonZeros(lnnz);
 		}
-		else { 
+		else {
 			//general case
-			for( int i=0; i<frame.getNumRows(); i++ ) 
+			for( int i=0; i<frame.getNumRows(); i++ )
 				for( int j=0; j<frame.getNumColumns(); j++ ) {
 					mb.appendValue(i, j, UtilFunctions.objectToDouble(
 						schema[j], frame.get(i, j)));
 				}
 		}
-		
+
 		//post-processing
 		mb.examSparsity();
-		
+
 		return mb;
 	}
-	
+
 	/**
 	 * Converts a frame block with arbitrary schema into a two dimensional
-	 * string array. 
-	 * 
+	 * string array.
+	 *
 	 * @param frame frame block
 	 * @return 2d string array
 	 */
-	public static String[][] convertToStringFrame(FrameBlock frame) 
+	public static String[][] convertToStringFrame(FrameBlock frame)
 	{
 		String[][] ret = new String[frame.getNumRows()][];
 		Iterator<String[]> iter = frame.getStringRowIterator();
@@ -667,59 +667,59 @@ public class DataConverter {
 			//deep copy output rows due to internal reuse
 			ret[i] = iter.next().clone();
 		}
-		
+
 		return ret;
 	}
-	
+
 	/**
-	 * Converts a two dimensions string array into a frame block of 
-	 * value type string. If the given array is null or of length 0, 
+	 * Converts a two dimensions string array into a frame block of
+	 * value type string. If the given array is null or of length 0,
 	 * we return an empty frame block.
-	 * 
+	 *
 	 * @param data 2d string array
 	 * @return frame block
 	 */
 	public static FrameBlock convertToFrameBlock(String[][] data) {
-		//check for empty frame block 
+		//check for empty frame block
 		if( data == null || data.length==0 )
 			return new FrameBlock();
-		
+
 		//create schema and frame block
 		ValueType[] schema = UtilFunctions.nCopies(data[0].length, ValueType.STRING);
 		return convertToFrameBlock(data, schema);
 	}
 
 	public static FrameBlock convertToFrameBlock(String[][] data, ValueType[] schema) {
-		//check for empty frame block 
+		//check for empty frame block
 		if( data == null || data.length==0 )
 			return new FrameBlock();
-		
+
 		//create frame block
 		return new FrameBlock(schema, data);
 	}
 
 	public static FrameBlock convertToFrameBlock(String[][] data, ValueType[] schema, String[] colnames) {
-		//check for empty frame block 
+		//check for empty frame block
 		if( data == null || data.length==0 )
 			return new FrameBlock();
-		
+
 		//create frame block
 		return new FrameBlock(schema, colnames, data);
 	}
-	
+
 	/**
 	 * Converts a matrix block into a frame block of value type double.
-	 * 
+	 *
 	 * @param mb matrix block
 	 * @return frame block of type double
 	 */
 	public static FrameBlock convertToFrameBlock(MatrixBlock mb) {
 		return convertToFrameBlock(mb, ValueType.FP64);
 	}
-	
+
 	/**
 	 * Converts a matrix block into a frame block of a given value type.
-	 * 
+	 *
 	 * @param mb matrix block
 	 * @param vt value type
 	 * @return frame block
@@ -734,7 +734,7 @@ public class DataConverter {
 	{
 		FrameBlock frame = new FrameBlock(schema);
 		Object[] row = new Object[mb.getNumColumns()];
-		
+
 		if( mb.isInSparseFormat() ) //SPARSE
 		{
 			SparseBlock sblock = mb.getSparseBlock();
@@ -756,7 +756,7 @@ public class DataConverter {
 		else //DENSE
 		{
 			int dFreq = UtilFunctions.frequency(schema, ValueType.FP64);
-			
+
 			if( schema.length==1 && dFreq==1 && mb.isAllocated() ) {
 				// special case double schema and single columns which
 				// allows for a shallow copy since the physical representation
@@ -765,7 +765,7 @@ public class DataConverter {
 				frame.appendColumns(new double[][]{mb.getDenseBlockValues()});
 			}
 			else if( dFreq == schema.length ) {
-				// special case double schema (without cell-object creation, 
+				// special case double schema (without cell-object creation,
 				// col pre-allocation, and cache-friendly row-column copy)
 				int m = mb.getNumRows();
 				int n = mb.getNumColumns();
@@ -801,7 +801,7 @@ public class DataConverter {
 				frame.reset();
 				frame.appendColumns(c);
 			}
-			else { 
+			else {
 				// general case
 				for( int i=0; i<mb.getNumRows(); i++ ) {
 					for( int j=0; j<mb.getNumColumns(); j++ ) {
@@ -812,10 +812,10 @@ public class DataConverter {
 				}
 			}
 		}
-		
+
 		return frame;
 	}
-	
+
 	public static TensorBlock convertToTensorBlock(MatrixBlock mb, ValueType vt, boolean toBasicTensor) {
 		TensorBlock ret;
 		if (toBasicTensor)
@@ -849,7 +849,7 @@ public class DataConverter {
 		long nnz = mb.getNonZeros();
 		boolean sparse = mb.isInSparseFormat();
 		double sparsity = ((double)nnz)/(rows*cols);
-		
+
 		if( colwise ) //COL PARTITIONS
 		{
 			//allocate output partitions
@@ -880,20 +880,20 @@ public class DataConverter {
 			for( int i=0; i<rows; i++ )
 				ret[i] = new MatrixBlock(1, cols, sparse, (long)(cols*sparsity));
 
-			//cache-friendly sparse/dense row slicing 
+			//cache-friendly sparse/dense row slicing
 			if( !mb.isEmptyBlock(false) ) {
 				for( int i=0; i<rows; i++ )
 					mb.slice(i, i, 0, cols-1, ret[i]);
 			}
 		}
-		
+
 		return ret;
 	}
-	
+
 	/**
 	 * Helper method that converts SystemDS matrix variable (<code>varname</code>) into a Array2DRowRealMatrix format,
 	 * which is useful in invoking Apache CommonsMath.
-	 * 
+	 *
 	 * @param mb matrix object
 	 * @return matrix as a commons-math3 Array2DRowRealMatrix
 	 */
@@ -901,7 +901,7 @@ public class DataConverter {
 		double[][] data = DataConverter.convertToDoubleMatrix(mb);
 		return new Array2DRowRealMatrix(data, false);
 	}
-	
+
 	public static BlockRealMatrix convertToBlockRealMatrix(MatrixBlock mb) {
 		BlockRealMatrix ret = new BlockRealMatrix(mb.getNumRows(), mb.getNumColumns());
 		if( mb.getNonZeros() > 0 ) {
@@ -921,7 +921,7 @@ public class DataConverter {
 		}
 		return ret;
 	}
-	
+
 	public static MatrixBlock convertToMatrixBlock(RealMatrix rm) {
 		MatrixBlock ret = new MatrixBlock(rm.getRowDimension(),
 			rm.getColumnDimension(), false).allocateDenseBlock();
@@ -936,10 +936,10 @@ public class DataConverter {
 	{
 		if( mb.isEmptyBlock(false) )
 			return; //quick path
-			
+
 		int rows = mb.getNumRows();
 		int cols = mb.getNumColumns();
-		
+
 		if( mb.isInSparseFormat() ) {
 			Iterator<IJV> iter = mb.getSparseBlockIterator();
 			while( iter.hasNext() ) {
@@ -952,7 +952,7 @@ public class DataConverter {
 			System.arraycopy(mb.getDenseBlockValues(), 0, dest, destPos, rows*cols);
 		}
 	}
-	
+
 	/**
 	 * Convenience method to print NaN & Infinity compliant with how as.scalar prints them.
 	 * {@link DecimalFormat} prints NaN as \uFFFD and Infinity as \u221E
@@ -972,7 +972,7 @@ public class DataConverter {
 	public static String toString(MatrixBlock mb) {
 		return toString(mb, false, " ", "\n", mb.getNumRows(), mb.getNumColumns(), 3);
 	}
-	
+
 	/**
 	 * Returns a string representation of a matrix
 	 * @param mb matrix block
@@ -987,7 +987,7 @@ public class DataConverter {
 	 */
 	public static String toString(MatrixBlock mb, boolean sparse, String separator, String lineseparator, int rowsToPrint, int colsToPrint, int decimal){
 		StringBuffer sb = new StringBuffer();
-		
+
 		// Setup number of rows and columns to print
 		int rlen = mb.getNumRows();
 		int clen = mb.getNumColumns();
@@ -997,13 +997,13 @@ public class DataConverter {
 			rowLength = rowsToPrint < rlen ? rowsToPrint : rlen;
 		if (colsToPrint >= 0)
 			colLength = colsToPrint < clen ? colsToPrint : clen;
-		
+
 		DecimalFormat df = new DecimalFormat();
 		df.setGroupingUsed(false);
 		if (decimal >= 0){
 			df.setMinimumFractionDigits(decimal);
 		}
-		
+
 		if (sparse){ // Sparse Print Format
 			if (mb.isInSparseFormat()){	// Block is in sparse format
 				Iterator<IJV> sbi = mb.getSparseBlockIterator();
@@ -1046,7 +1046,7 @@ public class DataConverter {
 				sb.append(lineseparator);
 			}
 		}
-		
+
 		return sb.toString();
 	}
 
@@ -1069,7 +1069,7 @@ public class DataConverter {
 	 * @return tensor as a string
 	 */
 	public static String toString(TensorBlock tb, boolean sparse, String separator, String lineseparator,
-	                              String leftBorder, String rightBorder, int rowsToPrint, int colsToPrint, int decimal){
+		String leftBorder, String rightBorder, int rowsToPrint, int colsToPrint, int decimal){
 		StringBuilder sb = new StringBuilder();
 
 		// Setup number of rows and columns to print
@@ -1197,7 +1197,7 @@ public class DataConverter {
 	public static String toString(FrameBlock fb, boolean sparse, String separator, String lineseparator, int rowsToPrint, int colsToPrint, int decimal)
 	{
 		StringBuffer sb = new StringBuffer();
-		
+
 		// Setup number of rows and columns to print
 		int rlen = fb.getNumRows();
 		int clen = fb.getNumColumns();
@@ -1207,12 +1207,12 @@ public class DataConverter {
 			rowLength = rowsToPrint < rlen ? rowsToPrint : rlen;
 		if (colsToPrint >= 0)
 			colLength = colsToPrint < clen ? colsToPrint : clen;
-		
+
 		//print frame header
 		sb.append("# FRAME: ");
 		sb.append("nrow = " + fb.getNumRows() + ", ");
 		sb.append("ncol = " + fb.getNumColumns() + lineseparator);
-		
+
 		//print column names
 		sb.append("#"); sb.append(separator);
 		for( int j=0; j<colLength; j++ ) {
@@ -1221,7 +1221,7 @@ public class DataConverter {
 				sb.append(separator);
 		}
 		sb.append(lineseparator);
-		
+
 		//print schema
 		sb.append("#"); sb.append(separator);
 		for( int j=0; j<colLength; j++ ) {
@@ -1230,13 +1230,13 @@ public class DataConverter {
 				sb.append(separator);
 		}
 		sb.append(lineseparator);
-		
+
 		//print data
 		DecimalFormat df = new DecimalFormat();
 		df.setGroupingUsed(false);
 		if (decimal >= 0)
 			df.setMinimumFractionDigits(decimal);
-		
+
 		Iterator<Object[]> iter = fb.getObjectRowIterator(0, rowLength);
 		while( iter.hasNext() ) {
 			Object[] row = iter.next();
@@ -1255,7 +1255,7 @@ public class DataConverter {
 			}
 			sb.append(lineseparator);
 		}
-		
+
 		return sb.toString();
 	}
 
@@ -1309,7 +1309,7 @@ public class DataConverter {
 					throw new DMLRuntimeException("Dimensions have to be passed as list, string, matrix or tensor.");
 				}
 				String dimensionString = ec.getScalarInput(dims.getName(), Types.ValueType.STRING, dims.isLiteral())
-						.getStringValue();
+					.getStringValue();
 				StringTokenizer dimensions = new StringTokenizer(dimensionString, DELIM);
 				tDims = new int[dimensions.countTokens()];
 				Arrays.setAll(tDims, (i) -> Integer.parseInt(dimensions.nextToken()));
@@ -1374,14 +1374,14 @@ public class DataConverter {
 			ret[i] = data[i];
 		return ret;
 	}
-	
+
 	public static double[] toDouble(long[] data) {
 		double[] ret = new double[data.length];
 		for(int i=0; i<data.length; i++)
 			ret[i] = data[i];
 		return ret;
 	}
-	
+
 	public static double[] toDouble(int[] data) {
 		double[] ret = new double[data.length];
 		for(int i=0; i<data.length; i++)
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderCSVTest.java b/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderCSVTest.java
new file mode 100644
index 0000000..2a2aa8b
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderCSVTest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.junit.Test;
+
+public class FrameGenerateReaderCSVTest extends GenerateReaderFrameTest {
+
+	private final static String TEST_NAME = "FrameGenerateReaderCSVTest";
+
+	@Override
+	protected String getTestName() {
+		return TEST_NAME;
+	}
+
+	private void extractSampleRawCSV(String separator) {
+		int nrows = data.length;
+		int ncols = data[0].length;
+		StringBuilder sb = new StringBuilder();
+		for(int r = 0; r < nrows; r++) {
+			for(int c = 0; c < ncols; c++) {
+				sb.append(data[r][c]);
+				if(c != ncols - 1)
+					sb.append(separator);
+			}
+			if(r != nrows - 1)
+				sb.append("\n");
+		}
+		sampleRaw = sb.toString();
+	}
+
+	@Test
+	public void test1() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		generateRandomData(10, 10, -100, 100, 1, naStrings);
+		extractSampleRawCSV(separator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test2() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		generateRandomData(10, 10, -10, 10, 1, naStrings);
+		extractSampleRawCSV(separator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test3() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = "****";
+		generateRandomData(100, 500, -10, 10, 1, naStrings);
+		extractSampleRawCSV(separator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test4() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		generateRandomData(10, 10, -10, 10, 0.7, naStrings);
+		extractSampleRawCSV(separator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test5() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",,,,";
+		generateRandomData(10, 10, -10, 10, 0.2, naStrings);
+		extractSampleRawCSV(separator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test6() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = "**";
+		generateRandomData(1000, 100, -10, 10, 0.2, naStrings);
+		extractSampleRawCSV(separator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test7() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = "**";
+		generateRandomData(1000, 100, -10, 10, 0.8, naStrings);
+		extractSampleRawCSV(separator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test8() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = "**";
+		generateRandomData(100000, 100, -10, 10, 0.5, naStrings);
+		extractSampleRawCSV(separator);
+		runGenerateReaderTest();
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderLibSVMTest.java b/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderLibSVMTest.java
new file mode 100644
index 0000000..204645d
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderLibSVMTest.java
@@ -0,0 +1,142 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.junit.Test;
+
+public class FrameGenerateReaderLibSVMTest extends GenerateReaderFrameTest {
+
+	private final static String TEST_NAME = "FrameGenerateReaderLibSVMTest";
+
+	@Override
+	protected String getTestName() {
+		return TEST_NAME;
+	}
+
+	private void extractSampleRawLibSVM(int firstIndex, String separator, String indexSeparator) {
+
+		int nrows = data.length;
+		int ncols = data[0].length;
+		int mid = ncols/2;
+		String[][] dataLibSVM = new String[2 * nrows][ncols];
+		StringBuilder sb = new StringBuilder();
+		int indexRow = 0;
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row1 = new StringBuilder();
+			StringBuilder row2 = new StringBuilder();
+			row1.append("+1");
+			for(int c = 0; c < ncols - 1; c++) {
+				if(mid > c) {
+					if(data[r][c] != null) {
+						dataLibSVM[indexRow][c] = data[r][c];
+						row1.append(separator).append(c + firstIndex).append(indexSeparator).append(data[r][c]);
+					}
+					else
+						dataLibSVM[indexRow][c] = defaultValue(schema[c]);
+				}
+				else
+					dataLibSVM[indexRow][c] = defaultValue(schema[c]);
+
+			}
+			dataLibSVM[indexRow++][ncols-1] = "+1";
+
+			row2.append("-1");
+			for(int c = 0; c < ncols - 1; c++) {
+				if(mid <= c) {
+					if(data[r][c] != null) {
+						dataLibSVM[indexRow][c] = data[r][c];
+						row2.append(separator).append(c + firstIndex).append(indexSeparator).append(data[r][c]);
+					}
+					else
+						dataLibSVM[indexRow][c] = defaultValue(schema[c]);
+				}
+				else
+					dataLibSVM[indexRow][c] = defaultValue(schema[c]);
+			}
+			dataLibSVM[indexRow++][ncols-1] = "-1";
+			sb.append(row1).append("\n");
+			sb.append(row2);
+			if(r != nrows - 1)
+				sb.append("\n");
+		}
+		sampleRaw = sb.toString();
+		data = dataLibSVM;
+	}
+
+	@Test
+	public void test1() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		String indexSeparator = ":";
+		generateRandomData(5, 10, -100, 100, 1, naStrings);
+		extractSampleRawLibSVM(0,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test2() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		String indexSeparator = ":";
+		generateRandomData(100, 200, -100, 100, 1, naStrings);
+		extractSampleRawLibSVM(0,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test3() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		String indexSeparator = ":";
+		generateRandomData(1000, 200, -100, 100, 1, naStrings);
+		extractSampleRawLibSVM(0,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test4() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",,,,,,";
+		String indexSeparator = ":";
+		generateRandomData(10, 10, -10, 10, 0.5, naStrings);
+		extractSampleRawLibSVM(0,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test5() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",,,,,";
+		String indexSeparator = ":";
+		generateRandomData(100, 10, -100, 100, 0.7, naStrings);
+		extractSampleRawLibSVM(0,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test6() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",,,,,";
+		String indexSeparator = ":";
+		generateRandomData(10, 1000, -100, 100, 0.5, naStrings);
+		extractSampleRawLibSVM(1,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderMatrixMarketTest.java b/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderMatrixMarketTest.java
new file mode 100644
index 0000000..d9e4241
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/FrameGenerateReaderMatrixMarketTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.junit.Test;
+
+public class FrameGenerateReaderMatrixMarketTest extends GenerateReaderFrameTest {
+
+	private final static String TEST_NAME = "FrameGenerateReaderMatrixMarketTest";
+
+	@Override
+	protected String getTestName() {
+		return TEST_NAME;
+	}
+
+	private void extractSampleRawMM(int firstIndex, String separator, String indexSeparator) {
+
+		int nrows = data.length;
+		int ncols = data[0].length;
+
+		StringBuilder sb = new StringBuilder();
+		for(int r = 0; r < nrows; r++) {
+			for(int c = 0; c < ncols; c++) {
+				if(data[r][c] != null && !data[r][c].equals("0")) {
+					String rs = (r + firstIndex) + separator + (c + firstIndex) + separator + data[r][c];
+					sb.append(rs);
+					if(r != nrows - 1 || c != ncols - 1)
+						sb.append("\n");
+				}
+			}
+		}
+		sampleRaw = sb.toString();
+	}
+
+	@Test
+	public void test1() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		String indexSeparator = ":";
+		generateRandomData(5, 10, -100, 100, 1, naStrings);
+		extractSampleRawMM(0,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test2() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		String indexSeparator = ":";
+		generateRandomData(50, 100, -100, 100, 1, naStrings);
+		extractSampleRawMM(0,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+	@Test
+	public void test3() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		String indexSeparator = ":";
+		generateRandomData(50, 100, -100, 100, 0.5, naStrings);
+		extractSampleRawMM(1,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test4() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		String indexSeparator = ":";
+		generateRandomData(50, 100, -100, 100, 0.2, naStrings);
+		extractSampleRawMM(1,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test5() {
+		String[] naStrings = {"NULL", "inf", "NaN"};
+		String separator = ",";
+		String indexSeparator = ":";
+		generateRandomData(50, 100, -100, 100, 0.8, naStrings);
+		extractSampleRawMM(0,separator, indexSeparator);
+		runGenerateReaderTest();
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/GenerateRandomFrame.java b/src/test/java/org/apache/sysds/test/functions/iogen/GenerateRandomFrame.java
new file mode 100644
index 0000000..9b71105
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/GenerateRandomFrame.java
@@ -0,0 +1,313 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.apache.sysds.common.Types;
+import org.apache.sysds.runtime.util.UtilFunctions;
+import org.apache.sysds.test.AutomatedTestBase;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Random;
+
+public class GenerateRandomFrame extends AutomatedTestBase {
+
+	protected final static String TEST_DIR = "functions/iogen/";
+
+	@Override
+	public void setUp() {
+	
+	}
+
+
+	protected Types.ValueType[] types = {Types.ValueType.STRING, Types.ValueType.INT32, Types.ValueType.INT64,
+		Types.ValueType.FP32, Types.ValueType.FP64};
+
+	protected String[][] generateRandomData(Types.ValueType[] types, int nrows, int ncols, double min, double max,
+		double sparsity, String[] naStrings) {
+		String[][] data = new String[nrows][ncols];
+		for(int i = 0; i < ncols; i++) {
+			if(types[i] == Types.ValueType.STRING)
+				generateRandomString(nrows, 100, naStrings, sparsity, data, i);
+			if(types[i].isNumeric()) {
+				generateRandomNumeric(nrows, types[i], min, max, naStrings, sparsity, data, i);
+			}
+		}
+		return data;
+	}
+
+	protected String getRandomString(int length) {
+		//String alphabet1 = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz1234567890";
+		String alphabet = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz";
+		StringBuilder salt = new StringBuilder();
+		Random rnd = new Random();
+		while(salt.length() < length) { // length of the random string.
+			int index = (int) (rnd.nextFloat() * alphabet.length());
+			salt.append(alphabet.charAt(index));
+		}
+		String saltStr = salt.toString();
+		return saltStr;
+
+	}
+
+	protected void generateRandomString(int size, int maxStringLength, String[] naStrings, double sparsity,
+		String[][] data, int colIndex) {
+
+		double[][] lengths = getRandomMatrix(size, 1, 10, maxStringLength, sparsity, 714);
+
+		for(int i = 0; i < size; i++) {
+			int length = (int) lengths[i][0];
+			if(length > 0) {
+				String generatedString = getRandomString(length);
+				data[i][colIndex] = generatedString;
+			}
+			else {
+				data[i][colIndex] = null;
+			}
+		}
+	}
+
+	@SuppressWarnings("incomplete-switch")
+	protected void generateRandomNumeric(int size, Types.ValueType type, double min, double max, String[] naStrings,
+		double sparsity, String[][] data, int colIndex) {
+
+		double[][] randomData = getRandomMatrix(size, 1, min, max, sparsity, -1);
+		for(int i = 0; i < size; i++) {
+			if(randomData[i][0] != 0) {
+				Object o = null;
+				switch(type) {
+					case INT32:
+						o = UtilFunctions.objectToObject(type, (int) randomData[i][0]);
+						break;
+					case INT64:
+						o = UtilFunctions.objectToObject(type, (long) randomData[i][0]);
+						break;
+					case FP32:
+						o = UtilFunctions.objectToObject(type, (float) randomData[i][0]);
+						break;
+					case FP64:
+						o = UtilFunctions.objectToObject(type, randomData[i][0]);
+						break;
+				}
+				String s = UtilFunctions.objectToString(o);
+				data[i][colIndex] = s;
+			}
+			else {
+				data[i][colIndex] = "0";
+			}
+		}
+	}
+	// Write 2D Data in CSV format
+	private static void writeInCSVFormat(String[][] data, int nrows, int ncols, String fileName, String separator,
+		String[] naString) throws Exception {
+
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".raw"));
+
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row = new StringBuilder();
+			for(int c = 0; c < ncols; c++) {
+				row.append(data[r][c]);
+				if(c != ncols - 1)
+					row.append(separator);
+			}
+			writer.write(row.toString());
+			if(r != nrows - 1)
+				writer.write("\n");
+		}
+		writer.close();
+	}
+
+	// Write 2D in LIBSVM format
+	private static String[][] writeInLIBSVMFormat(int firstIndex,Types.ValueType[] schema, String[][] data, int nrows, int ncols, String fileName,
+		String separator, String indexSeparator) throws IOException {
+
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".raw"));
+		int mid = ncols/2;
+		String[][] dataLibSVM = new String[2 * nrows][ncols+1];
+		StringBuilder sb = new StringBuilder();
+		int indexRow = 0;
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row1 = new StringBuilder();
+			StringBuilder row2 = new StringBuilder();
+			row1.append("+1");
+			for(int c = 0; c < ncols; c++) {
+				if(mid > c) {
+					dataLibSVM[indexRow][c] = data[r][c];
+					row1.append(separator).append(c + firstIndex).append(indexSeparator).append(data[r][c]);
+				}
+				else {
+					if(schema[c].isNumeric() || schema[c] == Types.ValueType.BOOLEAN){
+						dataLibSVM[indexRow][c] = "0";
+					}
+					else if(schema[c] == Types.ValueType.STRING)
+						dataLibSVM[indexRow][c] = "";
+				}
+			}
+			dataLibSVM[indexRow++][ncols] = "+1";
+
+			row2.append("-1");
+			for(int c = 0; c < ncols ; c++) {
+				if(mid <= c) {
+					dataLibSVM[indexRow][c] = data[r][c];
+					row2.append(separator).append(c + firstIndex).append(indexSeparator).append(data[r][c]);
+				}
+				else {
+					if(schema[c].isNumeric() || schema[c] == Types.ValueType.BOOLEAN){
+						dataLibSVM[indexRow][c] = "0";
+					}
+					else if(schema[c] == Types.ValueType.STRING)
+						dataLibSVM[indexRow][c] = "";
+				}
+			}
+			dataLibSVM[indexRow++][ncols] = "-1";
+			writer.write(row1.toString());
+			writer.write("\n");
+			writer.write(row2.toString());
+			if(r != nrows - 1)
+				writer.append("\n");
+
+			sb.append(row1).append("\n");
+			sb.append(row2);
+			if(r != nrows - 1)
+				sb.append("\n");
+		}
+		writer.close();
+		return dataLibSVM;
+	}
+
+	// Write in Matrix Market Format
+	private static void writeInMatrixMarketFormat(int firstIndex, String[][] data, int nrows, int ncols, String fileName,
+		String separator) throws IOException {
+
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".raw"));
+		for(int r = 0; r < nrows; r++) {
+			for(int c = 0; c < ncols; c++) {
+				if(data[r][c] != null && !data[r][c].equals("0")) {
+					String rs = (r + firstIndex) + separator + (c + firstIndex) + separator + data[r][c];
+					writer.write(rs);
+					if(r != nrows - 1 || c != ncols - 1)
+						writer.write("\n");
+				}
+			}
+		}
+		writer.close();
+	}
+
+
+
+	// Write 2D Data in CSV format
+	private static void writeSampleFrame(Types.ValueType[] schema, String[][] sample, String fileName, int nrows, int ncols)
+		throws Exception {
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".frame"));
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row = new StringBuilder();
+			for(int c = 0; c < ncols; c++) {
+				row.append(sample[r][c]);
+				if(c != ncols - 1)
+					row.append(",");
+			}
+			writer.write(row.toString());
+			if(r != nrows - 1)
+				writer.write("\n");
+		}
+		writer.close();
+
+		writer = new BufferedWriter(new FileWriter(fileName + ".schema"));
+		StringBuilder sb = new StringBuilder();
+		for(int c = 0; c < ncols; c++) {
+			sb.append(schema[c]);
+			if(c != ncols - 1)
+				sb.append(",");
+		}
+
+		writer.write(sb.toString());
+		writer.close();
+	}
+
+	@Test
+	@Ignore
+	public void generateDataset() throws Exception {
+		int nrows = 5000;
+		int ncols = 5000;
+		double sparsity = 1;
+		String HOME = SCRIPT_DIR + TEST_DIR;
+		String[] naStrings = {"Nan", "NAN", "", "inf", "null", "NULL"};
+		String[] names = new String[ncols];
+		Types.ValueType[] schema = new Types.ValueType[ncols];
+
+		for(int i = 0; i < nrows; i++) {
+			names[i] = "C_" + i;
+			Random rn = new Random();
+			int rnt = rn.nextInt(types.length);
+			schema[i] = types[rnt];
+		}
+		String[][] data = generateRandomData(schema, nrows, ncols, -100, 100, sparsity, naStrings);
+		saveData(schema, data, nrows, ncols, " ", ":", naStrings, HOME + "/data/", sparsity, false);
+
+		for(int r = 10; r <= 100; r += 10) {
+			saveData(schema, data, r, r, " ", ":", naStrings, HOME + "/samples/", sparsity, true);
+		}
+
+		BufferedWriter writer = new BufferedWriter(new FileWriter(HOME+"/data/data"+"_nrows_" + nrows + "_ncols_" + ncols + "_sparsity_" + sparsity + ".schema"));
+		StringBuilder sb = new StringBuilder();
+		for(int c = 0; c < ncols; c++) {
+			sb.append(schema[c]);
+			if(c != ncols - 1)
+				sb.append(",");
+		}
+		writer.write(sb.toString());
+		writer.close();
+	}
+
+	private static void saveData(Types.ValueType[] schema, String[][] data, int nrows, int ncols, String separator,
+		String indexSeparator, String[] naStrings, String HOME, double sparsity, boolean saveSampleFrame)
+		throws Exception {
+
+		String baseFileName = "_nrows_" + nrows + "_ncols_" + ncols + "_sparsity_" + sparsity;
+
+		String csv = HOME + "CSV" + baseFileName;
+
+		String libsvmFirstZero = HOME + "LIBSVM-FZ" + baseFileName;
+		String libsvmFirstOne = HOME + "LIBSVM-FO" + baseFileName;
+
+		String mmFirstZero = HOME + "MM-FZ" + baseFileName;
+		String mmFirstOne = HOME + "MM-FO" + baseFileName;
+
+		// Write all data as a source dataset
+		writeInCSVFormat(data, nrows, ncols, csv, separator, naStrings);
+		String[][] libsvm = writeInLIBSVMFormat(0,schema, data, nrows, ncols, libsvmFirstZero, separator, indexSeparator);
+		writeInLIBSVMFormat(1,schema, data, nrows, ncols, libsvmFirstOne, separator, indexSeparator);
+		writeInMatrixMarketFormat(0, data, nrows, ncols, mmFirstZero, separator);
+		writeInMatrixMarketFormat(1, data, nrows, ncols, mmFirstOne, separator);
+
+		if(saveSampleFrame) {
+			writeSampleFrame(schema,data, csv, nrows, ncols);
+			Types.ValueType[] libsvmSchema = new Types.ValueType[ncols+1];
+			for(int i=0;i<ncols;i++)
+				libsvmSchema[i] = schema[i];
+			libsvmSchema[ncols] = Types.ValueType.INT32;
+			writeSampleFrame(libsvmSchema,libsvm, HOME + "LIBSVM" + baseFileName, 2 * nrows, ncols + 1);
+			writeSampleFrame(schema,data, HOME + "MM" + baseFileName, nrows, ncols);
+		}
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/GenerateRandomMatrix.java b/src/test/java/org/apache/sysds/test/functions/iogen/GenerateRandomMatrix.java
new file mode 100644
index 0000000..dae7a46
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/GenerateRandomMatrix.java
@@ -0,0 +1,319 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.apache.sysds.test.AutomatedTestBase;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Random;
+
+public class GenerateRandomMatrix extends AutomatedTestBase {
+
+	protected final static String TEST_DIR = "functions/iogen/";
+
+	@Override
+	public void setUp() {
+	
+	}
+
+	// Generate Random Symmetric 2D Data
+	@SuppressWarnings("unused")
+	private double[][] generateRandomSymmetric(int size, double min, double max, double sparsity, boolean isSkew) {
+		double[][] sampleMatrix = getRandomMatrix(size, size, min, max, sparsity, 714);
+		int conf = isSkew ? -1 : 1;
+		for(int i = 0; i < size; i++) {
+			for(int j = 0; j <= i; j++) {
+
+				if(i != j)
+					sampleMatrix[i][j] = sampleMatrix[j][i] * conf;
+				else
+					sampleMatrix[i][j] = 0;
+			}
+		}
+		return sampleMatrix;
+	}
+
+	// Generate Random 2D Data
+	private double[][] generateRandom2DData(int nrows, int ncols, double min, double max, double sparsity) {
+		double[][] data = getRandomMatrix(nrows, ncols, min, max, sparsity, 714);
+		return data;
+	}
+
+	// Generate Random Symmetric 2D Data
+	protected double[][] getSymmetric2DData(double[][] data, int size, boolean isSkew) {
+		double[][] result = new double[size][size];
+		int conf = isSkew ? -1 : 1;
+		// Update Data
+		for(int i = 0; i < size; i++) {
+			for(int j = 0; j <= i; j++) {
+				if(i != j) {
+					result[i][j] = data[i][j] * conf;
+					result[j][i] = result[i][j];
+				}
+				else
+					result[i][j] = 0;
+			}
+		}
+		return result;
+	}
+
+	// Load Random 2D data from file
+	@SuppressWarnings("unused")
+	private static double[][] load2DData(String fileName, int nrows, int ncols) throws Exception {
+
+		Path path = Paths.get(fileName);
+		FileChannel inStreamRegularFile = FileChannel.open(path);
+		int bufferSize = ncols * 8;
+
+		double[][] result = new double[nrows][ncols];
+		try {
+			for(int r = 0; r < nrows; r++) {
+				inStreamRegularFile.position((long) r * ncols * 8);
+				ByteBuffer buffer = ByteBuffer.allocateDirect(bufferSize);
+				inStreamRegularFile.read(buffer);
+				buffer.flip();
+
+				for(int c = 0; c < ncols; c++) {
+					result[r][c] = buffer.getDouble();
+				}
+			}
+			inStreamRegularFile.close();
+		}
+		catch(IOException e) {
+			throw new Exception("Can't read matrix from ByteArray", e);
+		}
+		return result;
+	}
+
+	// Write 2D Data in CSV format
+	private static void writeInCSVFormat(double[][] data, int nrows, int ncols, String fileName, String separator,
+		String[] naString) throws Exception {
+
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".raw"));
+
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row = new StringBuilder();
+			for(int c = 0; c < ncols; c++) {
+				if(data[r][c] != 0) {
+					row.append(data[r][c]).append(separator);
+				}
+				else {
+					Random rn = new Random();
+					int rni = rn.nextInt(naString.length);
+					row.append(naString[rni]).append(separator);
+				}
+			}
+			String srow = row.substring(0, row.length() - separator.length());
+			writer.write(srow);
+			if(r != nrows - 1)
+				writer.write("\n");
+		}
+		writer.close();
+	}
+
+	// Write 2D in LIBSVM format
+	private static double[][] writeInLIBSVMFormat(int firstIndex, double[][] data, int nrows, int ncols, String fileName,
+		String separator, String indexSeparator) throws IOException {
+
+		int indexRow = 0;
+		double[][] sampleMatrix = new double[2 * nrows][ncols + 1];
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".raw"));
+
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row1 = new StringBuilder();
+			StringBuilder row2 = new StringBuilder();
+			row1.append("+1");
+
+			for(int c = 0; c < ncols; c++) {
+				if(data[r][c] > 0) {
+					sampleMatrix[indexRow][c] = data[r][c];
+					row1.append(separator).append(c + firstIndex).append(indexSeparator).append(data[r][c]);
+				}
+				else {
+					sampleMatrix[indexRow][c] = 0;
+				}
+			}
+			sampleMatrix[indexRow++][ncols] = 1;
+
+			row2.append("-1");
+			for(int c = 0; c < ncols; c++) {
+				if(data[r][c] < 0) {
+					sampleMatrix[indexRow][c] = data[r][c];
+					row2.append(separator).append(c + firstIndex).append(indexSeparator).append(data[r][c]);
+				}
+				else {
+					sampleMatrix[indexRow][c] = 0;
+				}
+			}
+
+			sampleMatrix[indexRow++][ncols] = -1;
+			writer.write(row1.toString());
+			writer.write("\n");
+			writer.write(row2.toString());
+			if(r != nrows - 1)
+				writer.append("\n");
+		}
+		writer.close();
+		return sampleMatrix;
+	}
+
+	// Write in Matrix Market Format
+	private static void writeInMatrixMarketFormat(int firstIndex, double[][] data, int nrows, int ncols, String fileName,
+		String separator) throws IOException {
+
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".raw"));
+
+		for(int r = 0; r < nrows; r++) {
+			for(int c = 0; c < ncols; c++) {
+				if(data[r][c] != 0) {
+					String rs = (r + firstIndex) + separator + (c + firstIndex) + separator + data[r][c];
+					writer.write(rs);
+					if(r != nrows - 1 || c != ncols - 1)
+						writer.write("\n");
+				}
+			}
+		}
+		writer.close();
+	}
+
+	private static void writeInSymmetricMatrixMarketFormat(int firstIndex, double[][] data, String fileName, int size,
+		String separator, boolean isUpperTriangular) throws IOException {
+
+		int start, end;
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".raw"));
+		for(int r = 0; r < size; r++) {
+			if(isUpperTriangular) {
+				start = r;
+				end = size;
+			}
+			else {
+				start = 0;
+				end = r + 1;
+			}
+			for(int c = start; c < end; c++) {
+				if(data[r][c] != 0) {
+					String rs = (r + firstIndex) + separator + (c + firstIndex) + separator + data[r][c];
+					writer.write(rs);
+					if(r != size - 1 || c != size - 1)
+						writer.write("\n");
+				}
+			}
+		}
+		writer.close();
+	}
+
+	// Write 2D Data in CSV format
+	private static void writeSampleMatrix(double[][] sample, String fileName, int nrows, int ncols) throws Exception {
+
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName + ".matrix"));
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row = new StringBuilder();
+			for(int c = 0; c < ncols; c++) {
+				row.append(sample[r][c]);
+				if(c != ncols - 1)
+					row.append(",");
+			}
+			writer.write(row.toString());
+			if(r != nrows - 1)
+				writer.write("\n");
+		}
+		writer.close();
+	}
+
+	@Test
+	@Ignore
+	public void generateDataset() throws Exception {
+		int nrows = 5000;
+		int ncols = 5000;
+		double sparsity = 1;
+		String HOME = SCRIPT_DIR + TEST_DIR;
+		String[] naString = {"Nan", "NAN", "", "inf", "null", "NULL"};
+		double[][] data = generateRandom2DData(nrows, ncols, -100, 100, sparsity);
+		saveData(data, nrows, ncols, " ", ":", naString, HOME + "/data/", sparsity, false);
+
+		for(int r = 10; r <= 100; r += 10) {
+			saveData(data, r, r, " ", ":", naString, HOME + "/samples/", sparsity, true);
+		}
+	}
+
+	private void saveData(double[][] data, int nrows, int ncols, String separator, String indexSeparator,
+		String[] naStrings, String HOME, double sparsity, boolean saveSampleMatrix) throws Exception {
+
+		String baseFileName = "_nrows_" + nrows + "_ncols_" + ncols + "_sparsity_" + sparsity;
+
+		String csv = HOME + "CSV" + baseFileName;
+
+		String libsvmFirstZero = HOME + "LIBSVM-FZ" + baseFileName;
+		String libsvmFirstOne = HOME + "LIBSVM-FO" + baseFileName;
+
+		String mmFirstZero = HOME + "MM-FZ" + baseFileName;
+		String mmFirstOne = HOME + "MM-FO" + baseFileName;
+
+		String mmFirstZeroSymUT = HOME + "MM-FZ-SYM-UT" + baseFileName;
+		String mmFirstZeroSymLT = HOME + "MM-FZ-SYM-LT" + baseFileName;
+		String mmFirstOneSymUT = HOME + "MM-FO-SYM-UT" + baseFileName;
+		String mmFirstOneSymLT = HOME + "MM-FO-SYM-LT" + baseFileName;
+
+		String mmFirstZeroSkewUT = HOME + "MM-FZ-SKEW-UT" + baseFileName;
+		String mmFirstZeroSkewLT = HOME + "MM-FZ-SKEW-LT" + baseFileName;
+		String mmFirstOneSkewUT = HOME + "MM-FO-SKEW-UT" + baseFileName;
+		String mmFirstOneSkewLT = HOME + "MM-FO-SKEW-LT" + baseFileName;
+
+		// Write all data as a source dataset
+		writeInCSVFormat(data, nrows, ncols, csv, separator, naStrings);
+		double[][] libsvm = writeInLIBSVMFormat(0, data, nrows, ncols, libsvmFirstZero, separator, indexSeparator);
+		writeInLIBSVMFormat(1, data, nrows, ncols, libsvmFirstOne, separator, indexSeparator);
+		writeInMatrixMarketFormat(0, data, nrows, ncols, mmFirstZero, separator);
+		writeInMatrixMarketFormat(1, data, nrows, ncols, mmFirstOne, separator);
+
+		if(saveSampleMatrix) {
+			writeSampleMatrix(data, csv, nrows, ncols);
+			writeSampleMatrix(libsvm, HOME + "LIBSVM" + baseFileName, 2 * nrows, ncols + 1);
+			writeSampleMatrix(data, HOME + "MM" + baseFileName, nrows, ncols);
+		}
+
+		// Write MM Symmetric and Skew
+		if(nrows == ncols) {
+			double[][] mm = getSymmetric2DData(data, nrows, false);
+			writeInSymmetricMatrixMarketFormat(0, mm, mmFirstZeroSymUT, ncols, separator, true);
+			writeInSymmetricMatrixMarketFormat(1, mm, mmFirstOneSymUT, ncols, separator, true);
+			writeInSymmetricMatrixMarketFormat(0, mm, mmFirstZeroSymLT, ncols, separator, false);
+			writeInSymmetricMatrixMarketFormat(1, mm, mmFirstOneSymLT, ncols, separator, false);
+			if(saveSampleMatrix)
+				writeSampleMatrix(mm, HOME + "MM-SYM" + baseFileName, nrows, nrows);
+
+			mm = getSymmetric2DData(data, nrows, true);
+			writeInSymmetricMatrixMarketFormat(0, mm, mmFirstZeroSkewUT, ncols, separator, true);
+			writeInSymmetricMatrixMarketFormat(1, mm, mmFirstOneSkewUT, ncols, separator, true);
+			writeInSymmetricMatrixMarketFormat(0, mm, mmFirstZeroSkewLT, ncols, separator, false);
+			writeInSymmetricMatrixMarketFormat(1, mm, mmFirstOneSkewLT, ncols, separator, false);
+			if(saveSampleMatrix)
+				writeSampleMatrix(mm, HOME + "MM-SKEW" + baseFileName, nrows, nrows);
+		}
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/GenerateReaderFrameTest.java b/src/test/java/org/apache/sysds/test/functions/iogen/GenerateReaderFrameTest.java
new file mode 100644
index 0000000..891890c
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/GenerateReaderFrameTest.java
@@ -0,0 +1,193 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.apache.sysds.api.DMLScript;
+import org.apache.sysds.common.Types;
+import org.apache.sysds.conf.CompilerConfig;
+import org.apache.sysds.runtime.io.FrameReader;
+import org.apache.sysds.runtime.iogen.GenerateReader;
+import org.apache.sysds.runtime.matrix.data.FrameBlock;
+import org.apache.sysds.runtime.util.UtilFunctions;
+import org.apache.sysds.test.AutomatedTestBase;
+import org.apache.sysds.test.TestConfiguration;
+import org.apache.sysds.test.TestUtils;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Random;
+
+public abstract class GenerateReaderFrameTest extends AutomatedTestBase {
+
+	protected final static String TEST_DIR = "functions/iogen/";
+	protected final static String TEST_CLASS_DIR = TEST_DIR + GenerateReaderFrameTest.class.getSimpleName() + "/";
+	protected String sampleRaw;
+	protected String[][] data;
+	protected String[] names;
+	protected Types.ValueType[] schema;
+	protected Types.ValueType[] types= {
+		Types.ValueType.STRING,
+		Types.ValueType.INT32,
+		Types.ValueType.INT64,
+		Types.ValueType.FP32,
+		Types.ValueType.FP64,
+		Types.ValueType.BOOLEAN};
+
+	protected Types.ValueType[] types1= { Types.ValueType.BOOLEAN};
+
+	protected abstract String getTestName();
+
+	@Override public void setUp() {
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(getTestName(), new TestConfiguration(TEST_DIR, getTestName(), new String[] {"Y"}));
+	}
+
+	protected String getRandomString(int length) {
+		//String alphabet1 = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz1234567890";
+		String alphabet = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz";
+		StringBuilder salt = new StringBuilder();
+		Random rnd = new Random();
+		while (salt.length() < length) { // length of the random string.
+			int index = (int) (rnd.nextFloat() * alphabet.length());
+			salt.append(alphabet.charAt(index));
+		}
+		String saltStr = salt.toString();
+		return saltStr;
+	}
+
+	@SuppressWarnings("incomplete-switch")
+	protected String defaultValue(Types.ValueType vt){
+		switch(vt){
+			case STRING: return "";
+			case BOOLEAN: return null;
+			case FP32:
+			case FP64:
+			case INT32:
+			case INT64:
+				return "0";
+		}
+		return null;
+	}
+
+	protected void generateRandomString(int size, int maxStringLength, String[] naStrings, double sparsity, String[][] data, int colIndex) {
+
+		double[][] lengths = getRandomMatrix(size, 1, 10, maxStringLength, sparsity, 714);
+
+		for(int i = 0; i < size; i++) {
+			int length = (int) lengths[i][0];
+			if(length > 0) {
+				String generatedString = getRandomString(length);
+				data[i][colIndex] = generatedString;
+			}
+			else {
+				data[i][colIndex] = null;
+			}
+		}
+	}
+
+	@SuppressWarnings("incomplete-switch")
+	protected void generateRandomNumeric(int size, Types.ValueType type, double min, double max, String[] naStrings,
+		double sparsity, String[][] data, int colIndex) {
+
+		double[][] randomData = getRandomMatrix(size, 1, min, max, sparsity, -1);
+		for(int i = 0; i < size; i++) {
+			if(randomData[i][0] != 0) {
+				Object o = null;
+				switch(type){
+					case INT32: o = UtilFunctions.objectToObject(type,(int)randomData[i][0]); break;
+					case INT64: o = UtilFunctions.objectToObject(type,(long)randomData[i][0]); break;
+					case FP32: o = UtilFunctions.objectToObject(type,(float)randomData[i][0]); break;
+					case FP64: o = UtilFunctions.objectToObject(type,randomData[i][0]); break;
+					case BOOLEAN: Boolean b= randomData[i][0] >0 ? true: null; o = UtilFunctions.objectToObject(type, b); break;
+				}
+				String s = UtilFunctions.objectToString(o);
+				data[i][colIndex] = s;
+			}
+			else {
+				if(type.isNumeric())
+					data[i][colIndex] ="0";
+				else
+					data[i][colIndex] =null;
+			}
+		}
+	}
+
+	protected void generateRandomData(int nrows, int ncols, double min, double max, double sparsity, String[] naStrings) {
+
+		names = new String[ncols];
+		schema = new Types.ValueType[ncols];
+		data = new String[nrows][ncols];
+
+		for(int i = 0; i < ncols; i++) {
+			names[i] = "C_" + i;
+
+			Random rn = new Random();
+			int rnt = rn.nextInt(types.length);
+			schema[i] = types[rnt];
+
+			if(types[rnt] == Types.ValueType.STRING)
+				generateRandomString(nrows,100,naStrings,sparsity,data,i);
+			else if(types[rnt].isNumeric() || types[rnt] == Types.ValueType.BOOLEAN)
+				generateRandomNumeric(nrows, types[rnt],min,max,naStrings, sparsity,data,i);
+			}
+	}
+	@SuppressWarnings("unused")
+	protected void runGenerateReaderTest() {
+
+		Types.ExecMode oldPlatform = rtplatform;
+		rtplatform = Types.ExecMode.SINGLE_NODE;
+
+		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+		boolean oldpar = CompilerConfig.FLAG_PARREADWRITE_TEXT;
+
+		try {
+			CompilerConfig.FLAG_PARREADWRITE_TEXT = false;
+
+			TestConfiguration config = getTestConfiguration(getTestName());
+			loadTestConfiguration(config);
+
+			FrameBlock sampleFrame = new FrameBlock(schema, names, data);
+
+			String HOME = SCRIPT_DIR + TEST_DIR;
+			String dataPath = HOME + "frame_data.raw";
+			int clen = data[0].length;
+			writeRawString(sampleRaw, dataPath);
+			GenerateReader.GenerateReaderFrame gr = new GenerateReader.GenerateReaderFrame(sampleRaw, sampleFrame);
+
+			FrameReader fr= gr.getReader();
+			FrameBlock grFrame = fr.readFrameFromHDFS(dataPath,schema,names,data.length, clen);
+		}
+		catch(Exception exception) {
+			exception.printStackTrace();
+		}
+		finally {
+			rtplatform = oldPlatform;
+			CompilerConfig.FLAG_PARREADWRITE_TEXT = oldpar;
+			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+		}
+	}
+
+	private static void writeRawString(String raw, String fileName) throws IOException {
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName));
+		writer.write(raw);
+		writer.close();
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/GenerateReaderMatrixTest.java b/src/test/java/org/apache/sysds/test/functions/iogen/GenerateReaderMatrixTest.java
new file mode 100644
index 0000000..1b4375a
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/GenerateReaderMatrixTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.apache.sysds.api.DMLScript;
+import org.apache.sysds.common.Types;
+import org.apache.sysds.conf.CompilerConfig;
+import org.apache.sysds.runtime.io.MatrixReader;
+import org.apache.sysds.runtime.iogen.GenerateReader;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.util.DataConverter;
+import org.apache.sysds.test.AutomatedTestBase;
+import org.apache.sysds.test.TestConfiguration;
+import org.apache.sysds.test.TestUtils;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+
+public abstract class GenerateReaderMatrixTest extends AutomatedTestBase {
+
+	protected final static String TEST_DIR = "functions/iogen/";
+	protected final static String TEST_CLASS_DIR = TEST_DIR + GenerateReaderMatrixTest.class.getSimpleName() + "/";
+	protected String sampleRaw;
+	protected double[][] sampleMatrix;
+
+	protected abstract String getTestName();
+
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(getTestName(), new TestConfiguration(TEST_DIR, getTestName(), new String[] {"Y"}));
+	}
+
+	protected void generateRandomSymmetric(int size, double min, double max, double sparsity, boolean isSkew) {
+		sampleMatrix = getRandomMatrix(size, size, min, max, sparsity, 714);
+		int conf = isSkew ? -1 : 1;
+		for(int i = 0; i < size; i++) {
+			for(int j = 0; j <= i; j++) {
+
+				if(i != j)
+					sampleMatrix[i][j] = sampleMatrix[j][i] * conf;
+				else
+					sampleMatrix[i][j] = 0;
+			}
+		}
+	}
+
+	@SuppressWarnings("unused")
+	protected void runGenerateReaderTest() {
+
+		Types.ExecMode oldPlatform = rtplatform;
+		rtplatform = Types.ExecMode.SINGLE_NODE;
+
+		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+		boolean oldpar = CompilerConfig.FLAG_PARREADWRITE_TEXT;
+
+		try {
+			CompilerConfig.FLAG_PARREADWRITE_TEXT = false;
+
+			TestConfiguration config = getTestConfiguration(getTestName());
+			loadTestConfiguration(config);
+
+			MatrixBlock sampleMB = DataConverter.convertToMatrixBlock(sampleMatrix);
+
+			String HOME = SCRIPT_DIR + TEST_DIR;
+			String dataPath = HOME + "matrix_data.raw";
+			int clen = sampleMatrix[0].length;
+			writeRawString(sampleRaw, dataPath);
+			GenerateReader.GenerateReaderMatrix gr = new GenerateReader.GenerateReaderMatrix(sampleRaw, sampleMB);
+
+			MatrixReader mr= gr.getReader();
+			MatrixBlock matrixBlock = mr.readMatrixFromHDFS(dataPath, -1, clen, -1, -1);
+		}
+		catch(Exception exception) {
+			exception.printStackTrace();
+		}
+		finally {
+			rtplatform = oldPlatform;
+			CompilerConfig.FLAG_PARREADWRITE_TEXT = oldpar;
+			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+		}
+	}
+
+	private static void writeRawString(String raw, String fileName) throws IOException {
+		BufferedWriter writer = new BufferedWriter(new FileWriter(fileName));
+		writer.write(raw);
+		writer.close();
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderCSVTest.java b/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderCSVTest.java
new file mode 100644
index 0000000..0a49706
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderCSVTest.java
@@ -0,0 +1,151 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.junit.Test;
+
+import java.util.Random;
+
+public class MatrixGenerateReaderCSVTest extends GenerateReaderMatrixTest {
+
+	private final static String TEST_NAME = "MatrixGenerateReaderCSVTest";
+
+	@Override
+	protected String getTestName() {
+		return TEST_NAME;
+	}
+
+	private void generateRandomCSV(int nrows, int ncols, double min, double max, double sparsity, String separator,
+		String[] naString) {
+
+		sampleMatrix = getRandomMatrix(nrows, ncols, min, max, sparsity, 714);
+		StringBuilder sb = new StringBuilder();
+
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row = new StringBuilder();
+			for(int c = 0; c < ncols; c++) {
+				if(sampleMatrix[r][c] != 0) {
+					row.append(sampleMatrix[r][c]).append(separator);
+				}
+				else {
+					Random rn = new Random();
+					int rni = rn.nextInt(naString.length);
+					row.append(naString[rni]).append(separator);
+				}
+			}
+
+			sb.append(row.substring(0, row.length() - separator.length()));
+			if(r != nrows - 1)
+				sb.append("\n");
+		}
+		sampleRaw = sb.toString();
+	}
+
+	@Test
+	public void test1() {
+		sampleRaw = "1,2,3,4,5\n" + "6,7,8,9,10\n" + "11,12,13,14,15";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5}, {6, 7, 8, 9, 10}, {11, 12, 13, 14, 15}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test2() {
+		String[] naString = {"NaN"};
+		generateRandomCSV(5, 5, -10, 10, 1, ",", naString);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test3() {
+		String[] naString = {"NaN"};
+		generateRandomCSV(5, 5, -10, 10, 1, ",,,", naString);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test4() {
+		String[] naString = {"Nan", "NAN", "", "inf", "null", "NULL"};
+		generateRandomCSV(50, 50, -10, 10, 0.5, ",,", naString);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test5() {
+		sampleRaw = "1.0,2.0,3.0,4.0,5.0\n" + "6.,7.,8.,9.,10.\n" + "11,12,13,14,15";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5}, {6, 7, 8, 9, 10}, {11, 12, 13, 14, 15}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test6() {
+		sampleRaw = "1.0,2.0,3.0,4.0,5.0\n" + "6.,7.,8.,9.,10.\n" + "11E0,12E0,13,14E0,15";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5}, {6, 7, 8, 9, 10}, {11, 12, 13, 14, 15}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test7() {
+		sampleRaw = "1.0,2.0,3.0,4.0,5.0\n" + "6.,7.,8.,9.,10.\n" + "1.1E1,1.2E1,13,1.4E1,15";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5}, {6, 7, 8, 9, 10}, {11, 12, 13, 14, 15}};
+		runGenerateReaderTest();
+	}
+
+	@Test public void test8() {
+		sampleRaw = "1.0,2.0,3.0,4.0,5.0\n" + "60.0E-1,7.,80.0E-1,9.,100.0E-1\n" + "1.1E1,1.2E1,13,1.4E1,15";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5}, {6, 7, 8, 9, 10}, {11, 12, 13, 14, 15}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test9() {
+		sampleRaw = ".1E1,.2E1,3.0,4.0,0.5E1\n" + "60.0E-1,7.,80.0E-1,9.,100.0E-1\n" + "1.1E1,1.2E1,13,1.4E1,15";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5}, {6, 7, 8, 9, 10}, {11, 12, 13, 14, 15}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test10() {
+		sampleRaw = "0.000001e6,2,3,4,5\n" + "6,7,8,9,10\n" + "11,12,13,14,15";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5}, {6, 7, 8, 9, 10}, {11, 12, 13, 14, 15}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test11() {
+		sampleRaw = "1,2,3,4,5,NAN\n" + "6,7,8,9,10,NAN\n" + "11,12,13,14,15,NAN";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5, 0}, {6, 7, 8, 9, 10, 0}, {11, 12, 13, 14, 15, 0}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test12() {
+		sampleRaw = "1,2,3,4,5,NAN,,\n" + "6,7,8,9,10,NAN,,\n" + "11,12,13,14,15,NAN,,";
+		sampleMatrix = new double[][] {{1, 2, 3, 4, 5, 0, 0, 0}, {6, 7, 8, 9, 10, 0, 0, 0},
+			{11, 12, 13, 14, 15, 0, 0, 0}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test13() {
+		String[] naString = {"Nan", "NAN", "", "inf", "null", "NULL"};
+		generateRandomCSV(1000, 2000, -10, 10, 0.5, ",,", naString);
+		runGenerateReaderTest();
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderLibSVMTest.java b/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderLibSVMTest.java
new file mode 100644
index 0000000..cc8c5b9
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderLibSVMTest.java
@@ -0,0 +1,169 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.junit.Test;
+
+public class MatrixGenerateReaderLibSVMTest extends GenerateReaderMatrixTest {
+
+	private final static String TEST_NAME = "MatrixGenerateReaderLibSVMTest";
+
+	@Override
+	protected String getTestName() {
+		return TEST_NAME;
+	}
+
+	private void generateRandomLIBSVM(int firstIndex, int nrows, int ncols, double min, double max, double sparsity,
+		String separator, String indexSeparator) {
+
+		double[][] random = getRandomMatrix(nrows, ncols, min, max, sparsity, 714);
+		sampleMatrix = new double[2 * nrows][ncols];
+		StringBuilder sb = new StringBuilder();
+		int indexRow = 0;
+		for(int r = 0; r < nrows; r++) {
+			StringBuilder row1 = new StringBuilder();
+			StringBuilder row2 = new StringBuilder();
+			row1.append("+1");
+
+			for(int c = 0; c < ncols - 1; c++) {
+				if(random[r][c] > 0) {
+					sampleMatrix[indexRow][c] = random[r][c];
+					row1.append(separator).append(c + firstIndex).append(indexSeparator).append(random[r][c]);
+				}
+				else {
+					sampleMatrix[indexRow][c] = 0;
+				}
+			}
+			sampleMatrix[indexRow++][ncols - 1] = 1;
+
+			row2.append("-1");
+			for(int c = 0; c < ncols - 1; c++) {
+				if(random[r][c] < 0) {
+					sampleMatrix[indexRow][c] = random[r][c];
+					row2.append(separator).append(c + firstIndex).append(indexSeparator).append(random[r][c]);
+				}
+				else {
+					sampleMatrix[indexRow][c] = 0;
+				}
+			}
+
+			sampleMatrix[indexRow++][ncols - 1] = -1;
+
+			sb.append(row1).append("\n");
+			sb.append(row2);
+			if(r != nrows - 1)
+				sb.append("\n");
+		}
+		sampleRaw = sb.toString();
+
+	}
+
+	// Index start from 0
+	@Test
+	public void test0_1() {
+		sampleRaw = "+1 2:3 4:5 6:7\n" + "-1 8:-9 10:-11";
+		sampleMatrix = new double[][] {{0, 0, 3, 0, 5, 0, 7, 0, 0, 0, 0, +1}, {0, 0, 0, 0, 0, 0, 0, 0, -9, 0, -11, -1}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_10() {
+		sampleRaw = "-1 8:-9 10:-11\n" + "+1 2:3 4:5 6:7\n";
+		sampleMatrix = new double[][] {{0, 0, 0, 0, 0, 0, 0, 0, -9, 0, -11, -1}, {0, 0, 3, 0, 5, 0, 7, 0, 0, 0, 0, +1}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_2() {
+		generateRandomLIBSVM(0, 10, 10, -10, 10, 1, "    ", ":");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_3() {
+		generateRandomLIBSVM(0, 100, 10, -100, 100, 1, " ", ":");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_4() {
+		generateRandomLIBSVM(0, 10, 10, -100, 100, 1, " ", ":");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_5() {
+		generateRandomLIBSVM(0, 10, 10, -100, 100, 1, ",,,,", "::");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_6() {
+		sampleRaw = "+1 2.0:3.0 4:5. 6.:7\n" + "-1 8.0:9.0E0 10.0:11e0";
+		sampleMatrix = new double[][] {{0, 0, 3, 0, 5, 0, 7, 0, 0, 0, 0, +1}, {0, 0, 0, 0, 0, 0, 0, 0, 9, 0, 11, -1}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_7() {
+		sampleRaw = "+10000e-4     2:3     4:5     6:7\n" + "-1     8:9     10:11";
+		sampleMatrix = new double[][] {{0, 0, 3, 0, 5, 0, 7, 0, 0, 0, 0, +1}, {0, 0, 0, 0, 0, 0, 0, 0, 9, 0, 11, -1}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_8() {
+		sampleRaw = "+10000e-4     2:3     4:5     6:7\n" + "-0.00001e5     8:9     10:11";
+		sampleMatrix = new double[][] {{0, 0, 3, 0, 5, 0, 7, 0, 0, 0, 0, +1}, {0, 0, 0, 0, 0, 0, 0, 0, 9, 0, 11, -1}};
+		runGenerateReaderTest();
+	}
+
+	// Index start from 1
+	@Test
+	public void test2() {
+		sampleRaw = "+1 2:3 4:5 6:7\n" + "-1 8:9 10:11";
+		sampleMatrix = new double[][] {{0, 3, 0, 5, 0, 7, 0, 0, 0, 0, +1}, {0, 0, 0, 0, 0, 0, 0, 9, 0, 11, -1}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test1_2() {
+		generateRandomLIBSVM(1, 10, 10, -10, 10, 1, " ", ":");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test1_3() {
+		generateRandomLIBSVM(1, 10, 10, -100, 100, 1, " ", ":");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test1_4() {
+		generateRandomLIBSVM(0, 10, 12, -100, 100, 1, ",,,,,,", ":::::");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test1_5() {
+		generateRandomLIBSVM(1, 100, 50, -100, 100, 1, ",,,,", "::");
+		runGenerateReaderTest();
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderMatrixMarketTest.java b/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderMatrixMarketTest.java
new file mode 100644
index 0000000..0213489
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/iogen/MatrixGenerateReaderMatrixMarketTest.java
@@ -0,0 +1,238 @@
+/*
+ * 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.sysds.test.functions.iogen;
+
+import org.junit.Test;
+
+public class MatrixGenerateReaderMatrixMarketTest extends GenerateReaderMatrixTest {
+
+	private final static String TEST_NAME = "MatrixGenerateReaderMatrixMarketTest";
+
+	@Override
+	protected String getTestName() {
+		return TEST_NAME;
+	}
+
+	private void generateRandomMM(int firstIndex, int nrows, int ncols, double min, double max, double sparsity,
+		String separator) {
+
+		sampleMatrix = getRandomMatrix(nrows, ncols, min, max, sparsity, 714);
+		StringBuilder sb = new StringBuilder();
+		for(int r = 0; r < nrows; r++) {
+			for(int c = 0; c < ncols; c++) {
+				if(sampleMatrix[r][c] != 0) {
+					String rs = (r + firstIndex) + separator + (c + firstIndex) + separator + sampleMatrix[r][c];
+					sb.append(rs);
+					if(r != nrows - 1 || c != ncols - 1)
+						sb.append("\n");
+				}
+			}
+		}
+		sampleRaw = sb.toString();
+	}
+
+	private void generateRandomSymmetricMM(int firstIndex, int size, double min, double max, double sparsity,
+		String separator, boolean isUpperTriangular, boolean isSkew) {
+
+		generateRandomSymmetric(size, min, max, sparsity, isSkew);
+
+		int start, end;
+		StringBuilder sb = new StringBuilder();
+
+		for(int r = 0; r < size; r++) {
+			if(isUpperTriangular) {
+				start = r;
+				end = size;
+			}
+			else {
+				start = 0;
+				end = r + 1;
+			}
+			for(int c = start; c < end; c++) {
+				if(sampleMatrix[r][c] != 0) {
+					String rs = (r + firstIndex) + separator + (c + firstIndex) + separator + sampleMatrix[r][c];
+					sb.append(rs);
+					if(r != size - 1 || c != size - 1)
+						sb.append("\n");
+				}
+			}
+		}
+		sampleRaw = sb.toString();
+	}
+
+	// Index from 0
+	@Test
+	public void test0_1() {
+		sampleRaw = "0,1,1\n" + "0,2,4\n" + "1,2,2\n" + "2,3,3";
+		sampleMatrix = new double[][] {{0, 1, 4, 0}, {0, 0, 2, 0}, {0, 0, 0, 3}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_2() {
+		sampleRaw = "0,0,-1\n" + "0,1,1\n" + "0,2,2\n" + "0,3,3\n" + "1,0,4\n" + "1,1,5\n" + "1,2,6\n" + "1,3,7";
+		sampleMatrix = new double[][] {{-1, 1, 2, 3}, {4, 5, 6, 7}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_3() {
+		sampleRaw = "0,0,-1\n" + "0,1,1\n" + "0,2,2.0\n" + "0,3,3.\n" + "1,0,4e0\n" + "1,1,5\n" + "1,2,6\n" + "1,3,7";
+		sampleMatrix = new double[][] {{-1, 1, 2, 3}, {4, 5, 6, 7}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_4() {
+		sampleRaw = "0,0,-1\n" + "0,1,0.00001e5\n" + "0,2,2.\n" + "0,3,3\n" + "1,0,4e0\n" + "1,1,5\n" + "1,2,6\n" + "1,3,7";
+		sampleMatrix = new double[][] {{-1, 1, 2, 3}, {4, 5, 6, 7}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_5() {
+		generateRandomMM(0, 5, 10, -100, 100, 1, ",");
+		runGenerateReaderTest();
+	}
+
+	@Test public void test0_6() {
+		generateRandomMM(0, 10, 10, -100, 100, 1, ",");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_7() {
+		generateRandomMM(0, 10, 10, -100, 100, 1, "   ,");
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test0_8() {
+		generateRandomMM(0, 10, 10, -100, 100, 0.5, ",");
+		runGenerateReaderTest();
+	}
+
+	// Index from 1
+	@Test
+	public void test1() {
+		sampleRaw = "1,1,1\n" + "1,2,4\n" + "2,2,2\n" + "3,3,3";
+		sampleMatrix = new double[][] {{1, 4, 0}, {0, 2, 0}, {0, 0, 3}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void test1_2() {
+		generateRandomMM(1, 5, 100, -100, 100, 1, ",,,,,");
+		runGenerateReaderTest();
+	}
+
+	// Symmetric Tests:
+	// Symmetric Index from 0
+	@Test
+	public void SymmetricTest0_1() {
+		sampleRaw = "0,0,1\n" + "1,0,2\n" + "1,1,3\n" + "2,0,4\n" + "2,1,5\n" + "2,2,6\n" + "3,0,7\n" + "3,1,8\n" + "3,2,9\n" + "3,3,10\n";
+		sampleMatrix = new double[][] {{1, 0, 0, 0}, {2, 3, 0, 0}, {4, 5, 6, 0}, {7, 8, 9, 10}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SymmetricTest0_2() {
+		sampleRaw = "0,0,1\n" + "0,1,2\n" + "0,2,3\n" + "0,0,1\n" + "1,0,2\n" + "1,1,3\n" + "2,0,4\n" + "2,1,5\n" + "2,2,6\n" + "3,0,7\n" + "3,1,8\n" + "3,2,9\n" + "3,3,10\n";
+		sampleMatrix = new double[][] {{1, 0, 0, 0}, {2, 3, 0, 0}, {4, 5, 6, 0}, {7, 8, 9, 10}};
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SymmetricTest0_3() {
+		generateRandomSymmetricMM(0, 5, -5, 5, 1, ",", true, false);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SymmetricTest0_4() {
+		generateRandomSymmetricMM(0, 50, -100, 100, 1, "  ", true, false);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SymmetricTest0_5() {
+		generateRandomSymmetricMM(0, 5, -5, 5, 1, ",", false, false);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SymmetricTest0_6() {
+		generateRandomSymmetricMM(0, 50, -100, 100, 1, "  ", false, false);
+		runGenerateReaderTest();
+	}
+
+	// Symmetric Index from 1
+	@Test
+	public void SymmetricTest1_1() {
+		generateRandomSymmetricMM(1, 5, -5, 5, 1, ",", true, false);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SymmetricTest1_2() {
+		generateRandomSymmetricMM(1, 50, -100, 100, 1, "  ", true, false);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SymmetricTest1_3() {
+		generateRandomSymmetricMM(1, 100, -5, 5, 1, ",", false, false);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SymmetricTest1_4() {
+		generateRandomSymmetricMM(1, 200, -100, 100, 1, "  ", false, false);
+		runGenerateReaderTest();
+	}
+
+	// Skew-Symmetric Tests:
+	// Skew-Symmetric Index from 0
+	@Test
+	public void SkewSymmetricTest0_1() {
+		generateRandomSymmetricMM(0, 5, -100, 100, 1, ",", false, true);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SkewSymmetricTest0_2() {
+		generateRandomSymmetricMM(0, 5, -100, 100, 1, "   ", true, true);
+		runGenerateReaderTest();
+	}
+
+	// Skew-Symmetric Index from 1
+	@Test
+	public void SkewSymmetricTest0_3() {
+		generateRandomSymmetricMM(1, 5, -100, 100, 1, ",", false, true);
+		runGenerateReaderTest();
+	}
+
+	@Test
+	public void SkewSymmetricTest0_4() {
+		generateRandomSymmetricMM(1, 5, -100, 100, 1, "   ", true, true);
+		runGenerateReaderTest();
+	}
+
+}