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/20 23:59:18 UTC

[3/5] incubator-systemml git commit: [SYSTEMML-382] Initial runtime integration sparse matrix blocks

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
index fe691b7..4cd00ca 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
@@ -94,6 +94,8 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	public static final double SPARSITY_TURN_POINT = 0.4;
 	//sparsity threshold for ultra-sparse matrix operations (40nnz in a 1kx1k block)
 	public static final double ULTRA_SPARSITY_TURN_POINT = 0.00004; 
+	//default sparse block type: modified compressed sparse rows 
+	public static final SparseBlock.Type DEFAULT_SPARSEBLOCK = SparseBlock.Type.MCSR;
 	//basic header (int rlen, int clen, byte type)
 	public static final int HEADER_SIZE = 9;
 	
@@ -112,7 +114,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	
 	//matrix data (sparse or dense)
 	protected double[] denseBlock    = null;
-	protected SparseRow[] sparseBlock = null;
+	protected SparseBlock sparseBlock = null;
 		
 	//sparse-block-specific attributes (allocation only)
 	protected int estimatedNNzsPerRow = -1; 
@@ -221,25 +223,19 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		reset(estnnzs);
 	}
 		
-	public void reset(int rl, int cl, boolean sp)
-	{
+	public void reset(int rl, int cl, boolean sp) {
 		sparse=sp;
 		reset(rl, cl);
 	}
 	
-	public void reset(int rl, int cl, boolean sp, long estnnzs)
-	{
+	public void reset(int rl, int cl, boolean sp, long estnnzs) {
 		sparse=sp;
 		reset(rl, cl, estnnzs);
 	}
 	
-	public void resetSparse()
-	{
-		if(sparseBlock!=null)
-		{
-			for(int i=0; i<Math.min(rlen, sparseBlock.length); i++)
-				if(sparseBlock[i]!=null)
-					sparseBlock[i].reset(estimatedNNzsPerRow, clen);
+	public void resetSparse() {
+		if(sparseBlock != null) {
+			sparseBlock.reset(estimatedNNzsPerRow, clen);
 		}
 	}
 	
@@ -386,8 +382,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	/**
 	 * 
 	 */
-	public void allocateDenseOrSparseBlock()
-	{
+	public void allocateDenseOrSparseBlock() {
 		if( sparse )
 			allocateSparseRowsBlock();
 		else
@@ -424,8 +419,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	/**
 	 * 
 	 */
-	public void allocateSparseRowsBlock()
-	{
+	public void allocateSparseRowsBlock() {
 		allocateSparseRowsBlock(true);
 	}
 	
@@ -435,16 +429,9 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	 */
 	public void allocateSparseRowsBlock(boolean clearNNZ)
 	{	
-		//allocate block if non-existing or too small (guaranteed to be 0-initialized),
-		if( sparseBlock == null ) {
-			sparseBlock=new SparseRow[rlen];
-		}
-		else if( sparseBlock.length < rlen ) {
-			SparseRow[] oldSparseRows=sparseBlock;
-			sparseBlock = new SparseRow[rlen];
-			for(int i=0; i<Math.min(oldSparseRows.length, rlen); i++) {
-				sparseBlock[i]=oldSparseRows[i];
-			}
+		//allocate block if non-existing or too small (guaranteed to be 0-initialized)
+		if( sparseBlock == null || sparseBlock.numRows()<rlen ) {
+			sparseBlock = SparseBlockFactory.createSparseBlock(DEFAULT_SPARSEBLOCK, rlen);
 		}
 		
 		//clear nnz if necessary
@@ -481,8 +468,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	 * from non-empty blocks would significantly increase the total memory consumption.
 	 * 
 	 */
-	public void cleanupBlock( boolean dense, boolean sparse )
-	{
+	public void cleanupBlock( boolean dense, boolean sparse ) {
 		if(dense)
 			denseBlock = null;
 		if(sparse)
@@ -492,8 +478,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	////////
 	// Metadata information 
 	
-	public int getNumRows()
-	{
+	public int getNumRows() {
 		return rlen;
 	}
 	
@@ -503,23 +488,19 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	 * 
 	 * @param _r
 	 */
-	public void setNumRows(int r) 
-	{
+	public void setNumRows(int r) {
 		rlen = r;
 	}
 	
-	public int getNumColumns()
-	{
+	public int getNumColumns() {
 		return clen;
 	}
 	
-	public void setNumColumns(int c) 
-	{
+	public void setNumColumns(int c) {
 		clen = c;
 	}
 	
-	public long getNonZeros()
-	{
+	public long getNonZeros() {
 		return nonZeros;
 	}
 	
@@ -527,8 +508,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		nonZeros = nnz;
 	}
 	
-	public boolean isVector() 
-	{
+	public boolean isVector() {
 		return (rlen == 1 || clen == 1);
 	}
 	
@@ -537,16 +517,14 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	 * Return the maximum row encountered WITHIN the current block
 	 *  
 	 */
-	public int getMaxRow() 
-	{
+	public int getMaxRow() {
 		if (!sparse) 
 			return getNumRows();
 		else 
 			return maxrow;
 	}
 	
-	public void setMaxRow(int r)
-	{
+	public void setMaxRow(int r) {
 		maxrow = r;
 	}
 	
@@ -555,27 +533,23 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	 * Return the maximum column encountered WITHIN the current block
 	 * 
 	 */
-	public int getMaxColumn() 
-	{
+	public int getMaxColumn() {
 		if (!sparse) 
 			return getNumColumns();
 		else 
 			return maxcolumn;
 	}
 	
-	public void setMaxColumn(int c) 
-	{
+	public void setMaxColumn(int c) {
 		maxcolumn = c;
 	}
 	
 	@Override
-	public boolean isEmpty()
-	{
+	public boolean isEmpty() {
 		return isEmptyBlock(false);
 	}
 	
-	public boolean isEmptyBlock()
-	{
+	public boolean isEmptyBlock() {
 		return isEmptyBlock(true);
 	}
 	
@@ -597,49 +571,66 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		return ret;
 	}
 	
-	public void setDiag()
-	{
+	public void setDiag() {
 		diag = true;
 	}
 	
-	public boolean isDiag()
-	{
+	public boolean isDiag() {
 		return diag;
 	}
 	
 	////////
 	// Data handling
 	
-	public double[] getDenseBlock()
-	{
-		if(sparse)
+	public double[] getDenseBlock() {
+		if( sparse )
 			return null;
 		return denseBlock;
 	}
 	
-	public SparseRow[] getSparseBlock()
-	{
-		if(!sparse)
+	public SparseBlock getSparseBlock() {
+		if( !sparse )
 			return null;
 		return sparseBlock;
 	}
 	
-	public Iterator<IJV> getSparseBlockIterator()
-	{
+	/**
+	 * 
+	 * @return
+	 */
+	public Iterator<IJV> getSparseBlockIterator() {
 		//check for valid format, should have been checked from outside
 		if( !sparse )
-			throw new RuntimeException("getSparseCellInterator should not be called for dense format");
+			throw new RuntimeException("getSparseBlockInterator should not be called for dense format");
 		
-		return new SparseRowsIterator(rlen, sparseBlock);
+		//check for existing sparse block: return empty list
+		if( sparseBlock==null )
+			return new ArrayList<IJV>().iterator();
+		
+		//get iterator over sparse block
+		if( rlen == sparseBlock.numRows() )
+			return sparseBlock.getIterator();
+		else
+			return sparseBlock.getIterator(rlen);
 	}
 	
-	public SparseRowsIterator getSparseBlockIterator(int rl, int ru)
-	{
+	/**
+	 * 
+	 * @param rl
+	 * @param ru
+	 * @return
+	 */
+	public Iterator<IJV> getSparseBlockIterator(int rl, int ru) {
 		//check for valid format, should have been checked from outside
 		if( !sparse )
-			throw new RuntimeException("getSparseCellInterator should not be called for dense format");
+			throw new RuntimeException("getSparseBlockInterator should not be called for dense format");
 		
-		return new SparseRowsIterator(rl, ru, sparseBlock);
+		//check for existing sparse block: return empty list
+		if( sparseBlock==null )
+			return new ArrayList<IJV>().iterator();
+		
+		//get iterator over sparse block
+		return sparseBlock.getIterator(rl, ru);
 	}
 	
 	@Override
@@ -672,9 +663,9 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	{
 		if(sparse)
 		{
-			if( sparseBlock==null || sparseBlock.length<=r || sparseBlock[r]==null )
+			if( sparseBlock==null )
 				return 0;
-			return sparseBlock[r].get(c);
+			return sparseBlock.get(r, c);
 		}
 		else
 		{
@@ -695,16 +686,15 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		if(sparse)
 		{
 			//early abort
-			if( (sparseBlock==null || sparseBlock.length<=r || sparseBlock[r]==null) && v==0 )
+			if( (sparseBlock==null || sparseBlock.isEmpty(r)) && v==0 )
 				return;
 			
 			//allocation on demand
 			allocateSparseRowsBlock(false);
-			if( sparseBlock[r]==null )
-				sparseBlock[r] = new SparseRow(estimatedNNzsPerRow, clen);
+			sparseBlock.allocate(r, estimatedNNzsPerRow, clen);
 			
 			//set value and maintain nnz
-			if( sparseBlock[r].set(c, v) )
+			if( sparseBlock.set(r, c, v) )
 				nonZeros += (v!=0) ? 1 : -1;
 		}
 		else
@@ -746,9 +736,9 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	
 	public double getValueSparseUnsafe(int r, int c) 
 	{
-		if(sparseBlock==null || sparseBlock.length<=r || sparseBlock[r]==null)
+		if(sparseBlock==null || sparseBlock.isEmpty(r))
 			return 0;
-		return sparseBlock[r].get(c);	
+		return sparseBlock.get(r, c);	
 	}
 	
 	/**
@@ -778,11 +768,10 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		{
 			//allocation on demand (w/o overwriting nnz)
 			allocateSparseRowsBlock(false);
-			if(sparseBlock[r]==null)
-				sparseBlock[r]=new SparseRow(estimatedNNzsPerRow, clen);
+			sparseBlock.allocate(r, estimatedNNzsPerRow, clen);
 			
 			//set value and maintain nnz
-			sparseBlock[r].append(c, v);
+			sparseBlock.append(r, c, v);
 			nonZeros++;
 		}
 	}
@@ -795,17 +784,21 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		{
 			//allocation on demand
 			allocateSparseRowsBlock(false);
-			if(sparseBlock[r]==null)
-				sparseBlock[r]=new SparseRow(values);
-			else
-				sparseBlock[r].copy(values);
+			sparseBlock.allocate(r, values.size(), -1);
+			
+			//TODO perf sparse block
+			int[] cols=values.indexes();
+			double[] vals=values.values();
+			for(int i=0; i<values.size(); i++)
+				sparseBlock.append(r, cols[i], vals[i]);
+			
 			nonZeros+=values.size();
 			
 		}
 		else
 		{
-			int[] cols=values.getIndexContainer();
-			double[] vals=values.getValueContainer();
+			int[] cols=values.indexes();
+			double[] vals=values.values();
 			for(int i=0; i<values.size(); i++)
 				quickSetValue(r, cols[i], vals[i]);
 		}
@@ -829,19 +822,17 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		{
 			for( int i=0; i<that.rlen; i++ )
 			{
-				SparseRow brow = that.sparseBlock[i];
-				if( brow!=null && !brow.isEmpty() )
-				{
+				SparseBlock b = that.sparseBlock;
+				if( !b.isEmpty(i) ) {
 					int aix = rowoffset+i;
-					int len = brow.size();
-					int[] ix = brow.getIndexContainer();
-					double[] val = brow.getValueContainer();
-					
-					if( sparseBlock[aix]==null )
-						sparseBlock[aix] = new SparseRow(estimatedNNzsPerRow,clen);
+					int pos = b.pos(i);
+					int len = b.size(i);
+					int[] ix = b.indexes(i);
+					double[] val = b.values(i);
 					
-					for( int j=0; j<len; j++ )
-						sparseBlock[aix].append(coloffset+ix[j], val[j]);		
+					sparseBlock.allocate(aix, estimatedNNzsPerRow,clen);
+					for( int j=pos; j<pos+len; j++ )
+						sparseBlock.append(aix, coloffset+ix[j], val[j]);		
 				}
 			}
 		}
@@ -853,11 +844,10 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 				for( int j=0, bix=i*that.clen; j<that.clen; j++ )
 				{
 					double val = that.denseBlock[bix+j];
-					if( val != 0 )
-					{
-						if( sparseBlock[aix]==null )//create sparserow only if required
-							sparseBlock[aix] = new SparseRow(estimatedNNzsPerRow,clen);
-						sparseBlock[aix].append(coloffset+j, val);
+					if( val != 0 ) {
+						//create sparserow only if required
+						sparseBlock.allocate(aix, estimatedNNzsPerRow,clen);
+						sparseBlock.append(aix, coloffset+j, val);
 					}
 				}
 			}
@@ -867,14 +857,10 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	/**
 	 * 
 	 */
-	public void sortSparseRows()
-	{
+	public void sortSparseRows() {
 		if( !sparse || sparseBlock==null )
-			return;
-		
-		for( SparseRow arow : sparseBlock )
-			if( arow!=null && arow.size()>1 )
-				arow.sort();
+			return;		
+		sparseBlock.sort();
 	}
 	
 	/**
@@ -1174,14 +1160,14 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		
 		//copy dense to sparse
 		double[] a = denseBlock;
-		SparseRow[] c = sparseBlock;
+		SparseBlock c = sparseBlock;
 		
 		for( int i=0, aix=0; i<rlen; i++ )
 			for(int j=0; j<clen; j++, aix++)
 				if( a[aix] != 0 ) {
-					if( c[i]==null ) //create sparse row only if required
-						c[i]=new SparseRow(estimatedNNzsPerRow, clen);
-					c[i].append(j, a[aix]);
+					//create sparse row only if required
+					c.allocate(i, estimatedNNzsPerRow, clen);
+					c.append(i, j, a[aix]);
 					nonZeros++;
 				}
 				
@@ -1213,15 +1199,16 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		Arrays.fill(denseBlock, 0, limit, 0);
 		
 		//copy sparse to dense
-		SparseRow[] a = sparseBlock;
+		SparseBlock a = sparseBlock;
 		double[] c = denseBlock;
 		
 		for( int i=0, cix=0; i<rlen; i++, cix+=clen)
-			if( a[i] != null && !a[i].isEmpty() ) {
-				int alen = a[i].size();
-				int[] aix = a[i].getIndexContainer();
-				double[] avals = a[i].getValueContainer();
-				for(int j=0; j<alen; j++)
+			if( !a.isEmpty(i) ) {
+				int apos = a.pos(i);
+				int alen = a.size(i);
+				int[] aix = a.indexes(i);
+				double[] avals = a.values(i);
+				for(int j=apos; j<apos+alen; j++)
 					if( avals[j] != 0 )
 						c[ cix+aix[j] ] = avals[j];
 			}
@@ -1235,16 +1222,13 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		nonZeros=0;
 		if( sparse && sparseBlock!=null )
 		{
-			int limit = Math.min(rlen, sparseBlock.length);
-			for(int i=0; i<limit; i++)
-				if(sparseBlock[i]!=null)
-					nonZeros += sparseBlock[i].size();
+			//note: rlen might be <= sparseBlock.numRows()
+			nonZeros = sparseBlock.size(0, rlen);
 		}
 		else if( !sparse && denseBlock!=null )
 		{
 			int limit=rlen*clen;
-			for(int i=0; i<limit; i++)
-			{
+			for(int i=0; i<limit; i++) {
 				//HotSpot JVM bug causes crash in presence of NaNs 
 				//nonZeros += (denseBlock[i]!=0) ? 1 : 0;
 				if( denseBlock[i]!=0 )
@@ -1260,30 +1244,20 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		{
 			if(sparseBlock!=null)
 			{
-				int rlimit = Math.min( ru+1, Math.min(rlen, sparseBlock.length) );
+				int rlimit = Math.min( ru+1, rlen);
 				if( cl==0 && cu==clen-1 ) //specific case: all cols
 				{
-					for(int i=rl; i<rlimit; i++)
-						if(sparseBlock[i]!=null && !sparseBlock[i].isEmpty())
-							nnz+=sparseBlock[i].size();	
+					nnz = sparseBlock.size(rl, ru+1);
 				}
 				else if( cl==cu ) //specific case: one column
 				{
 					for(int i=rl; i<rlimit; i++)
-						if(sparseBlock[i]!=null && !sparseBlock[i].isEmpty())
-							nnz += (sparseBlock[i].get(cl)!=0) ? 1 : 0;
+						if(!sparseBlock.isEmpty(i))
+							nnz += (sparseBlock.get(i, cl)!=0) ? 1 : 0;
 				}
 				else //general case
 				{
-					int astart,aend;
-					for(int i=rl; i<rlimit; i++)
-						if(sparseBlock[i]!=null && !sparseBlock[i].isEmpty())
-						{
-							SparseRow arow = sparseBlock[i];
-							astart = arow.searchIndexesFirstGTE(cl);
-							aend = arow.searchIndexesFirstGTE(cu);
-							nnz += (astart!=-1) ? (aend-astart+1) : 0;
-						}
+					nnz = sparseBlock.size(rl, ru+1, cl, cu+1);
 				}
 			}
 		}
@@ -1365,16 +1339,14 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		}
 	
 		allocateSparseRowsBlock(false);
-		for(int i=0; i<Math.min(that.sparseBlock.length, rlen); i++)
+		for(int i=0; i<Math.min(that.sparseBlock.numRows(), rlen); i++)
 		{
-			if(that.sparseBlock[i]!=null)
-			{
-				if(sparseBlock[i]==null)
-					sparseBlock[i]=new SparseRow(that.sparseBlock[i]);
-				else
-					sparseBlock[i].copy(that.sparseBlock[i]);
-			}else if(this.sparseBlock[i]!=null)
-				this.sparseBlock[i].reset(estimatedNNzsPerRow, clen);
+			if(!that.sparseBlock.isEmpty(i)) {
+				sparseBlock.set(i, new SparseRow(that.sparseBlock.get(i)));				
+			}
+			else if(!this.sparseBlock.isEmpty(i)) {
+				this.sparseBlock.reset(i,estimatedNNzsPerRow, clen);
+			}
 		}
 	}
 	
@@ -1412,15 +1384,17 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		allocateDenseBlock(false);
 		
 		int start=0;
-		for(int r=0; r<Math.min(that.sparseBlock.length, rlen); r++, start+=clen)
+		for(int r=0; r<Math.min(that.sparseBlock.numRows(), rlen); r++, start+=clen)
 		{
-			if(that.sparseBlock[r]==null) 
+			if(that.sparseBlock.isEmpty(r)) 
 				continue;
-			double[] values=that.sparseBlock[r].getValueContainer();
-			int[] cols=that.sparseBlock[r].getIndexContainer();
-			for(int i=0; i<that.sparseBlock[r].size(); i++)
-			{
-				denseBlock[start+cols[i]]=values[i];
+			int pos = that.sparseBlock.pos(r);
+			int len = that.sparseBlock.size(r);
+			int[] aix = that.sparseBlock.indexes(r);
+			double[] avals = that.sparseBlock.values(r);
+			
+			for(int i=pos; i<pos+len; i++) {
+				denseBlock[start+aix[i]]=avals[i];
 			}
 		}
 	}
@@ -1437,18 +1411,16 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		allocateSparseRowsBlock(false);
 	
 		for(int i=0, ix=0; i<rlen; i++)
-		{
-			if( sparseBlock[i]!=null ) 
-				sparseBlock[i].reset(estimatedNNzsPerRow, clen);
+		{			
+			sparseBlock.reset(i, estimatedNNzsPerRow, clen);
 			
 			for(int j=0; j<clen; j++)
 			{
 				double val = that.denseBlock[ix++];
-				if( val != 0 )
-				{
-					if(sparseBlock[i]==null) //create sparse row only if required
-						sparseBlock[i]=new SparseRow(estimatedNNzsPerRow, clen);
-					sparseBlock[i].append(j, val);
+				if( val != 0 ) {
+					//create sparse row only if required
+					sparseBlock.allocate(1, estimatedNNzsPerRow, clen);
+					sparseBlock.append(i, j, val);
 				}
 			}
 		}
@@ -1494,65 +1466,56 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 				copyEmptyToSparse(rl, ru, cl, cu, true);
 			return;		
 		}
+		
 		if(sparseBlock==null)
-			sparseBlock=new SparseRow[rlen];
-		else if( awareDestNZ )
-		{
+			allocateSparseRowsBlock(false);
+		else if( awareDestNZ ) {
 			copyEmptyToSparse(rl, ru, cl, cu, true);
 			//explicit clear if awareDestNZ because more efficient since
 			//src will have multiple columns and only few overwriting values
 		}
 		
-		//copy values
-		int alen;
-		int[] aix;
-		double[] avals;
+		SparseBlock a = src.sparseBlock;
+		SparseBlock b = sparseBlock;
 		
+		//copy values
 		for( int i=0; i<src.rlen; i++ )
 		{
-			SparseRow arow = src.sparseBlock[i];
-			if( arow != null && !arow.isEmpty() )
+			if( !a.isEmpty(i) )
 			{
-				alen = arow.size();
-				aix = arow.getIndexContainer();
-				avals = arow.getValueContainer();		
+				int apos = a.pos(i); 
+				int alen = a.size(i);
+				int[] aix = a.indexes(i);
+				double[] avals = a.values(i);		
 				
-				if( sparseBlock[rl+i] == null || sparseBlock[rl+i].isEmpty()  )
+				if( b.isEmpty(rl+i)  )
 				{
-					sparseBlock[rl+i] = new SparseRow(estimatedNNzsPerRow, clen); 
-					SparseRow brow = sparseBlock[rl+i];
-					for( int j=0; j<alen; j++ )
-						brow.append(cl+aix[j], avals[j]);
+					b.allocate(rl+i, estimatedNNzsPerRow, clen);
+					for( int j=apos; j<apos+alen; j++ )
+						b.append(rl+i, cl+aix[j], avals[j]);
 					
 					if( awareDestNZ )
-						nonZeros += brow.size();
+						nonZeros += b.size(rl+i);
 				}
 				else if( awareDestNZ ) //general case (w/ awareness NNZ)
 				{
-					SparseRow brow = sparseBlock[rl+i];
-					int lnnz = brow.size();
-					if( cl==cu && cl==aix[0] ) 
-					{
-						if (avals[0]==0)
-							brow.delete(cl);
-						else
-							brow.set(cl, avals[0] );
+					int lnnz = b.size(rl+i);
+					if( cl==cu && cl==aix[apos] ) {
+						b.set(rl+i, cl, avals[apos] );
 					}
-					else
-					{
-						brow.deleteIndexRange(cl, cu);
-						for( int j=0; j<alen; j++ )
-							brow.set(cl+aix[j], avals[j]);
+					else {
+						//TODO perf sparse row
+						b.deleteIndexRange(rl+i, cl, cu+1);
+						for( int j=apos; j<apos+alen; j++ )
+							b.set(rl+i, cl+aix[j], avals[j]);
 					}
-					nonZeros += (brow.size() - lnnz);
+					nonZeros += (b.size(rl+i) - lnnz);
 				}	
 				else //general case (w/o awareness NNZ)
 				{		
-					SparseRow brow = sparseBlock[rl+i];
-
-					//brow.set(cl, arow);	
-					for( int j=0; j<alen; j++ )
-						brow.set(cl+aix[j], avals[j]);
+					//TODO perf sparse row					
+					for( int j=apos; j<apos+alen; j++ )
+						b.set(rl+i, cl+aix[j], avals[j]);
 				}				
 			}
 		}
@@ -1579,20 +1542,17 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		}
 
 		//copy values
-		int alen;
-		int[] aix;
-		double[] avals;
-		
+		SparseBlock a = src.sparseBlock;
 		for( int i=0, ix=rl*clen; i<src.rlen; i++, ix+=clen )
 		{	
-			SparseRow arow = src.sparseBlock[i];
-			if( arow != null && !arow.isEmpty() )
+			if( !a.isEmpty(i) )
 			{
-				alen = arow.size();
-				aix = arow.getIndexContainer();
-				avals = arow.getValueContainer();
+				int apos = a.pos(i);
+				int alen = a.size(i);
+				int[] aix = a.indexes(i);
+				double[] avals = a.values(i);
 				
-				for( int j=0; j<alen; j++ )
+				for( int j=apos; j<apos+alen; j++ )
 					denseBlock[ix+cl+aix[j]] = avals[j];
 				
 				if(awareDestNZ)
@@ -1610,51 +1570,49 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 				copyEmptyToSparse(rl, ru, cl, cu, true);
 			return;		
 		}
-		if(sparseBlock==null)
-			sparseBlock=new SparseRow[rlen];
+		if(sparseBlock==null) {
+			sparseBlock=SparseBlockFactory
+			.createSparseBlock(DEFAULT_SPARSEBLOCK, rlen);
+		}
 		//no need to clear for awareDestNZ since overwritten  
 		
 		//copy values
-		double val;		
+		SparseBlock a = sparseBlock;
 		for( int i=0, ix=0; i<src.rlen; i++, ix+=src.clen )
 		{
 			int rix = rl + i;
-			if( sparseBlock[rix]==null || sparseBlock[rix].isEmpty() )
+			if( a.isEmpty(rix) )
 			{
-				for( int j=0; j<src.clen; j++ )
-					if( (val = src.denseBlock[ix+j]) != 0 )
-					{
-						if( sparseBlock[rix]==null )
-							sparseBlock[rix] = new SparseRow(estimatedNNzsPerRow, clen); 
-						sparseBlock[rix].append(cl+j, val); 
+				for( int j=0; j<src.clen; j++ ) {
+					double val = src.denseBlock[ix+j];
+					if( val != 0 ) {
+						a.allocate(rix, estimatedNNzsPerRow, clen);
+						sparseBlock.append(rix, cl+j, val); 
 					}
-				
-				if( awareDestNZ && sparseBlock[rix]!=null )
-					nonZeros += sparseBlock[rix].size();
+				}
+			
+				if( awareDestNZ && !a.isEmpty(rix) )
+					nonZeros += a.size(rix);
 			}
 			else if( awareDestNZ ) //general case (w/ awareness NNZ)
 			{
-				SparseRow brow = sparseBlock[rix];
-				int lnnz = brow.size();
-				if( cl==cu ) 
-				{
-					if ((val = src.denseBlock[ix])==0)
-						brow.delete(cl);
-					else
-						brow.set(cl, val);
+				int lnnz = a.size(rix);
+				if( cl==cu ) {
+					double val = src.denseBlock[ix];
+					a.set(rix, cl, val);
 				}
-				else
-				{
-					brow.setIndexRange(cl, cu, src.denseBlock, ix, src.clen);
+				else {
+					a.setIndexRange(rix, cl, cu+1, src.denseBlock, ix, src.clen);
 				}
-				nonZeros += (brow.size() - lnnz);
+				nonZeros += (a.size(rix) - lnnz);
 			}	
 			else //general case (w/o awareness NNZ)
 			{
-				SparseRow brow = sparseBlock[rix];
-				for( int j=0; j<src.clen; j++ )
-					if( (val = src.denseBlock[ix+j]) != 0 ) 
-						brow.set(cl+j, val);
+				for( int j=0; j<src.clen; j++ ) {
+					double val = src.denseBlock[ix+j];
+					if( val != 0 ) 
+						a.set(rix, cl+j, val);
+				}
 			}
 		}
 	}
@@ -1690,43 +1648,26 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	
 	private void copyEmptyToSparse(int rl, int ru, int cl, int cu, boolean updateNNZ ) 
 	{
+		SparseBlock a = sparseBlock;
+		
 		if( cl==cu ) //specific case: column vector
 		{
-			if( updateNNZ )
-			{
-				for( int i=rl; i<=ru; i++ )
-					if( sparseBlock[i] != null && !sparseBlock[i].isEmpty() )
-					{
-						int lnnz = sparseBlock[i].size();
-						sparseBlock[i].delete(cl);
-						nonZeros += (sparseBlock[i].size()-lnnz);
-					}
-			}
-			else
-			{
-				for( int i=rl; i<=ru; i++ )
-					if( sparseBlock[i] != null && !sparseBlock[i].isEmpty() )
-						sparseBlock[i].delete(cl);
-			}
+			for( int i=rl; i<=ru; i++ )
+				if( !a.isEmpty(i) ) {
+					boolean update = a.set(i, cl, 0);
+					if( updateNNZ )
+						nonZeros -= update ? 1 : 0;							
+				}			
 		}
 		else
 		{
-			if( updateNNZ )
-			{
-				for( int i=rl; i<=ru; i++ )
-					if( sparseBlock[i] != null && !sparseBlock[i].isEmpty() )
-					{
-						int lnnz = sparseBlock[i].size();
-						sparseBlock[i].deleteIndexRange(cl, cu);
-						nonZeros += (sparseBlock[i].size()-lnnz);
-					}						
-			}
-			else
-			{
-				for( int i=rl; i<=ru; i++ )
-					if( sparseBlock[i] != null && !sparseBlock[i].isEmpty() )
-						sparseBlock[i].deleteIndexRange(cl, cu);
-			}
+			for( int i=rl; i<=ru; i++ )
+				if( !a.isEmpty(i) ) {
+					int lnnz = a.size(i);
+					a.deleteIndexRange(i, cl, cu+1);
+					if( updateNNZ )
+						nonZeros += (a.size(i)-lnnz);
+				}	
 		}
 	}
 	
@@ -1794,18 +1735,18 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		if( that.sparse ) //DENSE <- SPARSE
 		{
 			double[] a = denseBlock;
-			SparseRow[] b = that.sparseBlock;
+			SparseBlock b = that.sparseBlock;
 			int m = rlen;
 			int n = clen;
 			
 			for( int i=0, aix=0; i<m; i++, aix+=n )
-				if( b[i] != null && !b[i].isEmpty() )
+				if( !b.isEmpty(i) )
 				{
-					SparseRow brow = b[i];
-					int blen = brow.size();
-					int[] bix = brow.getIndexContainer();
-					double[] bval = brow.getValueContainer();
-					for( int j=0; j<blen; j++ )
+					int bpos = b.pos(i);
+					int blen = b.size(i);
+					int[] bix = b.indexes(i);
+					double[] bval = b.values(i);
+					for( int j=bpos; j<bpos+blen; j++ )
 						if( bval[j] != 0 )
 							a[ aix + bix[j] ] = bval[j];
 				}
@@ -1830,42 +1771,41 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	{
 		if( that.sparse ) //SPARSE <- SPARSE
 		{
-			SparseRow[] a = sparseBlock;
-			SparseRow[] b = that.sparseBlock;
+			SparseBlock a = sparseBlock;
+			SparseBlock b = that.sparseBlock;
 			int m = rlen;
 			
 			for( int i=0; i<m; i++ ) 
 			{
-				if( b[i] != null && !b[i].isEmpty() )
+				if( !b.isEmpty(i) )
 				{
-					if( a[i] == null || a[i].isEmpty() ) {
+					if( a.isEmpty(i) ) {
 						//copy entire sparse row (no sort required)
-						a[i] = new SparseRow(b[i]); 
+						a.set(i, new SparseRow(b.get(i))); 
 					}
 					else
 					{
 						boolean appended = false;
-						SparseRow arow = a[i];
-						SparseRow brow = b[i];
-						int blen = brow.size();
-						int[] bix = brow.getIndexContainer();
-						double[] bval = brow.getValueContainer();
-						for( int j=0; j<blen; j++ ) {
+						int bpos = b.pos(i);
+						int blen = b.size(i);
+						int[] bix = b.indexes(i);
+						double[] bval = b.values(i);
+						for( int j=bpos; j<bpos+blen; j++ ) {
 							if( bval[j] != 0 ) {
-								arow.append(bix[j], bval[j]);
+								a.append(i, bix[j], bval[j]);
 								appended = true;
 							}
 						}
 						//only sort if value appended
 						if( !appendOnly && appended )
-							arow.sort();		
+							a.sort(i);		
 					}
 				}
 			}
 		}
 		else //SPARSE <- DENSE
 		{
-			SparseRow[] a = sparseBlock;
+			SparseBlock a = sparseBlock;
 			double[] b = that.denseBlock;
 			int m = rlen;
 			int n = clen;
@@ -1881,7 +1821,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 				}
 				//only sort if value appended
 				if( !appendOnly && appended )
-					a[i].sort();
+					a.sort(i);
 			}
 		}
 	}
@@ -2006,21 +1946,21 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		}
 		else //default deserialize
 		{
+			//TODO perf sparse block
+			
 			for(int r=0; r<rlen; r++)
 			{
 				int nr=in.readInt();
 				if(nr==0)
 				{
-					if(sparseBlock[r]!=null)
-						sparseBlock[r].reset(estimatedNNzsPerRow, clen);
+					if(!sparseBlock.isEmpty(r))
+						sparseBlock.reset(r, estimatedNNzsPerRow, clen);
 					continue;
 				}
-				if(sparseBlock[r]==null)
-					sparseBlock[r]=new SparseRow(nr);
-				else
-					sparseBlock[r].reset(nr, clen);
+				
+				sparseBlock.reset(r, nr, clen);
 				for(int j=0; j<nr; j++)
-					sparseBlock[r].append(in.readInt(), in.readDouble());
+					sparseBlock.append(r, in.readInt(), in.readDouble());
 			}
 		}
 	}
@@ -2066,10 +2006,9 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			for(long i=0; i<nonZeros; i++) {
 				int r = in.readInt();
 				int c = in.readInt();
-				double val = in.readDouble();			
-				if(sparseBlock[r]==null)
-					sparseBlock[r]=new SparseRow(1,clen);
-				sparseBlock[r].append(c, val);
+				double val = in.readDouble();
+				sparseBlock.allocate(r, 1,clen);
+				sparseBlock.append(r, c, val);
 			}
 		}
 		else //ULTRA-SPARSE COL
@@ -2077,10 +2016,9 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			//col: read iv-pairs (should never happen since always dense)
 			for(long i=0; i<nonZeros; i++) {
 				int r = in.readInt();
-				double val = in.readDouble();			
-				if(sparseBlock[r]==null)
-					sparseBlock[r]=new SparseRow(1,1);
-				sparseBlock[r].append(0, val);
+				double val = in.readDouble();		
+				sparseBlock.allocate(r, 1, 1);
+				sparseBlock.append(r, 0, val);
 			}
 		}	
 	}
@@ -2202,22 +2140,22 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		else //general case (if fast serialize not supported)
 		{
 			int r=0;
-			for(;r<Math.min(rlen, sparseBlock.length); r++)
+			for(;r<Math.min(rlen, sparseBlock.numRows()); r++)
 			{
-				if(sparseBlock[r]==null)
+				if( sparseBlock.isEmpty(r) )
 					out.writeInt(0);
 				else
 				{
-					int nr=sparseBlock[r].size();
+					int pos = sparseBlock.pos(r);
+					int nr = sparseBlock.size(r);
+					int[] cols = sparseBlock.indexes(r);
+					double[] values=sparseBlock.values(r);
+					
 					out.writeInt(nr);
-					int[] cols=sparseBlock[r].getIndexContainer();
-					double[] values=sparseBlock[r].getValueContainer();
-					for(int j=0; j<nr; j++)
-					{
+					for(int j=pos; j<pos+nr; j++) {
 						out.writeInt(cols[j]);
 						out.writeDouble(values[j]);
-					}
-					
+					}					
 				}	
 			}
 			for(;r<rlen; r++)
@@ -2240,13 +2178,15 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		if( clen > 1 ) //ULTRA-SPARSE BLOCK
 		{
 			//block: write ijv-triples
-			for(int r=0;r<Math.min(rlen, sparseBlock.length); r++)
-				if(sparseBlock[r]!=null && !sparseBlock[r].isEmpty() )
+			for(int r=0;r<Math.min(rlen, sparseBlock.numRows()); r++)
+				if( !sparseBlock.isEmpty(r) )
 				{
-					int alen = sparseBlock[r].size();
-					int[] aix = sparseBlock[r].getIndexContainer();
-					double[] avals = sparseBlock[r].getValueContainer();
-					for(int j=0; j<alen; j++) {
+					int apos = sparseBlock.pos(r);
+					int alen = sparseBlock.size(r);
+					int[] aix = sparseBlock.indexes(r);
+					double[] avals = sparseBlock.values(r);
+					
+					for(int j=apos; j<apos+alen; j++) {
 						//ultra-sparse block: write ijv-triples
 						out.writeInt(r);
 						out.writeInt(aix[j]);
@@ -2258,10 +2198,11 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		else //ULTRA-SPARSE COL
 		{
 			//block: write iv-pairs (should never happen since always dense)
-			for(int r=0;r<Math.min(rlen, sparseBlock.length); r++)
-				if(sparseBlock[r]!=null && !sparseBlock[r].isEmpty() ) {
+			for(int r=0;r<Math.min(rlen, sparseBlock.numRows()); r++)
+				if(!sparseBlock.isEmpty(r) ) {
+					int pos = sparseBlock.pos(r);
 					out.writeInt(r);
-					out.writeDouble(sparseBlock[r].getValueContainer()[0]);
+					out.writeDouble(sparseBlock.values(r)[pos]);
 					wnnz++;
 				}
 		}
@@ -2289,22 +2230,23 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 				out.writeDouble(0);
 		else //existing sparse block
 		{
+			SparseBlock a = sparseBlock;
 			for( int i=0; i<rlen; i++ )
 			{
-				if( i<sparseBlock.length && sparseBlock[i]!=null && !sparseBlock[i].isEmpty() )
+				if( i<a.numRows() && !a.isEmpty(i) )
 				{
-					SparseRow arow = sparseBlock[i];
-					int alen = arow.size();
-					int[] aix = arow.getIndexContainer();
-					double[] avals = arow.getValueContainer();
+					int apos = a.pos(i);
+					int alen = a.size(i);
+					int[] aix = a.indexes(i);
+					double[] avals = a.values(i);
 					//foreach non-zero value, fill with 0s if required
 					for( int j=0, j2=0; j2<alen; j++, j2++ ) {
-						for( ; j<aix[j2]; j++ )
+						for( ; j<aix[apos+j2]; j++ )
 							out.writeDouble( 0 );
-						out.writeDouble( avals[j2] );
+						out.writeDouble( avals[apos+j2] );
 					}					
 					//remaining 0 values in row
-					for( int j=aix[alen-1]+1; j<clen; j++)
+					for( int j=aix[apos+alen-1]+1; j<clen; j++)
 						out.writeDouble( 0 );
 				}
 				else //empty row
@@ -2898,16 +2840,17 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		
 		if( sparse ) //SPARSE <- SPARSE
 		{
-			SparseRow[] a = sparseBlock;
+			SparseBlock a = sparseBlock;
 			
 			for(int i=0; i<m; i++) {
-				if( a[i]!=null && !a[i].isEmpty() )
+				if( !a.isEmpty(i) )
 				{
-					int alen = a[i].size();
-					int[] aix = a[i].getIndexContainer();
-					double[] avals = a[i].getValueContainer();
+					int apos = a.pos(i);
+					int alen = a.size(i);
+					int[] aix = a.indexes(i);
+					double[] avals = a.values(i);
 					
-					for( int j=0; j<alen; j++ ) {
+					for( int j=apos; j<apos+alen; j++ ) {
 						double val = op.fn.execute(avals[j]);
 						ret.appendValue(i, aix[j], val);
 					}
@@ -2993,25 +2936,29 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		if(sparse)
 		{
 			nonZeros=0;
-			for(int r=0; r<Math.min(rlen, sparseBlock.length); r++)
+			for(int r=0; r<Math.min(rlen, sparseBlock.numRows()); r++)
 			{
-				if(sparseBlock[r]==null) 
+				if(sparseBlock.isEmpty(r)) 
 					continue;
-				double[] values=sparseBlock[r].getValueContainer();
-				int[] cols=sparseBlock[r].getIndexContainer();
+				
+				int apos = sparseBlock.pos(r);
+				int alen = sparseBlock.size(r);
+				int[] aix = sparseBlock.indexes(r);
+				double[] avals = sparseBlock.values(r);
+				
 				int pos=0;
-				for(int i=0; i<sparseBlock[r].size(); i++)
+				for(int i=apos; i<apos+alen; i++)
 				{
-					double v=op.fn.execute(values[i]);
-					if(v!=0)
-					{
-						values[pos]=v;
-						cols[pos]=cols[i];
+					double v=op.fn.execute(avals[i]);
+					if(v!=0) {
+						avals[pos]=v;
+						aix[pos]=aix[i];
 						pos++;
 						nonZeros++;
 					}
 				}
-				sparseBlock[r].truncate(pos);
+				//TODO perf sparse block: truncate replaced by deleteIndexrange
+				sparseBlock.deleteIndexRange(r, pos, clen);
 			}
 			
 		}
@@ -3209,18 +3156,18 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			{
 				if( newWithCor.isInSparseFormat() && aggOp.sparseSafe ) //SPARSE
 				{
-					SparseRow[] bRows = newWithCor.getSparseBlock();
-					if( bRows==null ) //early abort on empty block
+					SparseBlock b = newWithCor.getSparseBlock();
+					if( b==null ) //early abort on empty block
 						return;
-					for( int r=0; r<Math.min(rlen, bRows.length); r++ )
+					for( int r=0; r<Math.min(rlen, b.numRows()); r++ )
 					{
-						SparseRow brow = bRows[r];
-						if( brow != null && !brow.isEmpty() ) 
+						if( !b.isEmpty(r) ) 
 						{
-							int blen = brow.size();
-							int[] bix = brow.getIndexContainer();
-							double[] bvals = brow.getValueContainer();
-							for( int j=0; j<blen; j++)
+							int bpos = b.pos(r);
+							int blen = b.size(r);
+							int[] bix = b.indexes(r);
+							double[] bvals = b.values(r);
+							for( int j=bpos; j<bpos+blen; j++)
 							{
 								int c = bix[j];
 								buffer._sum = this.quickGetValue(r, c);
@@ -3607,19 +3554,22 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			CellIndex temp = new CellIndex(0, 0);
 			if(sparse)
 			{
-				if(sparseBlock!=null)
+				if(sparseBlock != null)
 				{
-					for(int r=0; r<Math.min(rlen, sparseBlock.length); r++)
+					for(int r=0; r<Math.min(rlen, sparseBlock.numRows()); r++)
 					{
-						if(sparseBlock[r]==null) 
+						if(sparseBlock.isEmpty(r)) 
 							continue;
-						int[] cols=sparseBlock[r].getIndexContainer();
-						double[] values=sparseBlock[r].getValueContainer();
-						for(int i=0; i<sparseBlock[r].size(); i++)
-						{
-							tempCellIndex.set(r, cols[i]);
+						
+						int apos = sparseBlock.pos(r);
+						int alen = sparseBlock.size(r);
+						int[] aix = sparseBlock.indexes(r);
+						double[] avals = sparseBlock.values(r);
+						
+						for(int i=apos; i<apos+alen; i++) {
+							tempCellIndex.set(r, aix[i]);
 							op.fn.execute(tempCellIndex, temp);
-							result.appendValue(temp.row, temp.column, values[i]);
+							result.appendValue(temp.row, temp.column, avals[i]);
 						}
 					}
 				}
@@ -4061,9 +4011,8 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			//note: always dense dest
 			dest.allocateDenseBlock();
 			for( int i=rl; i<=ru; i++ ) {
-				SparseRow arow = sparseBlock[i];
-				if( arow != null && !arow.isEmpty() ) {
-					double val = arow.get(cl);
+				if( !sparseBlock.isEmpty(i) ) {
+					double val = sparseBlock.get(i, cl);
 					if( val != 0 ) {
 						dest.denseBlock[i-rl] = val;
 						dest.nonZeros++;
@@ -4074,20 +4023,20 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		else if( rl==ru && cl==0 && cu==clen-1 ) //ROW VECTOR 
 		{
 			//note: always sparse dest, but also works for dense
-			dest.appendRow(0, sparseBlock[rl]);
+			dest.appendRow(0, sparseBlock.get(rl));
 		}
 		else //general case (sparse/dense dest)
 		{
 			for(int i=rl; i <= ru; i++) 
-				if(sparseBlock[i] != null && !sparseBlock[i].isEmpty()) 
+				if( !sparseBlock.isEmpty(i)) 
 				{
-					SparseRow arow = sparseBlock[i];
-					int alen = arow.size();
-					int[] aix = arow.getIndexContainer();
-					double[] avals = arow.getValueContainer();
-					int astart = (cl>0)?arow.searchIndexesFirstGTE(cl):0;
+					int apos = sparseBlock.pos(i);
+					int alen = sparseBlock.size(i);
+					int[] aix = sparseBlock.indexes(i);
+					double[] avals = sparseBlock.values(i);
+					int astart = (cl>0)?sparseBlock.posFIndexGTE(i, cl) : apos;
 					if( astart != -1 )
-						for( int j=astart; j<alen && aix[j] <= cu; j++ )
+						for( int j=astart; j<apos+alen && aix[j] <= cu; j++ )
 							dest.appendValue(i-rl, aix[j]-cl, avals[j]);	
 				}
 		}
@@ -4191,10 +4140,10 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			if(sparseBlock!=null)
 			{
 				int r=(int)range.rowStart;
-				for(; r<Math.min(Math.min(rowCut, sparseBlock.length), range.rowEnd+1); r++)
+				for(; r<Math.min(Math.min(rowCut, sparseBlock.numRows()), range.rowEnd+1); r++)
 					sliceHelp(r, range, colCut, topleft, topright, normalBlockRowFactor-rowCut, normalBlockRowFactor, normalBlockColFactor);
 				
-				for(; r<=Math.min(range.rowEnd, sparseBlock.length-1); r++)
+				for(; r<=Math.min(range.rowEnd, sparseBlock.numRows()-1); r++)
 					sliceHelp(r, range, colCut, bottomleft, bottomright, -rowCut, normalBlockRowFactor, normalBlockColFactor);
 				//System.out.println("in: \n"+this);
 				//System.out.println("outlist: \n"+outlist);
@@ -4230,15 +4179,15 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 	
 	private void sliceHelp(int r, IndexRange range, int colCut, MatrixBlock left, MatrixBlock right, int rowOffset, int normalBlockRowFactor, int normalBlockColFactor)
 	{
-		if(sparseBlock[r]==null) 
+		if(sparseBlock.isEmpty(r)) 
 			return;
 		
-		int[] cols=sparseBlock[r].getIndexContainer();
-		double[] values=sparseBlock[r].getValueContainer();
-		int start=sparseBlock[r].searchIndexesFirstGTE((int)range.colStart);
+		int[] cols=sparseBlock.indexes(r);
+		double[] values=sparseBlock.values(r);
+		int start=sparseBlock.posFIndexGTE(r, (int)range.colStart);
 		if(start<0) 
 			return;
-		int end=sparseBlock[r].searchIndexesFirstLTE((int)range.colEnd);
+		int end=sparseBlock.posFIndexLTE(r, (int)range.colEnd);
 		if(end<0 || start>end) 
 			return;
 		
@@ -4324,23 +4273,23 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			{
 				if(!complementary)//if zero out
 				{
-					for(int r=0; r<Math.min((int)range.rowStart, sparseBlock.length); r++)
-						((MatrixBlock) result).appendRow(r, sparseBlock[r]);
-					for(int r=Math.min((int)range.rowEnd+1, sparseBlock.length); r<Math.min(rlen, sparseBlock.length); r++)
-						((MatrixBlock) result).appendRow(r, sparseBlock[r]);
+					for(int r=0; r<Math.min((int)range.rowStart, sparseBlock.numRows()); r++)
+						((MatrixBlock) result).appendRow(r, sparseBlock.get(r));
+					for(int r=Math.min((int)range.rowEnd+1, sparseBlock.numRows()); r<Math.min(rlen, sparseBlock.numRows()); r++)
+						((MatrixBlock) result).appendRow(r, sparseBlock.get(r));
 				}
-				for(int r=(int)range.rowStart; r<=Math.min(range.rowEnd, sparseBlock.length-1); r++)
+				for(int r=(int)range.rowStart; r<=Math.min(range.rowEnd, sparseBlock.numRows()-1); r++)
 				{
-					if(sparseBlock[r]==null) 
+					if(sparseBlock.isEmpty(r)) 
 						continue;
-					int[] cols=sparseBlock[r].getIndexContainer();
-					double[] values=sparseBlock[r].getValueContainer();
+					int[] cols=sparseBlock.indexes(r);
+					double[] values=sparseBlock.values(r);
 					
 					if(complementary)//if selection
 					{
-						int start=sparseBlock[r].searchIndexesFirstGTE((int)range.colStart);
+						int start=sparseBlock.posFIndexGTE(r,(int)range.colStart);
 						if(start<0) continue;
-						int end=sparseBlock[r].searchIndexesFirstGT((int)range.colEnd);
+						int end=sparseBlock.posFIndexGT(r,(int)range.colEnd);
 						if(end<0 || start>end) 
 							continue;
 						
@@ -4350,16 +4299,18 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 						}
 					}else
 					{
-						int start=sparseBlock[r].searchIndexesFirstGTE((int)range.colStart);
-						if(start<0) start=sparseBlock[r].size();
-						int end=sparseBlock[r].searchIndexesFirstGT((int)range.colEnd);
-						if(end<0) end=sparseBlock[r].size();
+						int pos = sparseBlock.pos(r);
+						int len = sparseBlock.size(r);
+						int start=sparseBlock.posFIndexGTE(r,(int)range.colStart);
+						if(start<0) start=pos+len;
+						int end=sparseBlock.posFIndexGT(r,(int)range.colEnd);
+						if(end<0) end=pos+len;
 						
-						for(int i=0; i<start; i++)
+						for(int i=pos; i<start; i++)
 						{
 							((MatrixBlock) result).appendValue(r, cols[i], values[i]);
 						}
-						for(int i=end; i<sparseBlock[r].size(); i++)
+						for(int i=end; i<pos+len; i++)
 						{
 							((MatrixBlock) result).appendValue(r, cols[i], values[i]);
 						}
@@ -4486,18 +4437,22 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		{
 			if(sparseBlock!=null)
 			{
-				for(r=0; r<Math.min(rlen, sparseBlock.length); r++)
+				SparseBlock a = sparseBlock;
+				
+				for(r=0; r<Math.min(rlen, a.numRows()); r++)
 				{
-					if(sparseBlock[r]==null) 
+					if(a.isEmpty(r)) 
 						continue;
-					int[] cols=sparseBlock[r].getIndexContainer();
-					double[] values=sparseBlock[r].getValueContainer();
-					for(int i=0; i<sparseBlock[r].size(); i++)
-					{
-						tempCellIndex.set(r, cols[i]);
+					int apos = a.pos(r);
+					int alen = a.size(r);
+					int[] aix = a.indexes(r);
+					double[] aval = a.values(r);
+					
+					for(int i=apos; i<apos+alen; i++) {
+						tempCellIndex.set(r, aix[i]);
 						op.indexFn.execute(tempCellIndex, tempCellIndex);
-						incrementalAggregateUnaryHelp(op.aggOp, result, tempCellIndex.row, tempCellIndex.column, values[i], buffer);
-
+						incrementalAggregateUnaryHelp(op.aggOp, result, 
+								tempCellIndex.row, tempCellIndex.column, aval[i], buffer);
 					}
 				}
 			}
@@ -4654,8 +4609,8 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			{
 				if(sparseBlock!=null)
 					for(int i=1; i<=step; i++)
-						if(sparseBlock[rlen-i]!=null)
-							this.nonZeros-=sparseBlock[rlen-i].size();
+						if(!sparseBlock.isEmpty(rlen-i))
+							this.nonZeros-=sparseBlock.size(rlen-i);
 			}
 			else //DENSE
 			{
@@ -4678,14 +4633,17 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			{
 				if(sparseBlock!=null)
 				{
-					for(int r=0; r<Math.min(rlen, sparseBlock.length); r++)
-						if(sparseBlock[r]!=null)
+					for(int r=0; r<Math.min(rlen, sparseBlock.numRows()); r++)
+						if(!sparseBlock.isEmpty(r))
 						{
-							int newSize=sparseBlock[r].searchIndexesFirstGTE(clen-step);
-							if(newSize>=0)
+							int newSize=sparseBlock.posFIndexGTE(r, clen-step);
+							if(newSize >= 0)
 							{
-								this.nonZeros-=sparseBlock[r].size()-newSize;
-								sparseBlock[r].truncate(newSize);
+								this.nonZeros-=sparseBlock.size(r)-newSize;
+								int pos = sparseBlock.pos(r);
+								int cl = sparseBlock.indexes(r)[pos+newSize-1];
+								sparseBlock.deleteIndexRange(r, cl+1, clen);
+								//TODO perf sparse block: truncate replaced by deleteIndexRange
 							}
 						}
 				}
@@ -4747,13 +4705,15 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		int nzcount = 0;
 		if(sparse && sparseBlock!=null) //SPARSE
 		{
-			for(int r=0; r<Math.min(rlen, sparseBlock.length); r++)
+			for(int r=0; r<Math.min(rlen, sparseBlock.numRows()); r++)
 			{
-				if(sparseBlock[r]==null) 
+				if(sparseBlock.isEmpty(r)) 
 					continue;
-				double[] values=sparseBlock[r].getValueContainer();
-				for(int i=0; i<sparseBlock[r].size(); i++) {
-					op.fn.execute(cmobj, values[i]);
+				int apos = sparseBlock.pos(r);
+				int alen = sparseBlock.size(r);
+				double[] avals = sparseBlock.values(r);
+				for(int i=apos; i<apos+alen; i++) {
+					op.fn.execute(cmobj, avals[i]);
 					nzcount++;
 				}
 			}
@@ -5426,27 +5386,25 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			if( pattern != 0d ) //SPARSE <- SPARSE (sparse-safe)
 			{
 				ret.allocateSparseRowsBlock();
-				SparseRow[] a = sparseBlock;
-				SparseRow[] c = ret.sparseBlock;
+				SparseBlock a = sparseBlock;
+				SparseBlock c = ret.sparseBlock;
 				
-				for( int i=0; i<rlen; i++ )
-				{
-					SparseRow arow = a[ i ];
-					if( arow!=null && !arow.isEmpty() )
+				for( int i=0; i<rlen; i++ ) {
+					if( !a.isEmpty(i) )
 					{
-						SparseRow crow = new SparseRow(arow.size());
-						int alen = arow.size();
-						int[] aix = arow.getIndexContainer();
-						double[] avals = arow.getValueContainer();
-						for( int j=0; j<alen; j++ )
+						c.allocate(i);
+						int apos = a.pos(i);
+						int alen = a.size(i);
+						int[] aix = a.indexes(i);
+						double[] avals = a.values(i);
+						for( int j=apos; j<apos+alen; j++ )
 						{
 							double val = avals[j];
 							if( val== pattern || (NaNpattern && Double.isNaN(val)) )
-								crow.append(aix[j], replacement);
+								c.append(i, aix[j], replacement);
 							else
-								crow.append(aix[j], val);
+								c.append(i, aix[j], val);
 						}
-						c[ i ] = crow;
 					}
 				}
 			}
@@ -5454,7 +5412,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			{
 				ret.sparse = false;
 				ret.allocateDenseBlock();	
-				SparseRow[] a = sparseBlock;
+				SparseBlock a = sparseBlock;
 				double[] c = ret.denseBlock;
 				
 				//initialize with replacement (since all 0 values, see SPARSITY_TURN_POINT)
@@ -5464,13 +5422,13 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 				if( a != null  ) //check for empty matrix
 					for( int i=0, cix=0; i<rlen; i++, cix+=clen )
 					{
-						SparseRow arow = a[ i ];
-						if( arow!=null && !arow.isEmpty() )
+						if( !a.isEmpty(i) )
 						{
-							int alen = arow.size();
-							int[] aix = arow.getIndexContainer();
-							double[] avals = arow.getValueContainer();
-							for( int j=0; j<alen; j++ )
+							int apos = a.pos(i);
+							int alen = a.size(i);
+							int[] aix = a.indexes(i);
+							double[] avals = a.values(i);
+							for( int j=apos; j<apos+alen; j++ )
 								if( avals[ j ] != 0 )
 									c[ cix+aix[j] ] = avals[ j ];
 						}
@@ -5654,25 +5612,25 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			if( this.isEmptyBlock(false) && that.isEmptyBlock(false) )
 				return;
 			
-			SparseRow[] a = this.sparseBlock;
-			SparseRow[] b = that.sparseBlock;
+			SparseBlock a = this.sparseBlock;
+			SparseBlock b = that.sparseBlock;
 			for( int i=0; i<rlen; i++ )
 			{
-				SparseRow arow = a[i];
-				SparseRow brow = b[i];
-				if( arow != null && !arow.isEmpty() )
+				if( !a.isEmpty(i) )
 				{
-					int alen = arow.size();
-					double[] avals = arow.getValueContainer();
-					double[] bvals = brow.getValueContainer();
+					int alen = a.size(i);
+					int apos = a.pos(i);
+					double[] avals = a.values(i);
+					int bpos = b.pos(i);
+					double[] bvals = b.values(i); 
 					
 					if( resultBlock == null ) {
 						for( int j=0; j<alen; j++ )
-							ctable.execute(avals[j], bvals[j], w, ignoreZeros, resultMap);		
+							ctable.execute(avals[apos+j], bvals[bpos+j], w, ignoreZeros, resultMap);		
 					}
 					else {
 						for( int j=0; j<alen; j++ )
-							ctable.execute(avals[j], bvals[j], w, ignoreZeros, resultBlock);			
+							ctable.execute(avals[apos+j], bvals[bpos+j], w, ignoreZeros, resultBlock);			
 					}
 				}
 			}	
@@ -6142,29 +6100,14 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		
 		if(sparse)
 		{
-			int len=0;
-			if(sparseBlock!=null)
-				len = Math.min(rlen, sparseBlock.length);
-			int i=0;
-			for(; i<len; i++)
-			{
-				sb.append("row +");
-				sb.append(i);
-				sb.append(": ");
-				sb.append(sparseBlock[i]);
-				sb.append("\n");
-			}
-			for(; i<rlen; i++)
-			{
-				sb.append("row +");
-				sb.append(i);
-				sb.append(": null\n");
+			if( sparseBlock != null ) {
+				//overloaded implementation in sparse blocks
+				sb.append(sparseBlock.toString());
 			}
 		}
 		else
 		{
-			if(denseBlock!=null)
-			{
+			if(denseBlock!=null) {
 				for(int i=0, ix=0; i<rlen; i++, ix+=clen) {
 					for(int j=0; j<clen; j++) {
 						sb.append(this.denseBlock[ix+j]);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataInput.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataInput.java b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataInput.java
index e78b766..84cfa0c 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataInput.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataInput.java
@@ -54,6 +54,6 @@ public interface MatrixBlockDataInput
 	 * @param rows
 	 * @throws IOException
 	 */
-	public long readSparseRows(int rlen, SparseRow[] rows) 
+	public long readSparseRows(int rlen, SparseBlock rows) 
 		throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataOutput.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataOutput.java b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataOutput.java
index b89e0dd..fd8fca9 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataOutput.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlockDataOutput.java
@@ -52,6 +52,6 @@ public interface MatrixBlockDataOutput
 	 * @param rows
 	 * @throws IOException
 	 */
-	public void writeSparseRows(int rlen, SparseRow[] rows) 
+	public void writeSparseRows(int rlen, SparseBlock rows) 
 		throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/main/java/org/apache/sysml/runtime/matrix/data/SparseRowsIterator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/SparseRowsIterator.java b/src/main/java/org/apache/sysml/runtime/matrix/data/SparseRowsIterator.java
deleted file mode 100644
index 75ca433..0000000
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/SparseRowsIterator.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.sysml.runtime.matrix.data;
-
-import java.util.Iterator;
-
-/**
- * Iterator for external use of matrix blocks in sparse representation.
- * It allows to linearly iterate only over non-zero values which is
- * important for sparse safe operations.
- * 
- */
-public class SparseRowsIterator implements Iterator<IJV>
-{
-	
-	private int rlen = 0;
-	private SparseRow[] sparseRows = null;
-	private int curRow = -1;
-	private int curColIndex = -1;
-	private int[] colIndexes = null;
-	private double[] values = null;
-	private boolean nothingLeft = false;
-	private IJV retijv = new IJV();
-
-	//allow initialization from package or subclasses
-	protected SparseRowsIterator(int nrows, SparseRow[] mtx)
-	{
-		rlen=nrows;
-		sparseRows=mtx;
-		curRow=0;
-		
-		if(sparseRows==null)
-			nothingLeft=true;
-		else
-			findNextNonZeroRow();
-	}
-	
-	//allow initialization from package or subclasses
-	protected SparseRowsIterator(int currow, int nrows, SparseRow[] mtx)
-	{
-		rlen=nrows;
-		sparseRows=mtx;
-		curRow=currow;
-		
-		if(sparseRows==null)
-			nothingLeft=true;
-		else
-			findNextNonZeroRow();
-	}
-	
-	@Override
-	public boolean hasNext() {
-		if(nothingLeft)
-			return false;
-		else
-			return true;
-	}
-
-	@Override
-	public IJV next( ) {
-		retijv.set(curRow, colIndexes[curColIndex], values[curColIndex]);
-		curColIndex++;
-		if(curColIndex>=sparseRows[curRow].size())
-		{
-			curRow++;
-			findNextNonZeroRow();
-		}
-		return retijv;
-	}
-
-	@Override
-	public void remove() {
-		throw new RuntimeException("SparseCellIterator.remove should not be called!");
-		
-	}		
-	
-	/**
-	 * 
-	 */
-	private void findNextNonZeroRow() 
-	{
-		while(curRow<Math.min(rlen, sparseRows.length) && (sparseRows[curRow]==null || sparseRows[curRow].isEmpty()))
-			curRow++;
-		if(curRow>=Math.min(rlen, sparseRows.length))
-			nothingLeft=true;
-		else
-		{
-			curColIndex=0;
-			colIndexes=sparseRows[curRow].getIndexContainer();
-			values=sparseRows[curRow].getValueContainer();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataInputStream.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataInputStream.java b/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataInputStream.java
index 67f8194..74641bd 100644
--- a/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataInputStream.java
+++ b/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataInputStream.java
@@ -26,7 +26,7 @@ import java.io.IOException;
 import java.io.InputStream;
 
 import org.apache.sysml.runtime.matrix.data.MatrixBlockDataInput;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
+import org.apache.sysml.runtime.matrix.data.SparseBlock;
 
 /**
  * 
@@ -202,7 +202,7 @@ public class FastBufferedDataInputStream extends FilterInputStream implements Da
 	}
 
 	@Override
-	public long readSparseRows(int rlen, SparseRow[] rows) 
+	public long readSparseRows(int rlen, SparseBlock rows) 
 		throws IOException 
 	{
 		//counter for non-zero elements
@@ -216,9 +216,7 @@ public class FastBufferedDataInputStream extends FilterInputStream implements Da
 			if( lnnz > 0 ) //non-zero row
 			{
 				//get handle to sparse (allocate if necessary)
-				if( rows[i] == null )
-					rows[i] = new SparseRow(lnnz);
-				SparseRow arow = rows[i];
+				rows.allocate(i, lnnz);
 				
 				//read single sparse row
 				//note: cast to long to prevent overflows w/ lnnz*12
@@ -232,7 +230,7 @@ public class FastBufferedDataInputStream extends FilterInputStream implements Da
 						int aix = baToInt(_buff, j);
 						long tmp = baToLong(_buff, j+4);
 						double aval = Double.longBitsToDouble( tmp );
-						arow.append(aix, aval);
+						rows.append(i, aix, aval);
 					}
 				}
 				else
@@ -243,7 +241,7 @@ public class FastBufferedDataInputStream extends FilterInputStream implements Da
 						int aix = baToInt(_buff, 0);
 						long tmp = baToLong(_buff, 4);
 						double aval = Double.longBitsToDouble(tmp);
-						arow.append(aix, aval);
+						rows.append(i, aix, aval);
 					}
 				}
 				

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataOutputStream.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataOutputStream.java b/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataOutputStream.java
index f65c588..e7aebb4 100644
--- a/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataOutputStream.java
+++ b/src/main/java/org/apache/sysml/runtime/util/FastBufferedDataOutputStream.java
@@ -25,7 +25,7 @@ import java.io.IOException;
 import java.io.OutputStream;
 
 import org.apache.sysml.runtime.matrix.data.MatrixBlockDataOutput;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
+import org.apache.sysml.runtime.matrix.data.SparseBlock;
 
 /**
  * This buffered output stream is essentially a merged version of
@@ -236,21 +236,21 @@ public class FastBufferedDataOutputStream extends FilterOutputStream implements
 	}
 
 	@Override
-	public void writeSparseRows(int rlen, SparseRow[] rows) 
+	public void writeSparseRows(int rlen, SparseBlock rows) 
 		throws IOException
 	{
-		int lrlen = Math.min(rows.length, rlen);
+		int lrlen = Math.min(rows.numRows(), rlen);
 		
 		//process existing rows
 		for( int i=0; i<lrlen; i++ )
 		{
-			SparseRow arow = rows[i];
-			if( arow!=null && !arow.isEmpty() )
+			if( !rows.isEmpty(i) )
 			{
-				int alen = arow.size();
+				int apos = rows.pos(i);
+				int alen = rows.size(i);
 				int alen2 = alen*12;
-				int[] aix = arow.getIndexContainer();
-				double[] avals = arow.getValueContainer();
+				int[] aix = rows.indexes(i);
+				double[] avals = rows.values(i);
 				
 				writeInt( alen );
 				
@@ -259,7 +259,7 @@ public class FastBufferedDataOutputStream extends FilterOutputStream implements
 					if (_count+alen2 > _bufflen) 
 					    flushBuffer();
 					
-					for( int j=0; j<alen; j++ )
+					for( int j=apos; j<apos+alen; j++ )
 					{
 						long tmp2 = Double.doubleToRawLongBits(avals[j]);
 						intToBa(aix[j], _buff, _count);
@@ -270,7 +270,7 @@ public class FastBufferedDataOutputStream extends FilterOutputStream implements
 				else
 				{
 					//row does not fit in buffer
-					for( int j=0; j<alen; j++ )
+					for( int j=apos; j<apos+alen; j++ )
 					{
 						if (_count+12 > _bufflen) 
 						    flushBuffer();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockDelete.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockDelete.java b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockDelete.java
index c719bcb..392194b 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockDelete.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockDelete.java
@@ -29,7 +29,6 @@ import org.apache.sysml.runtime.matrix.data.SparseBlock;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCOO;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCSR;
 import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
 import org.apache.sysml.runtime.util.DataConverter;
 import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.utils.TestUtils;
@@ -115,11 +114,11 @@ public class SparseBlockDelete extends AutomatedTestBase
 			//init sparse block
 			SparseBlock sblock = null;
 			MatrixBlock mbtmp = DataConverter.convertToMatrixBlock(A);
-			SparseRow[] srtmp = mbtmp.getSparseBlock();			
+			SparseBlock srtmp = mbtmp.getSparseBlock();			
 			switch( btype ) {
-				case MCSR: sblock = new SparseBlockMCSR(srtmp,true); break;
-				case CSR: sblock = new SparseBlockCSR(srtmp, (int)mbtmp.getNonZeros()); break;
-				case COO: sblock = new SparseBlockCOO(srtmp, (int)mbtmp.getNonZeros()); break;
+				case MCSR: sblock = new SparseBlockMCSR(srtmp); break;
+				case CSR: sblock = new SparseBlockCSR(srtmp); break;
+				case COO: sblock = new SparseBlockCOO(srtmp); break;
 			}
 			
 			//delete range per row via set

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetFirstIndex.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetFirstIndex.java b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetFirstIndex.java
index 7e23dd0..e2800eb 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetFirstIndex.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetFirstIndex.java
@@ -26,7 +26,6 @@ import org.apache.sysml.runtime.matrix.data.SparseBlock;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCOO;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCSR;
 import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
 import org.apache.sysml.runtime.util.DataConverter;
 import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.utils.TestUtils;
@@ -207,11 +206,11 @@ public class SparseBlockGetFirstIndex extends AutomatedTestBase
 			//init sparse block
 			SparseBlock sblock = null;
 			MatrixBlock mbtmp = DataConverter.convertToMatrixBlock(A);
-			SparseRow[] srtmp = mbtmp.getSparseBlock();			
+			SparseBlock srtmp = mbtmp.getSparseBlock();			
 			switch( btype ) {
-				case MCSR: sblock = new SparseBlockMCSR(srtmp,true); break;
-				case CSR: sblock = new SparseBlockCSR(srtmp, (int)mbtmp.getNonZeros()); break;
-				case COO: sblock = new SparseBlockCOO(srtmp, (int)mbtmp.getNonZeros()); break;
+				case MCSR: sblock = new SparseBlockMCSR(srtmp); break;
+				case CSR: sblock = new SparseBlockCSR(srtmp); break;
+				case COO: sblock = new SparseBlockCOO(srtmp); break;
 			}
 			
 			//check for correct number of non-zeros

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetSet.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetSet.java b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetSet.java
index 9c9856e..be160c9 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetSet.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockGetSet.java
@@ -26,7 +26,6 @@ import org.apache.sysml.runtime.matrix.data.SparseBlock;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCOO;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCSR;
 import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
 import org.apache.sysml.runtime.util.DataConverter;
 import org.apache.sysml.runtime.util.LongLongDoubleHashMap;
 import org.apache.sysml.runtime.util.LongLongDoubleHashMap.LLDoubleEntry;
@@ -210,11 +209,11 @@ public class SparseBlockGetSet extends AutomatedTestBase
 			SparseBlock sblock = null;
 			if( itype == InitType.BULK ) {
 				MatrixBlock mbtmp = DataConverter.convertToMatrixBlock(A);
-				SparseRow[] srtmp = mbtmp.getSparseBlock();			
+				SparseBlock srtmp = mbtmp.getSparseBlock();			
 				switch( btype ) {
-					case MCSR: sblock = new SparseBlockMCSR(srtmp,true); break;
-					case CSR: sblock = new SparseBlockCSR(srtmp, (int)mbtmp.getNonZeros()); break;
-					case COO: sblock = new SparseBlockCOO(srtmp, (int)mbtmp.getNonZeros()); break;
+					case MCSR: sblock = new SparseBlockMCSR(srtmp); break;
+					case CSR: sblock = new SparseBlockCSR(srtmp); break;
+					case COO: sblock = new SparseBlockCOO(srtmp); break;
 				}
 			}
 			else if( itype == InitType.SEQ_SET || itype == InitType.RAND_SET ) {

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIndexRange.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIndexRange.java b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIndexRange.java
index 5a4ea4a..a483ab3 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIndexRange.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIndexRange.java
@@ -30,7 +30,6 @@ import org.apache.sysml.runtime.matrix.data.SparseBlock;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCOO;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCSR;
 import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
 import org.apache.sysml.runtime.util.DataConverter;
 import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.utils.TestUtils;
@@ -166,11 +165,11 @@ public class SparseBlockIndexRange extends AutomatedTestBase
 			//init sparse block
 			SparseBlock sblock = null;
 			MatrixBlock mbtmp = DataConverter.convertToMatrixBlock(A);
-			SparseRow[] srtmp = mbtmp.getSparseBlock();			
+			SparseBlock srtmp = mbtmp.getSparseBlock();			
 			switch( btype ) {
-				case MCSR: sblock = new SparseBlockMCSR(srtmp,true); break;
-				case CSR: sblock = new SparseBlockCSR(srtmp, (int)mbtmp.getNonZeros()); break;
-				case COO: sblock = new SparseBlockCOO(srtmp, (int)mbtmp.getNonZeros()); break;
+				case MCSR: sblock = new SparseBlockMCSR(srtmp); break;
+				case CSR: sblock = new SparseBlockCSR(srtmp); break;
+				case COO: sblock = new SparseBlockCOO(srtmp); break;
 			}
 			
 			//delete range per row via set

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIterator.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIterator.java b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIterator.java
index db388c2..85e8e71 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIterator.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockIterator.java
@@ -29,7 +29,6 @@ import org.apache.sysml.runtime.matrix.data.SparseBlock;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCOO;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCSR;
 import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
 import org.apache.sysml.runtime.util.DataConverter;
 import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.utils.TestUtils;
@@ -161,11 +160,11 @@ public class SparseBlockIterator extends AutomatedTestBase
 			//init sparse block
 			SparseBlock sblock = null;
 			MatrixBlock mbtmp = DataConverter.convertToMatrixBlock(A);
-			SparseRow[] srtmp = mbtmp.getSparseBlock();			
+			SparseBlock srtmp = mbtmp.getSparseBlock();			
 			switch( btype ) {
-				case MCSR: sblock = new SparseBlockMCSR(srtmp,true); break;
-				case CSR: sblock = new SparseBlockCSR(srtmp, (int)mbtmp.getNonZeros()); break;
-				case COO: sblock = new SparseBlockCOO(srtmp, (int)mbtmp.getNonZeros()); break;
+				case MCSR: sblock = new SparseBlockMCSR(srtmp); break;
+				case CSR: sblock = new SparseBlockCSR(srtmp); break;
+				case COO: sblock = new SparseBlockCOO(srtmp); break;
 			}
 			
 			//check for correct number of non-zeros

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockScan.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockScan.java b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockScan.java
index 4f89a17..53d2b6d 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockScan.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockScan.java
@@ -26,7 +26,6 @@ import org.apache.sysml.runtime.matrix.data.SparseBlock;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCOO;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCSR;
 import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
 import org.apache.sysml.runtime.util.DataConverter;
 import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.utils.TestUtils;
@@ -111,11 +110,11 @@ public class SparseBlockScan extends AutomatedTestBase
 			//init sparse block
 			SparseBlock sblock = null;
 			MatrixBlock mbtmp = DataConverter.convertToMatrixBlock(A);
-			SparseRow[] srtmp = mbtmp.getSparseBlock();			
+			SparseBlock srtmp = mbtmp.getSparseBlock();			
 			switch( btype ) {
-				case MCSR: sblock = new SparseBlockMCSR(srtmp,true); break;
-				case CSR: sblock = new SparseBlockCSR(srtmp, (int)mbtmp.getNonZeros()); break;
-				case COO: sblock = new SparseBlockCOO(srtmp, (int)mbtmp.getNonZeros()); break;
+				case MCSR: sblock = new SparseBlockMCSR(srtmp); break;
+				case CSR: sblock = new SparseBlockCSR(srtmp); break;
+				case COO: sblock = new SparseBlockCOO(srtmp); break;
 			}
 			
 			//check for correct number of non-zeros

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8ba0fdcc/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockSize.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockSize.java b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockSize.java
index aca66dc..afeb374 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockSize.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/sparse/SparseBlockSize.java
@@ -26,7 +26,6 @@ import org.apache.sysml.runtime.matrix.data.SparseBlock;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCOO;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCSR;
 import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
-import org.apache.sysml.runtime.matrix.data.SparseRow;
 import org.apache.sysml.runtime.util.DataConverter;
 import org.apache.sysml.test.integration.AutomatedTestBase;
 import org.apache.sysml.test.utils.TestUtils;
@@ -115,11 +114,11 @@ public class SparseBlockSize extends AutomatedTestBase
 			//init sparse block
 			SparseBlock sblock = null;
 			MatrixBlock mbtmp = DataConverter.convertToMatrixBlock(A);
-			SparseRow[] srtmp = mbtmp.getSparseBlock();			
+			SparseBlock srtmp = mbtmp.getSparseBlock();			
 			switch( btype ) {
-				case MCSR: sblock = new SparseBlockMCSR(srtmp,true); break;
-				case CSR: sblock = new SparseBlockCSR(srtmp, (int)mbtmp.getNonZeros()); break;
-				case COO: sblock = new SparseBlockCOO(srtmp, (int)mbtmp.getNonZeros()); break;
+				case MCSR: sblock = new SparseBlockMCSR(srtmp); break;
+				case CSR: sblock = new SparseBlockCSR(srtmp); break;
+				case COO: sblock = new SparseBlockCOO(srtmp); break;
 			}
 			
 			//prepare summary statistics nnz