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

[2/7] incubator-systemml git commit: [SYSTEMML-382] Refactoring matrix block (prep sparse block integration)

[SYSTEMML-382] Refactoring matrix block (prep sparse block integration)

This changes isolates mechanic refactorings due to renaming and
signature changes from the upcoming actual runtime integration.

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

Branch: refs/heads/master
Commit: cf7d206c64d7ccf85b5864e7d35d7c1e3bfedfce
Parents: c65e34e
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Sun Jan 17 22:36:06 2016 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sun Jan 17 23:12:10 2016 -0800

----------------------------------------------------------------------
 .../parfor/DataPartitionerLocal.java            |   5 +-
 .../parfor/DataPartitionerRemoteMapper.java     |  15 +-
 .../parfor/ResultMergeLocalFile.java            |   7 +-
 .../spark/functions/GetMLLibBlocks.java         |  10 +-
 .../spark/utils/RDDConverterUtils.java          |   2 +-
 .../sysml/runtime/io/WriterBinaryCell.java      |   5 +-
 .../sysml/runtime/io/WriterMatrixMarket.java    |   5 +-
 .../runtime/io/WriterMatrixMarketParallel.java  |   5 +-
 .../apache/sysml/runtime/io/WriterTextCSV.java  |   2 +-
 .../sysml/runtime/io/WriterTextCSVParallel.java |   2 +-
 .../apache/sysml/runtime/io/WriterTextCell.java |   5 +-
 .../runtime/io/WriterTextCellParallel.java      |   5 +-
 .../data/BinaryBlockToBinaryCellConverter.java  |  10 +-
 .../data/BinaryBlockToTextCellConverter.java    |  14 +-
 .../sysml/runtime/matrix/data/LibMatrixAgg.java |  56 +--
 .../runtime/matrix/data/LibMatrixBincell.java   |  92 ++--
 .../runtime/matrix/data/LibMatrixDatagen.java   |   4 +-
 .../runtime/matrix/data/LibMatrixMult.java      | 100 ++---
 .../runtime/matrix/data/LibMatrixOuterAgg.java  |  12 +-
 .../runtime/matrix/data/LibMatrixReorg.java     |  52 +--
 .../sysml/runtime/matrix/data/MatrixBlock.java  | 441 +++++++++----------
 .../runtime/matrix/mapred/CSVWriteReducer.java  |   3 +-
 .../runtime/matrix/mapred/ReblockBuffer.java    |   5 +-
 .../sysml/runtime/util/DataConverter.java       |  17 +-
 .../org/apache/sysml/test/utils/TestUtils.java  |   7 +-
 25 files changed, 435 insertions(+), 446 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerLocal.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerLocal.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerLocal.java
index 162330c..421ed86 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerLocal.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerLocal.java
@@ -24,6 +24,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Map.Entry;
 
@@ -38,7 +39,6 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
-
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
@@ -54,7 +54,6 @@ import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixCell;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.util.FastStringTokenizer;
 import org.apache.sysml.runtime.util.LocalFileUtils;
 
