You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by mb...@apache.org on 2017/10/24 06:43:30 UTC

[1/2] systemml git commit: [SYSTEMML-1970] Performance sparse conv2d operations w/ native libs

Repository: systemml
Updated Branches:
  refs/heads/master 596005a80 -> a472ae922


[SYSTEMML-1970] Performance sparse conv2d operations w/ native libs

This patch improves the performance of sparse conv2d operations with
enabled native libraries (BLAS and native conv2d ops). So far, we called
custom native ops for sparse or dense inputs/filters, which for certain
scenarios internally converted the sparse to dense inputs/filters
respectively. With this patch, we now decide the implementation with
sparsity awareness. Additionally, this also includes a minor improvement
that avoids unnecessary nnz maintenance after native BLAS class.

On an end-to-end cnn application this patch improved performance from
605s to 349s per epoch w/ selective native ops (490s with Java ops).

Furthermore, this also fixes a recently introduced issues of
thread-local nnz maintenance in the native conv2d backward data op.


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

Branch: refs/heads/master
Commit: 2c37d9f03117d118c9d10c6839c4b1d60a4a9afc
Parents: 596005a
Author: Matthias Boehm <mb...@gmail.com>
Authored: Mon Oct 23 19:33:04 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Mon Oct 23 23:44:02 2017 -0700

----------------------------------------------------------------------
 .../cp/ConvolutionCPInstruction.java            |  6 +-
 .../sysml/runtime/matrix/data/LibMatrixDNN.java | 10 ---
 .../LibMatrixDNNConv2dBackwardDataHelper.java   |  3 +-
 .../runtime/matrix/data/LibMatrixDNNHelper.java | 71 +++++++++++++-------
 4 files changed, 50 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/2c37d9f0/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java
index 2c7b972..c6b4698 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java
@@ -363,10 +363,8 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction {
 			}
 			else {
 				outputBlock = new MatrixBlock(N, C*H*W, false).allocateBlock();
-				if(instOpcode.equalsIgnoreCase("maxpooling_backward"))
-					LibMatrixDNN.maxpoolingBackward(matBlock, dout, outputBlock, params, false);
-				else
-					LibMatrixDNN.maxpoolingBackward(matBlock, dout, outputBlock, params, true);
+				LibMatrixDNN.maxpoolingBackward(matBlock, dout, outputBlock, params, 
+					!instOpcode.equalsIgnoreCase("maxpooling_backward"));
 			}
 			ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/2c37d9f0/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java