@@ -476,7 +475,7 @@ public class DataPartitionerLocal extends DataPartitioner
 						boolean sparse = value.isInSparseFormat();
 						if( sparse ) //SPARSE
 						{
-							SparseRowsIterator iter = value.getSparseRowsIterator();
+							Iterator<IJV> iter = value.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerRemoteMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerRemoteMapper.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerRemoteMapper.java
index 491dfc2..440a4cb 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerRemoteMapper.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/DataPartitionerRemoteMapper.java
@@ -20,6 +20,7 @@
 package org.apache.sysml.runtime.controlprogram.parfor;
 
 import java.io.IOException;
+import java.util.Iterator;
 
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
@@ -27,7 +28,6 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
-
 import org.apache.sysml.runtime.controlprogram.ParForProgramBlock.PDataPartitionFormat;
 import org.apache.sysml.runtime.controlprogram.parfor.util.PairWritableBlock;
 import org.apache.sysml.runtime.controlprogram.parfor.util.PairWritableCell;
@@ -37,7 +37,6 @@ import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixCell;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.matrix.mapred.MRJobConfiguration;
 import org.apache.sysml.runtime.util.FastStringTokenizer;
 import org.apache.sysml.runtime.util.MapReduceTool;
@@ -466,7 +465,7 @@ public class DataPartitionerRemoteMapper
 					case ROW_WISE:
 						if( sparse )
 						{
-							SparseRowsIterator iter = value2.getSparseRowsIterator();
+							Iterator<IJV> iter = value2.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();
@@ -500,7 +499,7 @@ public class DataPartitionerRemoteMapper
 						longKey.set((row_offset/_brlen+1));
 						if( sparse )
 						{
-							SparseRowsIterator iter = value2.getSparseRowsIterator();
+							Iterator<IJV> iter = value2.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();
@@ -534,7 +533,7 @@ public class DataPartitionerRemoteMapper
 							rowBlockIndex = ((row_offset%_n)/_brlen)+1;
 						if( sparse )
 						{
-							SparseRowsIterator iter = value2.getSparseRowsIterator();
+							Iterator<IJV> iter = value2.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();
@@ -563,7 +562,7 @@ public class DataPartitionerRemoteMapper
 					case COLUMN_WISE:
 						if( sparse )
 						{
-							SparseRowsIterator iter = value2.getSparseRowsIterator();
+							Iterator<IJV> iter = value2.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();
@@ -597,7 +596,7 @@ public class DataPartitionerRemoteMapper
 						longKey.set(col_offset/_bclen+1);
 						if( sparse )
 						{
-							SparseRowsIterator iter = value2.getSparseRowsIterator();
+							Iterator<IJV> iter = value2.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();
@@ -631,7 +630,7 @@ public class DataPartitionerRemoteMapper
 							colBlockIndex = ((col_offset%_n)/_bclen)+1;
 						if( sparse )
 						{
-							SparseRowsIterator iter = value2.getSparseRowsIterator();
+							Iterator<IJV> iter = value2.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ResultMergeLocalFile.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ResultMergeLocalFile.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ResultMergeLocalFile.java
index 5ef713b..33ee581 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ResultMergeLocalFile.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ResultMergeLocalFile.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Map.Entry;
 
@@ -39,7 +40,6 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
-
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
@@ -58,7 +58,6 @@ import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixCell;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.util.DataConverter;
 import org.apache.sysml.runtime.util.FastStringTokenizer;
 import org.apache.sysml.runtime.util.LocalFileUtils;
@@ -955,7 +954,7 @@ public class ResultMergeLocalFile extends ResultMerge
 					{
 						if( mb.isInSparseFormat() )
 						{
-							SparseRowsIterator iter = mb.getSparseRowsIterator();
+							Iterator<IJV> iter = mb.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();
@@ -1105,7 +1104,7 @@ public class ResultMergeLocalFile extends ResultMerge
 					{
 						if( mb.isInSparseFormat() )
 						{
-							SparseRowsIterator iter = mb.getSparseRowsIterator();
+							Iterator<IJV> iter = mb.getSparseBlockIterator();
 							while( iter.hasNext() )
 							{
 								IJV lcell = iter.next();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/instructions/spark/functions/GetMLLibBlocks.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/functions/GetMLLibBlocks.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/functions/GetMLLibBlocks.java
index 768a0eb..9d78650 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/functions/GetMLLibBlocks.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/functions/GetMLLibBlocks.java
@@ -20,6 +20,7 @@
 package org.apache.sysml.runtime.instructions.spark.functions;
 
 import java.util.ArrayList;
+import java.util.Iterator;
 
 import org.apache.spark.api.java.function.PairFunction;
 import org.apache.spark.mllib.linalg.DenseMatrix;
@@ -31,7 +32,6 @@ import scala.Tuple2;
 import org.apache.sysml.runtime.matrix.data.IJV;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.util.UtilFunctions;
 
 public class GetMLLibBlocks implements PairFunction<Tuple2<MatrixIndexes,MatrixBlock>, Tuple2<Object, Object>, Matrix> {
@@ -51,7 +51,7 @@ public class GetMLLibBlocks implements PairFunction<Tuple2<MatrixIndexes,MatrixB
 			return (int) blk.getNonZeros();
 		}
 		else if(blk.isInSparseFormat()) {
-			SparseRowsIterator iter = blk.getSparseRowsIterator();
+			Iterator<IJV> iter = blk.getSparseBlockIterator();
 			int nnz = 0;
 			while( iter.hasNext() ) {
 				nnz++;
@@ -59,7 +59,7 @@ public class GetMLLibBlocks implements PairFunction<Tuple2<MatrixIndexes,MatrixB
 			return nnz;
 		}
 		else {
-			return blk.getDenseArray().length;
+			return blk.getDenseBlock().length;
 		}
 	}
 	
@@ -88,7 +88,7 @@ public class GetMLLibBlocks implements PairFunction<Tuple2<MatrixIndexes,MatrixB
 		// ------------------------------------------------------------------
 				
 		if(blk.isInSparseFormat()) {
-			SparseRowsIterator iter = blk.getSparseRowsIterator();
+			Iterator<IJV> iter = blk.getSparseBlockIterator();
 			int nnz = getNNZ(blk);
 			double [] values = new double[nnz];
 			int [] rowIndices = new int[nnz];
@@ -115,7 +115,7 @@ public class GetMLLibBlocks implements PairFunction<Tuple2<MatrixIndexes,MatrixB
 			mllibBlock = new SparseMatrix(lrlen, lclen, colPtrs, rowIndices, values);
 		}
 		else {
-			mllibBlock = new DenseMatrix(lrlen, lclen, blk.getDenseArray());
+			mllibBlock = new DenseMatrix(lrlen, lclen, blk.getDenseBlock());
 		}
 		return new Tuple2<Tuple2<Object,Object>, Matrix>(new Tuple2<Object,Object>(blockRowIndex, blockColIndex), mllibBlock);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java
index 9681803..656a618 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java
@@ -280,7 +280,7 @@ public class RDDConverterUtils
 				}
 				else if( tmp.isInSparseFormat() ) //SPARSE ROW
 				{
-					SparseRow row = tmp.getSparseRows()[0]; 
+					SparseRow row = tmp.getSparseBlock()[0]; 
 					int rlen = row.size();
 					int[] rix = row.getIndexContainer();
 					double[] rvals = row.getValueContainer();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/io/WriterBinaryCell.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/io/WriterBinaryCell.java b/src/main/java/org/apache/sysml/runtime/io/WriterBinaryCell.java
index 4b6bc5b..7f5d4b2 100644
--- a/src/main/java/org/apache/sysml/runtime/io/WriterBinaryCell.java
+++ b/src/main/java/org/apache/sysml/runtime/io/WriterBinaryCell.java
@@ -20,12 +20,12 @@
 package org.apache.sysml.runtime.io;
 
 import java.io.IOException;
+import java.util.Iterator;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.mapred.JobConf;
-
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
@@ -33,7 +33,6 @@ import org.apache.sysml.runtime.matrix.data.IJV;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixCell;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.util.MapReduceTool;
 
 public class WriterBinaryCell extends MatrixWriter
@@ -110,7 +109,7 @@ public class WriterBinaryCell extends MatrixWriter
 			if( sparse ) //SPARSE
 			{
 				
-				SparseRowsIterator iter = src.getSparseRowsIterator();
+				Iterator<IJV> iter = src.getSparseBlockIterator();
 				while( iter.hasNext() )
 				{
 					IJV lcell = iter.next();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarket.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarket.java b/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarket.java
index 0f9bb06..3615573 100644
--- a/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarket.java
+++ b/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarket.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
+import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -32,13 +33,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapred.JobConf;
-
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.matrix.data.IJV;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.util.MapReduceTool;
 
 /**
@@ -121,7 +120,7 @@ public class WriterMatrixMarket extends MatrixWriter
             // output matrix cell
 			if( sparse ) //SPARSE
 			{			   
-				SparseRowsIterator iter = src.getSparseRowsIterator();
+				Iterator<IJV> iter = src.getSparseBlockIterator();
 				while( iter.hasNext() )
 				{
 					IJV cell = iter.next();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarketParallel.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarketParallel.java b/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarketParallel.java
index 23c2e51..eae4c46 100644
--- a/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarketParallel.java
+++ b/src/main/java/org/apache/sysml/runtime/io/WriterMatrixMarketParallel.java
@@ -23,6 +23,7 @@ import java.io.BufferedWriter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
@@ -32,14 +33,12 @@ import java.util.concurrent.Future;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
-
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysml.runtime.matrix.data.IJV;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.util.MapReduceTool;
 
 /**
@@ -152,7 +151,7 @@ public class WriterMatrixMarketParallel extends WriterMatrixMarket
 		        
 				if( _src.isInSparseFormat() ) //SPARSE
 				{			   
-					SparseRowsIterator iter = _src.getSparseRowsIterator(_rl, _ru);
+					Iterator<IJV> iter = _src.getSparseBlockIterator(_rl, _ru);
 
 					while( iter.hasNext() )
 					{

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/io/WriterTextCSV.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/io/WriterTextCSV.java b/src/main/java/org/apache/sysml/runtime/io/WriterTextCSV.java
index b6f56eb..96e31db 100644
--- a/src/main/java/org/apache/sysml/runtime/io/WriterTextCSV.java
+++ b/src/main/java/org/apache/sysml/runtime/io/WriterTextCSV.java
@@ -138,7 +138,7 @@ public class WriterTextCSV extends MatrixWriter
 			// Write data lines
 			if( sparse ) //SPARSE
 			{	
-				SparseRow[] sparseRows = src.getSparseRows();
+				SparseRow[] sparseRows = src.getSparseBlock();
 				for(int i=0; i < rlen; i++) 
 	            {
 					//write row chunk-wise to prevent OOM on large number of columns

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/io/WriterTextCSVParallel.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/io/WriterTextCSVParallel.java b/src/main/java/org/apache/sysml/runtime/io/WriterTextCSVParallel.java
index e1e849b..9758dc5 100644
--- a/src/main/java/org/apache/sysml/runtime/io/WriterTextCSVParallel.java
+++ b/src/main/java/org/apache/sysml/runtime/io/WriterTextCSVParallel.java
@@ -172,7 +172,7 @@ public class WriterTextCSVParallel extends WriterTextCSV
 				// Write data lines
 				if( sparse ) //SPARSE
 				{	
-					SparseRow[] sparseRows = _src.getSparseRows();
+					SparseRow[] sparseRows = _src.getSparseBlock();
 					for( int i=_rl; i<_ru; i++ )
 					{
 						//write row chunk-wise to prevent OOM on large number of columns

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/io/WriterTextCell.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/io/WriterTextCell.java b/src/main/java/org/apache/sysml/runtime/io/WriterTextCell.java
index 1c6eb69..66f6ec4 100644
--- a/src/main/java/org/apache/sysml/runtime/io/WriterTextCell.java
+++ b/src/main/java/org/apache/sysml/runtime/io/WriterTextCell.java
@@ -22,18 +22,17 @@ package org.apache.sysml.runtime.io;
 import java.io.BufferedWriter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
+import java.util.Iterator;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
-
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.matrix.data.IJV;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.util.MapReduceTool;
 
 public class WriterTextCell extends MatrixWriter
@@ -106,7 +105,7 @@ public class WriterTextCell extends MatrixWriter
 			
 			if( sparse ) //SPARSE
 			{			   
-				SparseRowsIterator iter = src.getSparseRowsIterator();
+				Iterator<IJV> iter = src.getSparseBlockIterator();
 				while( iter.hasNext() )
 				{
 					IJV cell = iter.next();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/io/WriterTextCellParallel.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/io/WriterTextCellParallel.java b/src/main/java/org/apache/sysml/runtime/io/WriterTextCellParallel.java
index 809f33f..6c1312c 100644
--- a/src/main/java/org/apache/sysml/runtime/io/WriterTextCellParallel.java
+++ b/src/main/java/org/apache/sysml/runtime/io/WriterTextCellParallel.java
@@ -23,6 +23,7 @@ import java.io.BufferedWriter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
@@ -32,14 +33,12 @@ import java.util.concurrent.Future;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
-
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysml.runtime.matrix.data.IJV;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.OutputInfo;
-import org.apache.sysml.runtime.matrix.data.SparseRowsIterator;
 import org.apache.sysml.runtime.util.MapReduceTool;
 
 public class WriterTextCellParallel extends WriterTextCell
@@ -139,7 +138,7 @@ public class WriterTextCellParallel extends WriterTextCell
 				
 				if( _src.isInSparseFormat() ) //SPARSE
 				{			   
-					SparseRowsIterator iter = _src.getSparseRowsIterator(_rl, _ru);
+					Iterator<IJV> iter = _src.getSparseBlockIterator(_rl, _ru);
 					
 					while( iter.hasNext() )
 					{

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToBinaryCellConverter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToBinaryCellConverter.java b/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToBinaryCellConverter.java
index fcbb79c..e4fefe4 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToBinaryCellConverter.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToBinaryCellConverter.java
@@ -20,6 +20,8 @@
 
 package org.apache.sysml.runtime.matrix.data;
 
+import java.util.Iterator;
+
 import org.apache.sysml.runtime.util.UtilFunctions;
 
 
@@ -28,7 +30,7 @@ public class BinaryBlockToBinaryCellConverter implements
 Converter<MatrixIndexes, MatrixBlock, MatrixIndexes, MatrixCell>
 {
 	
-	private SparseRowsIterator sparseIterator=null;
+	private Iterator<IJV> sparseIterator=null;
 	private double[] denseArray=null;
 	private int denseArraySize=0;
 	private int nextInDenseArray=-1;
@@ -62,13 +64,13 @@ Converter<MatrixIndexes, MatrixBlock, MatrixIndexes, MatrixCell>
 		thisBlockWidth=v1.getNumColumns();
 		if(sparse)
 		{
-			sparseIterator=v1.getSparseRowsIterator();
+			sparseIterator=v1.getSparseBlockIterator();
 		}
 		else
 		{
-			if(v1.getDenseArray()==null)
+			if(v1.getDenseBlock()==null)
 				return;
-			denseArray=v1.getDenseArray();
+			denseArray=v1.getDenseBlock();
 			nextInDenseArray=0;
 			denseArraySize=v1.getNumRows()*v1.getNumColumns();
 		}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToTextCellConverter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToTextCellConverter.java b/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToTextCellConverter.java
index da63d64..0004adc 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToTextCellConverter.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/BinaryBlockToTextCellConverter.java
@@ -21,18 +21,18 @@
 package org.apache.sysml.runtime.matrix.data;
 
 
+import java.util.Iterator;
+
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
-
 import org.apache.sysml.runtime.util.UtilFunctions;
 
 
 
 public class BinaryBlockToTextCellConverter implements 
 Converter<MatrixIndexes, MatrixBlock, NullWritable, Text>
-{
-	
-	private SparseRowsIterator sparseIterator=null;
+{	
+	private Iterator<IJV> sparseIterator=null;
 	private double[] denseArray=null;
 	private int denseArraySize=0;
 	private int nextInDenseArray=-1;
@@ -68,13 +68,13 @@ Converter<MatrixIndexes, MatrixBlock, NullWritable, Text>
 		thisBlockWidth=v1.getNumColumns();
 		if(sparse)
 		{
-			sparseIterator=v1.getSparseRowsIterator();
+			sparseIterator=v1.getSparseBlockIterator();
 		}
 		else
 		{
-			if(v1.getDenseArray()==null)
+			if(v1.getDenseBlock()==null)
 				return;
-			denseArray=v1.getDenseArray();
+			denseArray=v1.getDenseBlock();
 			nextInDenseArray=0;
 			denseArraySize=v1.getNumRows()*v1.getNumColumns();
 		}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixAgg.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixAgg.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixAgg.java
index ee20f25..f856e06 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixAgg.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixAgg.java
@@ -512,7 +512,7 @@ public class LibMatrixAgg
 		if( (type == AggType.MAX_INDEX || type == AggType.MIN_INDEX) && ix.getColumnIndex()!=1 ) //MAXINDEX or MININDEX
 		{
 			int m = out.rlen;
-			double[] c = out.getDenseArray();
+			double[] c = out.getDenseBlock();
 			for( int i=0, cix=0; i<m; i++, cix+=2 )
 				c[cix] = UtilFunctions.cellIndexCalculation(ix.getColumnIndex(), bclen, (int)c[cix]-1);
 		}
@@ -689,7 +689,7 @@ public class LibMatrixAgg
 		
 		if( in1.sparse )
 		{
-			SparseRow[] a = in1.sparseRows;
+			SparseRow[] a = in1.sparseBlock;
 			
 			for( int i=rl; i<ru; i++ )
 				if( a[i]!=null && !a[i].isEmpty() ) {
@@ -757,11 +757,11 @@ public class LibMatrixAgg
 			
 			if( target.sparse ) //SPARSE target
 			{
-				if( target.sparseRows[0]!=null )
+				if( target.sparseBlock[0]!=null )
 				{
-					int len = target.sparseRows[0].size();
-					int[] aix = target.sparseRows[0].getIndexContainer();
-					double[] avals = target.sparseRows[0].getValueContainer();	
+					int len = target.sparseBlock[0].size();
+					int[] aix = target.sparseBlock[0].getIndexContainer();
+					double[] avals = target.sparseBlock[0].getValueContainer();	
 					for( int j=0; j<len; j++ ) //for each nnz
 					{
 						int g = (int) groups.quickGetValue(aix[j], 0);		
@@ -795,7 +795,7 @@ public class LibMatrixAgg
 		{
 			if( target.sparse ) //SPARSE target
 			{
-				SparseRow[] a = target.sparseRows;
+				SparseRow[] a = target.sparseBlock;
 				
 				for( int i=0; i < groups.getNumRows(); i++ ) 
 				{
@@ -873,7 +873,7 @@ public class LibMatrixAgg
 		//column vector or matrix
 		if( target.sparse ) //SPARSE target
 		{
-			SparseRow[] a = target.sparseRows;
+			SparseRow[] a = target.sparseBlock;
 			
 			for( int i=0; i < groups.getNumRows(); i++ ) 
 			{
@@ -976,9 +976,9 @@ public class LibMatrixAgg
 		aggVal.allocateDenseBlock(); //should always stay in dense
 		aggCorr.allocateDenseBlock(); //should always stay in dense
 		
-		double[] a = in.getDenseArray();
-		double[] c = aggVal.getDenseArray();
-		double[] cc = aggCorr.getDenseArray();
+		double[] a = in.getDenseBlock();
+		double[] c = aggVal.getDenseBlock();
+		double[] cc = aggCorr.getDenseBlock();
 		
 		KahanObject buffer1 = new KahanObject(0, 0);
 		KahanPlus akplus = KahanPlus.getKahanPlusFnObject();
@@ -1015,16 +1015,16 @@ public class LibMatrixAgg
 	private static void aggregateBinaryMatrixSparseDense(MatrixBlock in, MatrixBlock aggVal, MatrixBlock aggCorr) 
 			throws DMLRuntimeException
 	{
-		if( in.sparseRows==null || in.isEmptyBlock(false) )
+		if( in.isEmptyBlock(false) )
 			return;
 		
 		//allocate output arrays (if required)
 		aggVal.allocateDenseBlock(); //should always stay in dense
 		aggCorr.allocateDenseBlock(); //should always stay in dense
 		
-		SparseRow[] a = in.getSparseRows();
-		double[] c = aggVal.getDenseArray();
-		double[] cc = aggCorr.getDenseArray();
+		SparseRow[] a = in.getSparseBlock();
+		double[] c = aggVal.getDenseBlock();
+		double[] cc = aggCorr.getDenseBlock();
 		
 		KahanObject buffer1 = new KahanObject(0, 0);
 		KahanPlus akplus = KahanPlus.getKahanPlusFnObject();
@@ -1070,10 +1070,10 @@ public class LibMatrixAgg
 	private static void aggregateBinaryMatrixSparseGeneric(MatrixBlock in, MatrixBlock aggVal, MatrixBlock aggCorr) 
 			throws DMLRuntimeException
 	{
-		if( in.sparseRows==null || in.isEmptyBlock(false) )
+		if( in.isEmptyBlock(false) )
 			return;
 		
-		SparseRow[] a = in.getSparseRows();
+		SparseRow[] a = in.getSparseBlock();
 		
 		KahanObject buffer1 = new KahanObject(0, 0);
 		KahanPlus akplus = KahanPlus.getKahanPlusFnObject();
@@ -1124,7 +1124,7 @@ public class LibMatrixAgg
 		final int m = in.rlen;
 		final int n = in.clen;
 		
-		double[] a = in.getDenseArray();
+		double[] a = in.getDenseBlock();
 		
 		KahanObject buffer = new KahanObject(0, 0);
 		KahanPlus akplus = KahanPlus.getKahanPlusFnObject();
@@ -1161,7 +1161,7 @@ public class LibMatrixAgg
 		final int n = in.clen;
 		final int cix = (m-1)*n;
 		
-		double[] a = in.getDenseArray();
+		double[] a = in.getDenseBlock();
 		
 		KahanObject buffer = new KahanObject(0, 0);
 		KahanPlus akplus = KahanPlus.getKahanPlusFnObject();
@@ -1191,10 +1191,10 @@ public class LibMatrixAgg
 			throws DMLRuntimeException
 	{
 		//sparse-safe operation
-		if( in.sparseRows==null || in.isEmptyBlock(false) )
+		if( in.isEmptyBlock(false) )
 			return;
 		
-		SparseRow[] a = in.getSparseRows();
+		SparseRow[] a = in.getSparseBlock();
 		
 		KahanObject buffer1 = new KahanObject(0, 0);
 		KahanPlus akplus = KahanPlus.getKahanPlusFnObject();
@@ -1243,7 +1243,7 @@ public class LibMatrixAgg
 		final int m = in.rlen;
 		final int n = in.clen;
 		
-		double[] a = in.getDenseArray();
+		double[] a = in.getDenseBlock();
 		
 		KahanObject buffer = new KahanObject(0, 0);
 		KahanPlus akplus = KahanPlus.getKahanPlusFnObject();
@@ -1273,10 +1273,10 @@ public class LibMatrixAgg
 			throws DMLRuntimeException
 	{
 		//sparse-safe operation
-		if( in.sparseRows==null || in.isEmptyBlock(false) )
+		if( in.isEmptyBlock(false) )
 			return;
 		
-		SparseRow[] a = in.getSparseRows();
+		SparseRow[] a = in.getSparseBlock();
 		
 		KahanObject buffer1 = new KahanObject(0, 0);
 		KahanPlus akplus = KahanPlus.getKahanPlusFnObject();
@@ -1325,8 +1325,8 @@ public class LibMatrixAgg
 		final int m = in.rlen;
 		final int n = in.clen;
 		
-		double[] a = in.getDenseArray();
-		double[] c = out.getDenseArray();		
+		double[] a = in.getDenseBlock();
+		double[] c = out.getDenseBlock();		
 		
 		switch( optype )
 		{
@@ -1455,8 +1455,8 @@ public class LibMatrixAgg
 		final int m = in.rlen;
 		final int n = in.clen;
 		
-		SparseRow[] a = in.getSparseRows();
-		double[] c = out.getDenseArray();
+		SparseRow[] a = in.getSparseBlock();
+		double[] c = out.getDenseBlock();
 		
 		switch( optype )
 		{

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixBincell.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixBincell.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixBincell.java
index cead016..1147c5e 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixBincell.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixBincell.java
@@ -262,12 +262,12 @@ public class LibMatrixBincell
 					ret.allocateSparseRowsBlock();	
 				
 				//both sparse blocks existing
-				if(m1.sparseRows!=null && m2.sparseRows!=null)
+				if(m1.sparseBlock!=null && m2.sparseBlock!=null)
 				{
 					for(int r=0; r<rlen; r++)
 					{
-						SparseRow lrow = (m1.sparseRows.length>r && m1.sparseRows[r]!=null) ? m1.sparseRows[r] : null; 
-						SparseRow rrow = (m2.sparseRows.length>r && m2.sparseRows[r]!=null) ? m2.sparseRows[r] : null; 
+						SparseRow lrow = (m1.sparseBlock.length>r && m1.sparseBlock[r]!=null) ? m1.sparseBlock[r] : null; 
+						SparseRow rrow = (m2.sparseBlock.length>r && m2.sparseBlock[r]!=null) ? m2.sparseBlock[r] : null; 
 						
 						if( lrow!=null && rrow!=null)
 						{
@@ -289,23 +289,23 @@ public class LibMatrixBincell
 					}
 				}
 				//right sparse block existing
-				else if( m2.sparseRows!=null )
+				else if( m2.sparseBlock!=null )
 				{
-					for(int r=0; r<Math.min(rlen, m2.sparseRows.length); r++)
-						if(m2.sparseRows[r]!=null)
+					for(int r=0; r<Math.min(rlen, m2.sparseBlock.length); r++)
+						if(m2.sparseBlock[r]!=null)
 						{
-							appendRightForSparseBinary(op, m2.sparseRows[r].getValueContainer(), 
-									m2.sparseRows[r].getIndexContainer(), m2.sparseRows[r].size(), 0, r, ret);
+							appendRightForSparseBinary(op, m2.sparseBlock[r].getValueContainer(), 
+									m2.sparseBlock[r].getIndexContainer(), m2.sparseBlock[r].size(), 0, r, ret);
 						}
 				}
 				//left sparse block existing
 				else
 				{
 					for(int r=0; r<rlen; r++)
-						if( m1.sparseRows[r]!=null )
+						if( m1.sparseBlock[r]!=null )
 						{
-							appendLeftForSparseBinary(op, m1.sparseRows[r].getValueContainer(), 
-									m1.sparseRows[r].getIndexContainer(), m1.sparseRows[r].size(), 0, r, ret);
+							appendLeftForSparseBinary(op, m1.sparseBlock[r].getValueContainer(), 
+									m1.sparseBlock[r].getIndexContainer(), m1.sparseBlock[r].size(), 0, r, ret);
 						}
 				}
 			}
@@ -328,10 +328,10 @@ public class LibMatrixBincell
 				{
 					Arrays.fill(ret.denseBlock, 0, ret.denseBlock.length, 0); 
 					
-					if( m1.sparseRows != null )
+					if( m1.sparseBlock != null )
 					{
 						for( int i=0, ix=0; i<m; i++, ix+=n ) {
-							SparseRow arow = m1.sparseRows[i];
+							SparseRow arow = m1.sparseBlock[i];
 							if( arow != null && !arow.isEmpty() )
 							{
 								alen = arow.size();
@@ -354,10 +354,10 @@ public class LibMatrixBincell
 				//2) process right input: op.fn (+,-,*), * only if dense
 				if( m2.sparse ) //SPARSE right
 				{				
-					if(m2.sparseRows!=null)
+					if(m2.sparseBlock!=null)
 					{
 						for( int i=0, ix=0; i<m; i++, ix+=n ) {
-							SparseRow arow = m2.sparseRows[i];
+							SparseRow arow = m2.sparseBlock[i];
 							if( arow != null && !arow.isEmpty() )
 							{
 								alen = arow.size();
@@ -509,7 +509,7 @@ public class LibMatrixBincell
 		
 		int rlen = m1.rlen;
 		int clen = m1.clen;
-		SparseRow[] a = m1.sparseRows;
+		SparseRow[] a = m1.sparseBlock;
 		BinaryAccessType atype = getBinaryAccessType(m1, m2);
 		
 		//early abort on skip and empty
@@ -672,7 +672,7 @@ public class LibMatrixBincell
 			if( m2.sparse && isMultiply ) //SPARSE *
 			{
 				//note: sparse block guaranteed to be allocated (otherwise early about)
-				SparseRow brow = m2.sparseRows[0];
+				SparseRow brow = m2.sparseBlock[0];
 				if( brow != null && !brow.isEmpty() ) 
 				{
 					int blen = brow.size();
@@ -786,12 +786,12 @@ public class LibMatrixBincell
 				if(bOp.fn instanceof LessThan || bOp.fn instanceof GreaterThanEquals 
 						|| bOp.fn instanceof GreaterThan || bOp.fn instanceof LessThanEquals 
 						|| bOp.fn instanceof Equals)	{
-					Arrays.fill(mbOut.getDenseArray(), iOffSet+iStartPos, iOffSet+iEndPos, 1.0);
+					Arrays.fill(mbOut.getDenseBlock(), iOffSet+iStartPos, iOffSet+iEndPos, 1.0);
 					lNNZ += (iEndPos-iStartPos);
 				}
 				else if (bOp.fn instanceof NotEquals) {
-					Arrays.fill(mbOut.getDenseArray(), iOffSet, iOffSet+iStartPos, 1.0);
-					Arrays.fill(mbOut.getDenseArray(), iOffSet+iEndPos, iOffSet+bv.length, 1.0);
+					Arrays.fill(mbOut.getDenseBlock(), iOffSet, iOffSet+iStartPos, 1.0);
+					Arrays.fill(mbOut.getDenseBlock(), iOffSet+iEndPos, iOffSet+bv.length, 1.0);
 					lNNZ += (iStartPos+(bv.length-iEndPos));
 				}
 			}
@@ -921,10 +921,10 @@ public class LibMatrixBincell
 		{	
 			//allocate sparse row structure
 			ret.allocateSparseRowsBlock();
-			SparseRow[] a = m1.sparseRows;
-			SparseRow[] c = ret.sparseRows;
+			SparseRow[] a = m1.sparseBlock;
+			SparseRow[] c = ret.sparseBlock;
 			
-			for(int r=0; r<Math.min(m1.rlen, m1.sparseRows.length); r++) {
+			for(int r=0; r<Math.min(m1.rlen, m1.sparseBlock.length); r++) {
 				if( a[r]!=null && !a[r].isEmpty() )
 				{
 					int alen = a[r].size();
@@ -1012,7 +1012,7 @@ public class LibMatrixBincell
 		{
 			ret.allocateDenseBlock();
 			
-			SparseRow[] a = m1.sparseRows;
+			SparseRow[] a = m1.sparseBlock;
 			double[] c = ret.denseBlock;			
 			int m = m1.rlen;
 			int n = m1.clen;
@@ -1082,61 +1082,61 @@ public class LibMatrixBincell
 		
 		if(m1ret.sparse && m2.sparse)
 		{
-			if(m1ret.sparseRows!=null)
+			if(m1ret.sparseBlock!=null)
 				m1ret.allocateSparseRowsBlock(false);
-			if(m2.sparseRows!=null)
+			if(m2.sparseBlock!=null)
 				m2.allocateSparseRowsBlock(false);
 			
-			if(m1ret.sparseRows!=null && m2.sparseRows!=null)
+			if(m1ret.sparseBlock!=null && m2.sparseBlock!=null)
 			{
 				for(int r=0; r<rlen; r++)
 				{
-					if(m1ret.sparseRows[r]==null && m2.sparseRows[r]==null)
+					if(m1ret.sparseBlock[r]==null && m2.sparseBlock[r]==null)
 						continue;
 					
-					if(m2.sparseRows[r]==null)
+					if(m2.sparseBlock[r]==null)
 					{
-						double[] values=m1ret.sparseRows[r].getValueContainer();
-						for(int i=0; i<m1ret.sparseRows[r].size(); i++)
+						double[] values=m1ret.sparseBlock[r].getValueContainer();
+						for(int i=0; i<m1ret.sparseBlock[r].size(); i++)
 							values[i]=op.fn.execute(values[i], 0);
 					}else
 					{
 						int estimateSize=0;
-						if(m1ret.sparseRows[r]!=null)
-							estimateSize+=m1ret.sparseRows[r].size();
-						if(m2.sparseRows[r]!=null)
-							estimateSize+=m2.sparseRows[r].size();
+						if(m1ret.sparseBlock[r]!=null)
+							estimateSize+=m1ret.sparseBlock[r].size();
+						if(m2.sparseBlock[r]!=null)
+							estimateSize+=m2.sparseBlock[r].size();
 						estimateSize=Math.min(clen, estimateSize);
 						
 						//temp
-						SparseRow thisRow=m1ret.sparseRows[r];
-						m1ret.sparseRows[r]=new SparseRow(estimateSize, clen);
+						SparseRow thisRow=m1ret.sparseBlock[r];
+						m1ret.sparseBlock[r]=new SparseRow(estimateSize, clen);
 						
 						if(thisRow!=null)
 						{
 							m1ret.nonZeros-=thisRow.size();
 							mergeForSparseBinary(op, thisRow.getValueContainer(), 
 									thisRow.getIndexContainer(), thisRow.size(),
-									m2.sparseRows[r].getValueContainer(), 
-									m2.sparseRows[r].getIndexContainer(), m2.sparseRows[r].size(), r, m1ret);
+									m2.sparseBlock[r].getValueContainer(), 
+									m2.sparseBlock[r].getIndexContainer(), m2.sparseBlock[r].size(), r, m1ret);
 							
 						}else
 						{
-							appendRightForSparseBinary(op, m2.sparseRows[r].getValueContainer(), 
-									m2.sparseRows[r].getIndexContainer(), m2.sparseRows[r].size(), 0, r, m1ret);
+							appendRightForSparseBinary(op, m2.sparseBlock[r].getValueContainer(), 
+									m2.sparseBlock[r].getIndexContainer(), m2.sparseBlock[r].size(), 0, r, m1ret);
 						}
 					}
 				}	
 			}
-			else if(m1ret.sparseRows==null)
+			else if(m1ret.sparseBlock==null)
 			{
-				m1ret.sparseRows=new SparseRow[rlen];
+				m1ret.sparseBlock=new SparseRow[rlen];
 				for(int r=0; r<rlen; r++)
 				{
-					SparseRow brow = m2.sparseRows[r];
+					SparseRow brow = m2.sparseBlock[r];
 					if( brow!=null && !brow.isEmpty() )
 					{
-						m1ret.sparseRows[r] = new SparseRow( brow.size(), clen );
+						m1ret.sparseBlock[r] = new SparseRow( brow.size(), clen );
 						appendRightForSparseBinary(op, brow.getValueContainer(), brow.getIndexContainer(), brow.size(), 0, r, m1ret);
 					}
 				}				
@@ -1145,7 +1145,7 @@ public class LibMatrixBincell
 			{
 				if( !(op.fn instanceof Plus || op.fn instanceof Minus || op.fn instanceof Or) ){
 					for(int r=0; r<rlen; r++){
-						SparseRow arow = m1ret.sparseRows[r];
+						SparseRow arow = m1ret.sparseBlock[r];
 						if( arow!=null && !arow.isEmpty() )
 						{
 							int alen = arow.size();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDatagen.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDatagen.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDatagen.java
index 7844ac3..96b929f 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDatagen.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDatagen.java
@@ -671,7 +671,7 @@ public class LibMatrixDatagen
 				// irrelevant but we need to ensure consistency with MR)
 				boolean localSparse = MatrixBlock.evalSparseFormatInMemory(blockrows, blockcols, nnzInBlocks[blockID] ); //(long)(sparsity*blockrows*blockcols));  
 				if ( localSparse ) {
-					SparseRow[] c = out.sparseRows;
+					SparseRow[] c = out.sparseBlock;
 					
 					int idx = 0;  // takes values in range [1, brlen*bclen] (both ends including)
 					int ridx=0, cidx=0; // idx translates into (ridx, cidx) entry within the block
@@ -716,7 +716,7 @@ public class LibMatrixDatagen
 							 * 
 							 */
 							// In this case, entire matrix is in sparse format but the current block is dense
-							SparseRow[] c = out.sparseRows;
+							SparseRow[] c = out.sparseBlock;
 							for(int ii=0; ii < blockrows; ii++) {
 								for(int jj=0; jj < blockcols; jj++) {
 									if(nnzPRNG.nextDouble() <= sparsity) {

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java
index 50247b9..f617f40 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java
@@ -1170,7 +1170,7 @@ public class LibMatrixMult
 			final int blocksizeK = 32; 
 			//note: in contrast to dense-dense, no blocking over j (would require maintaining blocksizeK indexes, counter-productive on skew)
 			
-			SparseRow[] b = m2.sparseRows;
+			SparseRow[] b = m2.sparseBlock;
 			
 			if( pm2 && m==1 )          //VECTOR-MATRIX
 			{
@@ -1219,7 +1219,7 @@ public class LibMatrixMult
 					double val = a[aix];
 					if( val!=0 )
 					{
-						SparseRow brow = m2.sparseRows[ k ];
+						SparseRow brow = m2.sparseBlock[ k ];
 						if( brow != null && !brow.isEmpty() ) 
 						{
 							int blen = brow.size();
@@ -1253,7 +1253,7 @@ public class LibMatrixMult
 		{
 			if( m==1 && n==1 )         //DOT PRODUCT
 			{
-				SparseRow arow = m1.sparseRows[0];
+				SparseRow arow = m1.sparseBlock[0];
 				if( arow != null && !arow.isEmpty() )
 				{
 					int alen = arow.size();
@@ -1265,9 +1265,9 @@ public class LibMatrixMult
 			}
 			else if( n==1 )            //MATRIX-VECTOR
 			{
-				for( int i=rl; i<Math.min(ru, m1.sparseRows.length); i++ )
+				for( int i=rl; i<Math.min(ru, m1.sparseBlock.length); i++ )
 				{
-					SparseRow arow = m1.sparseRows[i];
+					SparseRow arow = m1.sparseBlock[i];
 					if( arow != null && !arow.isEmpty() ) 
 					{
 						int alen = arow.size();
@@ -1281,7 +1281,7 @@ public class LibMatrixMult
 			else if( pm2 && m==1 )     //VECTOR-MATRIX
 			{
 				//parallelization over rows in rhs matrix
-				SparseRow arow = m1.sparseRows[0];
+				SparseRow arow = m1.sparseBlock[0];
 				if( arow != null && !arow.isEmpty() ) 
 				{
 					int alen = arow.size();
@@ -1300,7 +1300,7 @@ public class LibMatrixMult
 			}
 			else if( pm2 && m<=16 )    //MATRIX-MATRIX (short lhs) 
 			{
-				SparseRow[] a = m1.sparseRows;
+				SparseRow[] a = m1.sparseBlock;
 				for( int i=0, cix=0; i<a.length; i++, cix+=n )
 					if( a[i] != null && !a[i].isEmpty() ) 
 					{
@@ -1330,9 +1330,9 @@ public class LibMatrixMult
 			}
 			else                       //MATRIX-MATRIX
 			{
-				for( int i=rl, cix=rl*n; i<Math.min(ru, m1.sparseRows.length); i++, cix+=n )
+				for( int i=rl, cix=rl*n; i<Math.min(ru, m1.sparseBlock.length); i++, cix+=n )
 				{
-					SparseRow arow = m1.sparseRows[i];
+					SparseRow arow = m1.sparseBlock[i];
 					if( arow != null && !arow.isEmpty() ) 
 					{
 						int alen = arow.size();
@@ -1366,9 +1366,9 @@ public class LibMatrixMult
 		}
 		else
 		{
-			for( int i=rl, cix=rl*n; i<Math.min(ru, m1.sparseRows.length); i++, cix+=n )
+			for( int i=rl, cix=rl*n; i<Math.min(ru, m1.sparseBlock.length); i++, cix+=n )
 			{
-				SparseRow arow = m1.sparseRows[i];
+				SparseRow arow = m1.sparseBlock[i];
 				if( arow != null && !arow.isEmpty() ) 
 				{
 					int alen = arow.size();
@@ -1396,7 +1396,7 @@ public class LibMatrixMult
 	private static void matrixMultSparseSparse(MatrixBlock m1, MatrixBlock m2, MatrixBlock ret, boolean pm2, int rl, int ru) 
 		throws DMLRuntimeException
 	{	
-		SparseRow[] b = m2.sparseRows;
+		SparseRow[] b = m2.sparseBlock;
 		double[] c = ret.denseBlock;
 		int m = m1.rlen;
 		int n = m2.clen;
@@ -1407,7 +1407,7 @@ public class LibMatrixMult
 			if( pm2 && m==1 )          //VECTOR-MATRIX
 			{
 				//parallelization over rows in rhs matrix
-				SparseRow arow = m1.sparseRows[0];
+				SparseRow arow = m1.sparseBlock[0];
 				if( arow != null && !arow.isEmpty() ) 
 				{
 					int alen = arow.size();
@@ -1428,9 +1428,9 @@ public class LibMatrixMult
 			}	
 			else                       //MATRIX-MATRIX
 			{
-				for( int i=rl, cix=rl*n; i<Math.min(ru, m1.sparseRows.length); i++, cix+=n )
+				for( int i=rl, cix=rl*n; i<Math.min(ru, m1.sparseBlock.length); i++, cix+=n )
 				{
-					SparseRow arow = m1.sparseRows[i];
+					SparseRow arow = m1.sparseBlock[i];
 					if( arow != null && !arow.isEmpty() ) 
 					{
 						int alen = arow.size();
@@ -1456,9 +1456,9 @@ public class LibMatrixMult
 		}
 		else
 		{
-			for( int i=rl, cix=rl*n; i<Math.min(ru, m1.sparseRows.length); i++, cix+=n )
+			for( int i=rl, cix=rl*n; i<Math.min(ru, m1.sparseBlock.length); i++, cix+=n )
 			{
-				SparseRow arow = m1.sparseRows[i];
+				SparseRow arow = m1.sparseBlock[i];
 				if( arow != null && !arow.isEmpty() ) 
 				{
 					int alen = arow.size();
@@ -1468,7 +1468,7 @@ public class LibMatrixMult
 					for(int k = 0; k < alen; k++) 
 					{
 						double val = avals[k];
-						SparseRow brow = m2.sparseRows[ aix[k] ];
+						SparseRow brow = m2.sparseBlock[ aix[k] ];
 						if( brow != null && !brow.isEmpty() ) 
 						{
 							int blen = brow.size();
@@ -1510,7 +1510,7 @@ public class LibMatrixMult
 			
 			for( int i=rl; i<ru; i++ )
 			{
-				SparseRow arow = m1.sparseRows[ i ];
+				SparseRow arow = m1.sparseBlock[ i ];
 				if( arow != null && !arow.isEmpty() ) 
 				{
 					int alen = arow.size();
@@ -1521,11 +1521,11 @@ public class LibMatrixMult
 					{
 						int aix = aixs[0];
 						if( rightSparse ) { //sparse right matrix (full row copy)
-							if( m2.sparseRows!=null && m2.sparseRows[aix]!=null ) {
+							if( m2.sparseBlock!=null && m2.sparseBlock[aix]!=null ) {
 								ret.rlen=m;
 								ret.allocateSparseRowsBlock(false); //allocation on demand
-								ret.sparseRows[i] = new SparseRow(m2.sparseRows[aix]); 
-								ret.nonZeros += ret.sparseRows[i].size();
+								ret.sparseBlock[i] = new SparseRow(m2.sparseBlock[aix]); 
+								ret.nonZeros += ret.sparseBlock[i].size();
 							}
 						}
 						else { //dense right matrix (append all values)
@@ -1555,7 +1555,7 @@ public class LibMatrixMult
 		{
 			for(int k = 0; k < cd; k++ ) 
 			{			
-				SparseRow brow = m2.sparseRows[ k ];
+				SparseRow brow = m2.sparseBlock[ k ];
 				if( brow != null && !brow.isEmpty() ) 
 				{
 					int blen = brow.size();
@@ -1647,7 +1647,7 @@ public class LibMatrixMult
 	 */
 	private static void matrixMultChainSparse(MatrixBlock mX, MatrixBlock mV, MatrixBlock mW, MatrixBlock ret, ChainType ct, int rl, int ru) 
 	{
-		SparseRow[] a = mX.sparseRows;
+		SparseRow[] a = mX.sparseBlock;
 		double[] b = mV.denseBlock;
 		double[] w = (mW!=null) ? mW.denseBlock : null;
 		double[] c = ret.denseBlock;
@@ -1858,7 +1858,7 @@ public class LibMatrixMult
 			//algorithm: scan rows, foreach row self join (KIJ)
 			if( LOW_LEVEL_OPTIMIZATION )
 			{
-				for( SparseRow arow : m1.sparseRows )
+				for( SparseRow arow : m1.sparseBlock )
 					if( arow != null && !arow.isEmpty() ) 
 					{
 						int alen = arow.size();
@@ -1879,7 +1879,7 @@ public class LibMatrixMult
 			}
 			else
 			{
-				for( SparseRow arow : m1.sparseRows )
+				for( SparseRow arow : m1.sparseBlock )
 					if( arow != null && !arow.isEmpty() ) 
 					{
 						int alen = arow.size();
@@ -1902,7 +1902,7 @@ public class LibMatrixMult
 		{
 			if( m==1 ) //VECTOR 
 			{
-				SparseRow arow = m1.sparseRows[0];
+				SparseRow arow = m1.sparseBlock[0];
 				if( arow !=null && !arow.isEmpty() )
 				{
 					int alen = arow.size();
@@ -1921,7 +1921,7 @@ public class LibMatrixMult
 				//algorithm: scan rows, foreach row self join (KIJ)
 				if( LOW_LEVEL_OPTIMIZATION )
 				{
-					for( SparseRow arow : m1.sparseRows )
+					for( SparseRow arow : m1.sparseBlock )
 						if( arow != null && !arow.isEmpty() ) 
 						{
 							int alen = arow.size();
@@ -1942,7 +1942,7 @@ public class LibMatrixMult
 				}
 				else
 				{
-					for( SparseRow arow : m1.sparseRows )
+					for( SparseRow arow : m1.sparseBlock )
 						if( arow != null && !arow.isEmpty() ) 
 						{
 							int alen = arow.size();
@@ -2023,7 +2023,7 @@ public class LibMatrixMult
 	{
 		double[] a = pm1.denseBlock;
 		double[] b = m2.denseBlock;
-		SparseRow[] c = ret1.sparseRows;
+		SparseRow[] c = ret1.sparseBlock;
 
 		final int n = m2.clen;
 		final int brlen = ret1.getNumRows();
@@ -2044,7 +2044,7 @@ public class LibMatrixMult
 					ret2.sparse = true;
 					ret2.rlen=ret1.rlen;
 					ret2.allocateSparseRowsBlock();
-					c = ret2.sparseRows;		
+					c = ret2.sparseBlock;		
 				}
 		
 				//append entire dense row into sparse target position
@@ -2069,8 +2069,8 @@ public class LibMatrixMult
 	private static void matrixMultPermuteSparse( MatrixBlock pm1, MatrixBlock m2, MatrixBlock ret1, MatrixBlock ret2, int rl, int ru)
 	{
 		double[] a = pm1.denseBlock;
-		SparseRow[] b = m2.sparseRows;
-		SparseRow[] c = ret1.sparseRows;
+		SparseRow[] b = m2.sparseBlock;
+		SparseRow[] c = ret1.sparseBlock;
 
 		final int brlen = ret1.getNumRows();
 		
@@ -2089,7 +2089,7 @@ public class LibMatrixMult
 				if( lastblk!=-1 && lastblk<blk ){ 
 					ret2.sparse = true;
 					ret2.allocateSparseRowsBlock();
-					c = ret2.sparseRows;		
+					c = ret2.sparseBlock;		
 				}
 		
 				//memcopy entire sparse row into target position
@@ -2198,8 +2198,8 @@ public class LibMatrixMult
 	 */
 	private static void matrixMultWSLossSparseDense(MatrixBlock mX, MatrixBlock mU, MatrixBlock mV, MatrixBlock mW, MatrixBlock ret, WeightsType wt, int rl, int ru)
 	{
-		SparseRow[] x = mX.sparseRows;
-		SparseRow[] w = (mW!=null)? mW.sparseRows : null;
+		SparseRow[] x = mX.sparseBlock;
+		SparseRow[] w = (mW!=null)? mW.sparseBlock : null;
 		double[] u = mU.denseBlock;
 		double[] v = mV.denseBlock;
 		final int n = mX.clen; 
@@ -2316,7 +2316,7 @@ public class LibMatrixMult
 			// approach: iterate over W, point-wise in order to exploit sparsity
 			if( mW.sparse ) //SPARSE
 			{
-				SparseRow[] wrows = mW.sparseRows;
+				SparseRow[] wrows = mW.sparseBlock;
 				
 				for( int i=rl; i<ru; i++ )
 					if( wrows[i] != null && !wrows[i].isEmpty() ){
@@ -2349,7 +2349,7 @@ public class LibMatrixMult
 			// approach: iterate over W, point-wise in order to exploit sparsity
 			if( mW.sparse ) //SPARSE
 			{
-				SparseRow[] xrows = mX.sparseRows;
+				SparseRow[] xrows = mX.sparseBlock;
 				
 				for( int i=rl; i<ru; i++ )
 					if( xrows[i] != null && !xrows[i].isEmpty() ){
@@ -2469,8 +2469,8 @@ public class LibMatrixMult
 	private static void matrixMultWSigmoidSparseDense(MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WSigmoidType wt, int rl, int ru) 
 		throws DMLRuntimeException
 	{
-		SparseRow[] w = mW.sparseRows;
-		SparseRow[] c = ret.sparseRows;
+		SparseRow[] w = mW.sparseBlock;
+		SparseRow[] c = ret.sparseBlock;
 		double[] u = mU.denseBlock;
 		double[] v = mV.denseBlock;
 		final int n = mW.clen; 
@@ -2519,8 +2519,8 @@ public class LibMatrixMult
 		if( mW.sparse ) //SPARSE
 		{
 			//w and c always in same representation
-			SparseRow[] w = mW.sparseRows;
-			SparseRow[] c = ret.sparseRows;
+			SparseRow[] w = mW.sparseBlock;
+			SparseRow[] c = ret.sparseBlock;
 			
 			for( int i=rl; i<ru; i++ )
 				if( w[i] != null && !w[i].isEmpty() ) {
@@ -2622,7 +2622,7 @@ public class LibMatrixMult
 		final boolean minus = wt.isMinus();
 		final int cd = mU.clen;
 		
-		SparseRow[] w = mW.sparseRows;
+		SparseRow[] w = mW.sparseBlock;
 		double[] u = mU.denseBlock;
 		double[] v = mV.denseBlock;
 		double[] c = ret.denseBlock;
@@ -2676,7 +2676,7 @@ public class LibMatrixMult
 		//approach: iterate over non-zeros of w, selective mm computation
 		if( mW.sparse ) //SPARSE
 		{
-			SparseRow[] w = mW.sparseRows;
+			SparseRow[] w = mW.sparseBlock;
 			
 			for( int i=rl; i<ru; i++ ) {
 				if( w[i] != null && !w[i].isEmpty() ) {
@@ -2769,7 +2769,7 @@ public class LibMatrixMult
 	 */
 	private static void matrixMultWCeMMSparseDense(MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WCeMMType wt, int rl, int ru)
 	{
-		SparseRow[] w = mW.sparseRows;
+		SparseRow[] w = mW.sparseBlock;
 		double[] u = mU.denseBlock;
 		double[] v = mV.denseBlock;
 		final int cd = mU.clen;
@@ -2811,7 +2811,7 @@ public class LibMatrixMult
 		//approach: iterate over non-zeros of w, selective mm computation
 		if( mW.sparse ) //SPARSE
 		{
-			SparseRow[] w = mW.sparseRows;
+			SparseRow[] w = mW.sparseBlock;
 			
 			for( int i=rl; i<ru; i++ )
 				if( w[i] != null && !w[i].isEmpty() ) {
@@ -2905,8 +2905,8 @@ public class LibMatrixMult
 	private static void matrixMultWuMMSparseDense(MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WUMMType wt, ValueFunction fn, int rl, int ru) 
 		throws DMLRuntimeException
 	{
-		SparseRow[] w = mW.sparseRows;
-		SparseRow[] c = ret.sparseRows;
+		SparseRow[] w = mW.sparseBlock;
+		SparseRow[] c = ret.sparseBlock;
 		double[] u = mU.denseBlock;
 		double[] v = mV.denseBlock;
 		final int n = mW.clen; 
@@ -2953,8 +2953,8 @@ public class LibMatrixMult
 		if( mW.sparse ) //SPARSE
 		{
 			//w and c always in same representation
-			SparseRow[] w = mW.sparseRows;
-			SparseRow[] c = ret.sparseRows;
+			SparseRow[] w = mW.sparseBlock;
+			SparseRow[] c = ret.sparseBlock;
 			
 			for( int i=rl; i<ru; i++ )
 				if( w[i] != null && !w[i].isEmpty() ) {

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixOuterAgg.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixOuterAgg.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixOuterAgg.java
index 01c7751..c46e2c5 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixOuterAgg.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixOuterAgg.java
@@ -894,14 +894,14 @@ public class LibMatrixOuterAgg
 
 		//allocate and initialize output values (not indices) 
 		out.allocateDenseBlock(true);
-		Arrays.fill(out.getDenseArray(), 0, out.getNumColumns(), agg0);
+		Arrays.fill(out.getDenseBlock(), 0, out.getNumColumns(), agg0);
 		if(agg0 != 0.0)
 			out.setNonZeros(out.getNumColumns());
 		
 		if( in.isEmptyBlock(false) )
 			return;
 			
-		SparseRow[] aSparseRows = in.getSparseRows();		
+		SparseRow[] aSparseRows = in.getSparseBlock();		
 		for (int j = 0; j < aSparseRows.length; ++j)
 		if( aSparseRows[j]!=null && !aSparseRows[j].isEmpty() )
 		{
@@ -954,14 +954,14 @@ public class LibMatrixOuterAgg
 
 		//allocate and initialize output values (not indices) 
 		out.allocateDenseBlock(true);
-		Arrays.fill(out.getDenseArray(), 0, out.getNumColumns(), agg0);
+		Arrays.fill(out.getDenseBlock(), 0, out.getNumColumns(), agg0);
 		if(agg0 != 0.0)
 			out.setNonZeros(out.getNumColumns());
 		
 		if( in.isEmptyBlock(false) )
 			return;
 			
-		SparseRow[] aSparseRows = in.getSparseRows();		
+		SparseRow[] aSparseRows = in.getSparseBlock();		
 		for (int j = 0; j < aSparseRows.length; ++j)
 		if( aSparseRows[j]!=null && !aSparseRows[j].isEmpty() )
 		{
@@ -1014,14 +1014,14 @@ public class LibMatrixOuterAgg
 
 		//allocate and initialize output values (not indices) 
 		out.allocateDenseBlock(true);
-		Arrays.fill(out.getDenseArray(), 0, out.getNumColumns(), agg0);
+		Arrays.fill(out.getDenseBlock(), 0, out.getNumColumns(), agg0);
 		if(agg0 != 0.0)
 			out.setNonZeros(out.getNumColumns());
 		
 		if( in.isEmptyBlock(false) )
 			return;
 			
-		SparseRow[] aSparseRows = in.getSparseRows();		
+		SparseRow[] aSparseRows = in.getSparseBlock();		
 		for (int j = 0; j < aSparseRows.length; ++j)
 		if( aSparseRows[j]!=null && !aSparseRows[j].isEmpty() )
 		{

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cf7d206c/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixReorg.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixReorg.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixReorg.java
index 545d425..4ea4d1a 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixReorg.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixReorg.java
@@ -365,8 +365,8 @@ public class LibMatrixReorg
 				out.allocateSparseRowsBlock(false);
 				for( int i=0; i<rlen; i++ ) {
 					int ix = vix[i];
-					if( in.sparseRows[ix]!=null && !in.sparseRows[ix].isEmpty() ) {
-						out.sparseRows[i] = new SparseRow(in.sparseRows[ix]);
+					if( in.sparseBlock[ix]!=null && !in.sparseBlock[ix].isEmpty() ) {
+						out.sparseBlock[i] = new SparseRow(in.sparseBlock[ix]);
 					}
 				}
 			}
@@ -751,8 +751,8 @@ public class LibMatrixReorg
 		//allocate output arrays (if required)
 		out.allocateDenseBlock(false);
 		
-		double[] a = in.getDenseArray();
-		double[] c = out.getDenseArray();
+		double[] a = in.getDenseBlock();
+		double[] c = out.getDenseBlock();
 		
 		if( m==1 || n==1 ) //VECTOR TRANSPOSE
 		{
@@ -798,8 +798,8 @@ public class LibMatrixReorg
 		out.reset(m2, n2, true); //always sparse
 		out.allocateSparseRowsBlock();
 				
-		double[] a = in.getDenseArray();
-		SparseRow[] c = out.getSparseRows();
+		double[] a = in.getDenseBlock();
+		SparseRow[] c = out.getSparseBlock();
 		
 		//blocking according to typical L2 cache sizes 
 		final int blocksizeI = 128;
@@ -841,8 +841,8 @@ public class LibMatrixReorg
 		out.reset(m2, n2, true); //always sparse
 		out.allocateSparseRowsBlock();
 		
-		SparseRow[] a = in.getSparseRows();
-		SparseRow[] c = out.getSparseRows();
+		SparseRow[] a = in.getSparseBlock();
+		SparseRow[] c = out.getSparseBlock();
 
 		//initial pass to determine capacity (this helps to prevent
 		//sparse row reallocations and mem inefficiency w/ skew
@@ -920,8 +920,8 @@ public class LibMatrixReorg
 		out.reset(m2, n2, false); //always dense
 		out.allocateDenseBlock();
 		
-		SparseRow[] a = in.getSparseRows();
-		double[] c = out.getDenseArray();
+		SparseRow[] a = in.getSparseBlock();
+		double[] c = out.getDenseBlock();
 		
 		if( m==1 ) //ROW VECTOR TRANSPOSE
 		{
@@ -1020,8 +1020,8 @@ public class LibMatrixReorg
 		out.nonZeros = in.nonZeros;
 		out.allocateDenseBlock(false);
 		
-		double[] a = in.getDenseArray();
-		double[] c = out.getDenseArray();
+		double[] a = in.getDenseBlock();
+		double[] c = out.getDenseBlock();
 		
 		//copy all rows into target positions
 		if( n == 1 ) { //column vector
@@ -1051,8 +1051,8 @@ public class LibMatrixReorg
 		
 		out.allocateSparseRowsBlock(false);
 		
-		SparseRow[] a = in.getSparseRows();
-		SparseRow[] c = out.getSparseRows();
+		SparseRow[] a = in.getSparseBlock();
+		SparseRow[] c = out.getSparseBlock();
 		
 		//copy all rows into target positions
 		for( int i=0; i<m; i++ ) {
@@ -1200,8 +1200,8 @@ public class LibMatrixReorg
 		int estnnz = (int) (in.nonZeros/rows);
 		
 		//sparse reshape
-		SparseRow[] aRows = in.sparseRows;
-		SparseRow[] cRows = out.sparseRows;
+		SparseRow[] aRows = in.sparseBlock;
+		SparseRow[] cRows = out.sparseBlock;
 		
 		if( rowwise )
 		{
@@ -1323,13 +1323,13 @@ public class LibMatrixReorg
 			return;
 		
 		//allocate block if necessary
-		if(out.sparseRows==null)
-			out.sparseRows=new SparseRow[rows];
+		if(out.sparseBlock==null)
+			out.sparseBlock=new SparseRow[rows];
 		int estnnz = (int) (in.nonZeros/rows);
 		
 		//sparse reshape
 		double[] a = in.denseBlock;
-		SparseRow[] cRows = out.sparseRows;
+		SparseRow[] cRows = out.sparseBlock;
 		
 		if( rowwise )
 		{
@@ -1403,14 +1403,14 @@ public class LibMatrixReorg
 		int clen = in.clen;
 		
 		//reshape empty block
-		if( in.sparseRows == null )
+		if( in.sparseBlock == null )
 			return;
 		
 		//allocate block if necessary
 		out.allocateDenseBlock(false);
 		
 		//sparse/dense reshape
-		SparseRow[] aRows = in.sparseRows;
+		SparseRow[] aRows = in.sparseBlock;
 		double[] c = out.denseBlock;
 		
 		if( rowwise )
@@ -1699,7 +1699,7 @@ public class LibMatrixReorg
 			return;
 		
 		int rlen = in.rlen;
-		SparseRow[] aRows = in.sparseRows;
+		SparseRow[] aRows = in.sparseBlock;
 		
 		//append all values to right blocks
 		MatrixIndexes ixtmp = new MatrixIndexes();
@@ -1831,7 +1831,7 @@ public class LibMatrixReorg
 			
 			if( in.sparse ) //SPARSE 
 			{
-				SparseRow[] a = in.sparseRows;
+				SparseRow[] a = in.sparseBlock;
 				
 				for ( int i=0; i < m; i++ )
 					if ( a[i] != null && !a[i].isEmpty() ) {
@@ -1872,7 +1872,7 @@ public class LibMatrixReorg
 			//note: output dense or sparse
 			for( int i=0, cix=0; i<m; i++ )
 				if( flags[i] )
-					ret.appendRow(cix++, in.sparseRows[i]);
+					ret.appendRow(cix++, in.sparseBlock[i]);
 		}
 		else if( !in.sparse && !ret.sparse )  //DENSE <- DENSE
 		{
@@ -1930,7 +1930,7 @@ public class LibMatrixReorg
 			flags = new boolean[ n ]; //false
 			if( in.sparse ) //SPARSE 
 			{
-				SparseRow[] a = in.sparseRows;
+				SparseRow[] a = in.sparseBlock;
 				
 				for( int i=0; i<m; i++ ) 
 					if ( a[i] != null && !a[i].isEmpty() ) {
@@ -1978,7 +1978,7 @@ public class LibMatrixReorg
 		if( in.sparse ) //* <- SPARSE 
 		{
 			//note: output dense or sparse
-			SparseRow[] a = in.sparseRows;
+			SparseRow[] a = in.sparseBlock;
 			
 			for( int i=0; i<m; i++ ) 
 				if ( a[i] != null && !a[i].isEmpty() ) {