index ac66e51..096574a 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java
@@ -34,7 +34,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysml.runtime.util.ConvolutionUtils;
-import org.apache.sysml.utils.Statistics;
 
 /*
  * This class allows users to invoke deep learning related operations 
@@ -161,9 +160,6 @@ public class LibMatrixDNN {
 		if(params.bias != null && params.bias.isInSparseFormat())
 			params.bias.sparseToDense(); // Since bias is extremely small array
 		
-		if(isEligibleForConv2dSparse(params))
-			Statistics.numNativeSparseConv2dCalls.increment();
-		
 		long nnz = execute(LibMatrixDNNHelper.getConv2dWorkers(params), params);
 		
 		//post-processing: maintain nnz
@@ -183,9 +179,6 @@ public class LibMatrixDNN {
 	public static void conv2dBackwardData(MatrixBlock filter, MatrixBlock dout, MatrixBlock outputBlock, ConvolutionParameters params) throws DMLRuntimeException {
 		checkInputsConv2dBackwardData(filter, dout, outputBlock, params);
 		
-		if(isEligibleForConv2dBackwardDataDense(params))
-			Statistics.numNativeSparseConv2dBwdDataCalls.increment();
-		
 		long nnz = execute(LibMatrixDNNHelper.getConv2dBackwardDataWorkers(params), params);
 		
 		//post-processing: maintain nnz
@@ -205,9 +198,6 @@ public class LibMatrixDNN {
 	public static void conv2dBackwardFilter(MatrixBlock input, MatrixBlock dout, MatrixBlock outputBlock, ConvolutionParameters params) throws DMLRuntimeException {
 		checkInputsConv2dBackwardFilter(input, dout, outputBlock, params);
 		
-		if(isEligibleForConv2dBackwardFilterSparseDense(params))
-			Statistics.numNativeSparseConv2dBwdFilterCalls.increment();
-		
 		execute(LibMatrixDNNHelper.getConv2dBackwardFilterWorkers(params), params);
 		
 		//post-processing: maintain nnz

http://git-wip-us.apache.org/repos/asf/systemml/blob/2c37d9f0/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java
index cd50000..960cea6 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java
@@ -57,7 +57,8 @@ public class LibMatrixDNNConv2dBackwardDataHelper {
 						_params.R, _params.S, _params.stride_h, _params.stride_w, _params.pad_h, _params.pad_w, _params.P, _params.Q, 1);
 				System.arraycopy(ret, 0, _params.output.getDenseBlock(), n*CHW, CHW);
 			}
-			return 0L;
+			//multi-threaded nnz maintenance of current working set
+			return _params.output.recomputeNonZeros(_rl, _ru-1);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/2c37d9f0/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNHelper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNHelper.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNHelper.java
index 92eb79b..55f6e4c 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNHelper.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNHelper.java
@@ -24,11 +24,15 @@ import java.util.concurrent.Callable;
 
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.matrix.data.LibMatrixDNNConv2dBackwardFilterHelper.Conv2dBackwardFilter;
-import org.apache.sysml.runtime.matrix.data.LibMatrixDNNConv2dBackwardFilterHelper.SparseNativeConv2dBackwardFilterDense;
+import org.apache.sysml.runtime.matrix.data.LibMatrixDNNConv2dBackwardDataHelper.*;
+import org.apache.sysml.runtime.matrix.data.LibMatrixDNNConv2dBackwardFilterHelper.*;
+import org.apache.sysml.runtime.matrix.data.LibMatrixDNNConv2dHelper.*;
+import org.apache.sysml.runtime.matrix.data.LibMatrixDNNPoolingBackwardHelper.*;
+import org.apache.sysml.runtime.matrix.data.LibMatrixDNNPoolingHelper.*;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.util.ConvolutionUtils;
 import org.apache.sysml.utils.NativeHelper;
+import org.apache.sysml.utils.Statistics;
 
 
 public class LibMatrixDNNHelper {
@@ -51,9 +55,9 @@ public class LibMatrixDNNHelper {
 		int taskSize = (int)(Math.ceil((double)params.N / k));
 		for(int i = 0; i*taskSize < params.N; i++) {
 			if(params.input1.isInSparseFormat())
-				ret.add(new LibMatrixDNNPoolingHelper.SparseMaxPooling(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
+				ret.add(new SparseMaxPooling(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
 			else
-				ret.add(new LibMatrixDNNPoolingHelper.DenseMaxPooling(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
+				ret.add(new DenseMaxPooling(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
 		}
 		return ret;
 	}
@@ -73,15 +77,15 @@ public class LibMatrixDNNHelper {
 		for(int i = 0; i*taskSize < params.N; i++) {
 			if(!params.input1.isInSparseFormat()) {
 				if(!params.input2.isInSparseFormat()) 
-					ret.add(new LibMatrixDNNPoolingBackwardHelper.PoolingBackwardDenseDense(i*taskSize, Math.min((i+1)*taskSize, params.N), params, performReluBackward));
+					ret.add(new PoolingBackwardDenseDense(i*taskSize, Math.min((i+1)*taskSize, params.N), params, performReluBackward));
 				else
-					ret.add(new LibMatrixDNNPoolingBackwardHelper.PoolingBackwardDenseSparse(i*taskSize, Math.min((i+1)*taskSize, params.N), params, performReluBackward));
+					ret.add(new PoolingBackwardDenseSparse(i*taskSize, Math.min((i+1)*taskSize, params.N), params, performReluBackward));
 			}
 			else {
 				if(!params.input2.isInSparseFormat()) 
-					ret.add(new LibMatrixDNNPoolingBackwardHelper.PoolingBackwardSparseDense(i*taskSize, Math.min((i+1)*taskSize, params.N), params, performReluBackward));
+					ret.add(new PoolingBackwardSparseDense(i*taskSize, Math.min((i+1)*taskSize, params.N), params, performReluBackward));
 				else
-					ret.add(new LibMatrixDNNPoolingBackwardHelper.PoolingBackwardSparseSparse(i*taskSize, Math.min((i+1)*taskSize, params.N), params, performReluBackward));
+					ret.add(new PoolingBackwardSparseSparse(i*taskSize, Math.min((i+1)*taskSize, params.N), params, performReluBackward));
 			}
 		}
 		return ret;
@@ -123,32 +127,36 @@ public class LibMatrixDNNHelper {
 		// TODO: Decide here based on params whether to use LoopedIm2ColConv2dAllChannels or LoopedIm2ColConv2dOneChannel
 		// For now, let's stick to the existing approach of converting [1, CHW] to [CRS, PQ] as it allows matrix multiplication large enough matrix.
 		boolean allChannels = true; ArrayList<MatrixBlock> filters = null;
-		if(!allChannels) {
+		if(!allChannels)
 			filters = splitFilter(params);
-		}
 		
 		MatrixBlock in1 = params.input1;
 		boolean isEmptyDenseInput = !in1.isInSparseFormat() && in1.denseBlock == null;
 		boolean isTransPref = in1.sparse && !params.input2.sparse && 
 			MatrixBlock.evalSparseFormatInMemory(in1.clen, in1.rlen, in1.nonZeros);
+		boolean applyNative = LibMatrixDNN.isEligibleForConv2dSparse(params)
+			&& !(!isEmptyDenseInput && allChannels && isTransPref);
+		if( applyNative )
+			Statistics.numNativeSparseConv2dCalls.increment();
 		
 		//transpose filter once for efficient sparse-dense multiplies in LoopedIm2ColConv2dTransAllChan
 		//in order to share the temporary object and its creation costs across threads
-		if( !LibMatrixDNN.isEligibleForConv2dSparse(params) 
-			&& !isEmptyDenseInput && allChannels && isTransPref ) {
+		if( !applyNative && !isEmptyDenseInput && allChannels && isTransPref ) {
 			params.input2 = LibMatrixReorg.transpose(params.input2, 
 				new MatrixBlock(params.input2.clen, params.input2.rlen, false), k);
 		}
 		
 		for(int i = 0; i*taskSize < params.N; i++) {
-			if(LibMatrixDNN.isEligibleForConv2dSparse(params)) 
-				ret.add(new LibMatrixDNNConv2dHelper.SparseNativeConv2d(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
+			//note: we prefer the java backend for sparse inputs because the native 
+			//implementation simply converts the sparse input into dense rows
+			if( applyNative ) 
+				ret.add(new SparseNativeConv2d(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
 			else if(!isEmptyDenseInput && allChannels && isTransPref)
-				ret.add(new LibMatrixDNNConv2dHelper.LoopedIm2ColConv2dTransAllChan(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
+				ret.add(new LoopedIm2ColConv2dTransAllChan(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
 			else if(!isEmptyDenseInput && allChannels)
-				ret.add(new LibMatrixDNNConv2dHelper.LoopedIm2ColConv2dAllChan(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
+				ret.add(new LoopedIm2ColConv2dAllChan(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
 			else if(!isEmptyDenseInput && !allChannels)
-				ret.add(new LibMatrixDNNConv2dHelper.LoopedIm2ColConv2dOneChan(i*taskSize, Math.min((i+1)*taskSize, params.N), params, filters));
+				ret.add(new LoopedIm2ColConv2dOneChan(i*taskSize, Math.min((i+1)*taskSize, params.N), params, filters));
 			else
 				throw new DMLRuntimeException("Unsupported operator");
 		}
@@ -172,9 +180,15 @@ public class LibMatrixDNNHelper {
 		
 		boolean isEmptyDenseInput = (!params.input1.isInSparseFormat() && params.input1.denseBlock == null) || 
 			(!params.input2.isInSparseFormat() && params.input2.denseBlock == null);
+		boolean applyNative = LibMatrixDNN.isEligibleForConv2dBackwardFilterSparseDense(params)
+			&& !params.input2.isInSparseFormat();
+		if( applyNative )
+			Statistics.numNativeSparseConv2dBwdFilterCalls.increment();
 		
 		for(int i = 0; i*taskSize < params.N; i++) {
-			if(LibMatrixDNN.isEligibleForConv2dBackwardFilterSparseDense(params)) 
+			//note: we prefer the java backend for sparse filters because the native 
+			//implementation simply rotates the sparse filters into dense rows
+			if( applyNative ) 
 				ret.add(new SparseNativeConv2dBackwardFilterDense(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
 			else if(!isEmptyDenseInput)
 				ret.add(new Conv2dBackwardFilter(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
@@ -202,12 +216,18 @@ public class LibMatrixDNNHelper {
 		
 		boolean isEmptyDenseInput = (!params.input1.isInSparseFormat() && params.input1.denseBlock == null) || 
 			(!params.input2.isInSparseFormat() && params.input2.denseBlock == null);
+		boolean applyNative = LibMatrixDNN.isEligibleForConv2dBackwardDataDense(params)
+			&& !params.input2.isInSparseFormat();
+		if( applyNative )
+			Statistics.numNativeSparseConv2dBwdDataCalls.increment();
 		
 		for(int i = 0; i*taskSize < params.N; i++) {
-			if(LibMatrixDNN.isEligibleForConv2dBackwardDataDense(params)) 
-				ret.add(new LibMatrixDNNConv2dBackwardDataHelper.SparseNativeConv2dBackwardDataDense(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
+			//note: we prefer the java backend for sparse filters because the native 
+			//implementation simply converts the sparse filters into dense rows
+			if( applyNative ) 
+				ret.add(new SparseNativeConv2dBackwardDataDense(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
 			else if(!isEmptyDenseInput)
-				ret.add(new LibMatrixDNNConv2dBackwardDataHelper.Conv2dBackwardData(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
+				ret.add(new Conv2dBackwardData(i*taskSize, Math.min((i+1)*taskSize, params.N), params));
 			else
 				throw new DMLRuntimeException("Unsupported operator");
 		}
@@ -319,20 +339,21 @@ public class LibMatrixDNNHelper {
 	// Single-threaded matrix multiplication
 	static void singleThreadedMatMult(MatrixBlock m1, MatrixBlock m2, MatrixBlock ret, 
 			boolean recomputeNNZM1, boolean recomputeNNZM2, ConvolutionParameters params) throws DMLRuntimeException {
-		if(!params.enableNative || m1.isInSparseFormat() || m2.isInSparseFormat()) {
+		if( !params.enableNative || m1.sparse || m2.sparse ) {
 			prepNonZerosForMatrixMult(m1, recomputeNNZM1);
 			prepNonZerosForMatrixMult(m2, recomputeNNZM2);
 			LibMatrixMult.matrixMult(m1, m2, ret, false);
-			ret.setNonZeros((long)ret.rlen*ret.clen);
 		}
 		else {
 			ret.sparse = false;
 			if(ret.getDenseBlock() == null)
 				ret.allocateDenseBlock();
 			NativeHelper.matrixMultDenseDense(m1.denseBlock, m2.denseBlock, 
-					ret.denseBlock, m1.getNumRows(), m1.getNumColumns(), m2.getNumColumns(), 1);
-			ret.recomputeNonZeros();
+				ret.denseBlock, m1.rlen, m1.clen, m2.clen, 1);
 		}
+		
+		//no need to maintain nnz exactly, as consumed by other operations
+		ret.setNonZeros((long)ret.rlen*ret.clen);
 	}
 	
 	static void addBias(int r, double [] out, double [] bias, int K, int PQ) {


[2/2] systemml git commit: [SYSTEMML-1972] Fix rewrite remove right indexing (w/ invalid ix range)

Posted by mb...@apache.org.
[SYSTEMML-1972] Fix rewrite remove right indexing (w/ invalid ix range)

This patch hardens the existing rewrite of removing unnecessary right
indexing operations whose input and output are of equal size, which is
only valid with valid indexing ranges. Although we check this during
validation, there are scenarios with unknown sizes or index expressions
that cause invalid results despite invalid index ranges. We now simply
check for valid row-lower and column-lower indexing ranges which both
needs to be 1 for the rewrite to be valid.


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

Branch: refs/heads/master
Commit: a472ae922827b437e00ca8331ff3db5f6c19f443
Parents: 2c37d9f
Author: Matthias Boehm <mb...@gmail.com>
Authored: Mon Oct 23 23:43:46 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Mon Oct 23 23:44:04 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/sysml/hops/IndexingOp.java  |  5 +----
 .../sysml/hops/rewrite/HopRewriteUtils.java     | 13 +++++++++++
 .../RewriteAlgebraicSimplificationDynamic.java  | 23 ++++++--------------
 3 files changed, 21 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/a472ae92/src/main/java/org/apache/sysml/hops/IndexingOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/IndexingOp.java b/src/main/java/org/apache/sysml/hops/IndexingOp.java
index 23d0630..5989c66 100644
--- a/src/main/java/org/apache/sysml/hops/IndexingOp.java
+++ b/src/main/java/org/apache/sysml/hops/IndexingOp.java
@@ -118,10 +118,7 @@ public class IndexingOp extends Hop
 		Hop input = getInput().get(0);
 		
 		//rewrite remove unnecessary right indexing
-		if( dimsKnown() && input.dimsKnown() 
-			&& getDim1() == input.getDim1() && getDim2() == input.getDim2()
-			&& !(getDim1()==1 && getDim2()==1))
-		{
+		if( HopRewriteUtils.isUnnecessaryRightIndexing(this) ) {
 			setLops( input.constructLops() );
 		}
 		//actual lop construction, incl operator selection 

http://git-wip-us.apache.org/repos/asf/systemml/blob/a472ae92/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java b/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
index 68068eb..ad2392a 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
@@ -1000,6 +1000,19 @@ public class HopRewriteUtils
 			&& hop.getInput().get(4) instanceof LiteralOp;
 	}
 	
+	public static boolean isUnnecessaryRightIndexing(Hop hop) {
+		if( !(hop instanceof IndexingOp) )
+			return false;
+		//note: in addition to equal sizes, we also check a valid
+		//starting row and column ranges of 1 in order to guard against
+		//invalid modifications in the presence of invalid index ranges
+		//(e.g., X[,2] on a column vector needs to throw an error)
+		return isEqualSize(hop, hop.getInput().get(0))
+			&& !(hop.getDim1()==1 && hop.getDim2()==1)
+			&& isLiteralOfValue(hop.getInput().get(1), 1)  //rl
+			&& isLiteralOfValue(hop.getInput().get(3), 1); //cl
+	}
+	
 	public static boolean isScalarMatrixBinaryMult( Hop hop ) {
 		return hop instanceof BinaryOp && ((BinaryOp)hop).getOp()==OpOp2.MULT
 			&& ((hop.getInput().get(0).getDataType()==DataType.SCALAR && hop.getInput().get(1).getDataType()==DataType.MATRIX)

http://git-wip-us.apache.org/repos/asf/systemml/blob/a472ae92/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
index 5437535..eba06fc 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
@@ -230,23 +230,14 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 	
 	private static Hop removeUnnecessaryRightIndexing(Hop parent, Hop hi, int pos)
 	{
-		if( hi instanceof IndexingOp ) //indexing op
-		{
+		if( HopRewriteUtils.isUnnecessaryRightIndexing(hi) ) {
+			//remove unnecessary right indexing
 			Hop input = hi.getInput().get(0);
-			if( HopRewriteUtils.isEqualSize(hi, input)     //equal dims
-				&& !(hi.getDim1()==1 && hi.getDim2()==1) ) //not 1-1 matrix/frame	
-			{
-				//equal dims of right indexing input and output -> no need for indexing
-				//(not applied for 1-1 matrices because low potential and issues w/ error
-				//handling if out of range indexing)
-				
-				//remove unnecessary right indexing
-				HopRewriteUtils.replaceChildReference(parent, hi, input, pos);
-				HopRewriteUtils.cleanupUnreferenced(hi);
-				hi = input;
-				
-				LOG.debug("Applied removeUnnecessaryRightIndexing");
-			}			
+			HopRewriteUtils.replaceChildReference(parent, hi, input, pos);
+			HopRewriteUtils.cleanupUnreferenced(hi);
+			hi = input;
+			
+			LOG.debug("Applied removeUnnecessaryRightIndexing");
 		}
 		
 		return hi;