You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by ni...@apache.org on 2017/08/24 21:41:33 UTC

[3/5] systemml git commit: [SYSTEMML-1793] Support matrix range indexing on GPU

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/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 5f2ce34..6a3ddf4 100644
--- a/src/main/java/org/apache/sysml/hops/IndexingOp.java
+++ b/src/main/java/org/apache/sysml/hops/IndexingOp.java
@@ -19,6 +19,7 @@
 
 package org.apache.sysml.hops;
 
+import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.hops.AggBinaryOp.SparkAggType;
 import org.apache.sysml.hops.rewrite.HopRewriteUtils;
 import org.apache.sysml.lops.Aggregate;
@@ -97,7 +98,13 @@ public class IndexingOp extends Hop
 	
 	@Override
 	public boolean isGPUEnabled() {
-		return false;
+		if(!DMLScript.USE_ACCELERATOR) {
+			return false;
+		}
+		else {
+			// only matrix indexing is supported on GPU
+			return (getDataType() == DataType.MATRIX);
+		}
 	}
 
 	@Override
@@ -172,7 +179,7 @@ public class IndexingOp extends Hop
 					setLineNumbers(reindex);
 					setLops(reindex);
 				}
-				else //CP
+				else //CP or GPU
 				{
 					Lop dummy = Data.createLiteralLop(ValueType.INT, Integer.toString(-1));
 					RangeBasedReIndex reindex = new RangeBasedReIndex(

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/java/org/apache/sysml/runtime/instructions/GPUInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/GPUInstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/GPUInstructionParser.java
index 36f57b4..8abfc69 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/GPUInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/GPUInstructionParser.java
@@ -27,6 +27,7 @@ import org.apache.sysml.runtime.instructions.gpu.BuiltinBinaryGPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.BuiltinUnaryGPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.ConvolutionGPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.GPUInstruction;
+import org.apache.sysml.runtime.instructions.gpu.MatrixIndexingGPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.MatrixMatrixAxpyGPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.GPUInstruction.GPUINSTRUCTION_TYPE;
 import org.apache.sysml.runtime.instructions.gpu.MMTSJGPUInstruction;
@@ -128,6 +129,10 @@ public class GPUInstructionParser  extends InstructionParser
 		String2GPUInstructionType.put( ">"    , GPUINSTRUCTION_TYPE.RelationalBinary);
 		String2GPUInstructionType.put( "<="   , GPUINSTRUCTION_TYPE.RelationalBinary);
 		String2GPUInstructionType.put( ">="   , GPUINSTRUCTION_TYPE.RelationalBinary);
+		
+		// Indexing 
+		// right indexing: output = X[1:3, 4:5]
+		String2GPUInstructionType.put( "rangeReIndex", GPUINSTRUCTION_TYPE.MatrixIndexing); 
 	}
 	
 	public static GPUInstruction parseSingleInstruction (String str ) 
@@ -187,6 +192,9 @@ public class GPUInstructionParser  extends InstructionParser
 			case RelationalBinary:
 				return RelationalBinaryGPUInstruction.parseInstruction(str);
 
+			case MatrixIndexing:
+				return MatrixIndexingGPUInstruction.parseInstruction(str);
+				
 			default: 
 				throw new DMLRuntimeException("Invalid GPU Instruction Type: " + gputype );
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java
index b962eb7..0f0b28e 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java
@@ -43,7 +43,8 @@ public abstract class GPUInstruction extends Instruction
 		ArithmeticBinary,
 		BuiltinUnary,
 		BuiltinBinary,
-		Builtin
+		Builtin,
+		MatrixIndexing
 	};
 
 	// Memory/conversions
@@ -112,6 +113,9 @@ public abstract class GPUInstruction extends Instruction
 	public final static String MISC_TIMER_REDUCE_ALL_KERNEL =                "rallk";  // time spent in reduce all kernel
 	public final static String MISC_TIMER_REDUCE_ROW_KERNEL =                "rrowk";  // time spent in reduce row kernel
 	public final static String MISC_TIMER_REDUCE_COL_KERNEL =                "rcolk";  // time spent in reduce column kernel
+	
+	public final static String MISC_TIMER_RIX_DENSE_OP =                     "drix";    // time spent in the right indexing dense kernel
+	public final static String MISC_TIMER_RIX_SPARSE_DENSE_OP =              "sdrix";   // time spent in the right indexing sparse dense kernel
 
 	// Deep learning operators
 	public final static String MISC_TIMER_ACTIVATION_FORWARD_LIB =         "nnaf";  // time spent in cudnnActivationForward

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/java/org/apache/sysml/runtime/instructions/gpu/MatrixIndexingGPUInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/MatrixIndexingGPUInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/MatrixIndexingGPUInstruction.java
new file mode 100644
index 0000000..5e2c8fc
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/MatrixIndexingGPUInstruction.java
@@ -0,0 +1,148 @@
+/*
+ * 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.instructions.gpu;
+
+import org.apache.sysml.parser.Expression.DataType;
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
+import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
+import org.apache.sysml.runtime.instructions.InstructionUtils;
+import org.apache.sysml.runtime.instructions.cp.CPOperand;
+import org.apache.sysml.runtime.matrix.data.LibMatrixCUDA;
+import org.apache.sysml.runtime.matrix.operators.Operator;
+import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
+import org.apache.sysml.runtime.util.IndexRange;
+import org.apache.sysml.utils.GPUStatistics;
+
+public class MatrixIndexingGPUInstruction extends GPUInstruction {
+	CPOperand rowLower, rowUpper, colLower, colUpper;
+	CPOperand input1; CPOperand input2; CPOperand output;
+	
+	public MatrixIndexingGPUInstruction(Operator op, CPOperand in, 
+			CPOperand rl, CPOperand ru, CPOperand cl, CPOperand cu, CPOperand out, String opcode, String istr){
+		super(op, opcode, istr);
+		_gputype = GPUINSTRUCTION_TYPE.MatrixIndexing;
+		rowLower = rl;
+		rowUpper = ru;
+		colLower = cl;
+		colUpper = cu;
+		input1 = in;
+		output = out;
+	}
+	
+	public MatrixIndexingGPUInstruction(Operator op, CPOperand lhsInput, CPOperand rhsInput, 
+			CPOperand rl, CPOperand ru, CPOperand cl, CPOperand cu, CPOperand out, String opcode, String istr){
+		super(op, opcode, istr);
+		_gputype = GPUINSTRUCTION_TYPE.MatrixIndexing;
+		rowLower = rl;
+		rowUpper = ru;
+		colLower = cl;
+		colUpper = cu;
+		input1 = lhsInput;
+		input2 = rhsInput;
+		output = out;
+	}
+	
+	public static MatrixIndexingGPUInstruction parseInstruction ( String str ) throws DMLRuntimeException {
+		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);
+		String opcode = parts[0];
+		
+		if ( opcode.equalsIgnoreCase("rangeReIndex") ) {
+			if ( parts.length == 7 ) {
+				// Example: rangeReIndex:mVar1:Var2:Var3:Var4:Var5:mVar6
+				CPOperand in, rl, ru, cl, cu, out;
+				in = new CPOperand();
+				rl = new CPOperand();
+				ru = new CPOperand();
+				cl = new CPOperand();
+				cu = new CPOperand();
+				out = new CPOperand();
+				in.split(parts[1]);
+				rl.split(parts[2]);
+				ru.split(parts[3]);
+				cl.split(parts[4]);
+				cu.split(parts[5]);
+				out.split(parts[6]);
+				if( in.getDataType()==DataType.MATRIX )
+					return new MatrixIndexingGPUInstruction(new SimpleOperator(null), in, rl, ru, cl, cu, out, opcode, str);
+				else 
+					throw new DMLRuntimeException("Can index only on Matrices in GPU");
+			}
+			else {
+				throw new DMLRuntimeException("Invalid number of operands in instruction: " + str);
+			}
+		} 
+		else if ( opcode.equalsIgnoreCase("leftIndex")) {
+			if ( parts.length == 8 ) {
+				// Example: leftIndex:mVar1:mvar2:Var3:Var4:Var5:Var6:mVar7
+				CPOperand lhsInput, rhsInput, rl, ru, cl, cu, out;
+				lhsInput = new CPOperand();
+				rhsInput = new CPOperand();
+				rl = new CPOperand();
+				ru = new CPOperand();
+				cl = new CPOperand();
+				cu = new CPOperand();
+				out = new CPOperand();
+				lhsInput.split(parts[1]);
+				rhsInput.split(parts[2]);
+				rl.split(parts[3]);
+				ru.split(parts[4]);
+				cl.split(parts[5]);
+				cu.split(parts[6]);
+				out.split(parts[7]);
+				if( lhsInput.getDataType()==DataType.MATRIX )
+					return new MatrixIndexingGPUInstruction(new SimpleOperator(null), lhsInput, rhsInput, rl, ru, cl, cu, out, opcode, str);
+				else 
+					throw new DMLRuntimeException("Can index only on Matrices in GPU");
+			}
+			else {
+				throw new DMLRuntimeException("Invalid number of operands in instruction: " + str);
+			}
+		}
+		else {
+			throw new DMLRuntimeException("Unknown opcode while parsing a MatrixIndexingGPUInstruction: " + str);
+		}
+	}
+
+	@Override
+	public void processInstruction(ExecutionContext ec)
+			throws DMLRuntimeException {
+		GPUStatistics.incrementNoOfExecutedGPUInst();
+		String opcode = getOpcode();
+		
+		IndexRange ixrange = getIndexRange(ec);
+		if ( opcode.equalsIgnoreCase("rangeReIndex") ) {
+			MatrixObject mat1 = getMatrixInputForGPUInstruction(ec, input1.getName());
+			LibMatrixCUDA.sliceOperations(ec, ec.getGPUContext(0), getExtendedOpcode(), mat1, ixrange, output.getName());
+			ec.releaseMatrixInputForGPUInstruction(input1.getName());
+			ec.releaseMatrixOutputForGPUInstruction(output.getName());
+		}
+		else {
+			throw new DMLRuntimeException("Unsupported GPU operator:" + opcode);
+		}
+	}
+	
+	IndexRange getIndexRange(ExecutionContext ec) throws DMLRuntimeException {
+		return new IndexRange( //rl, ru, cl, ru
+			(int)(ec.getScalarInput(rowLower.getName(), rowLower.getValueType(), rowLower.isLiteral()).getLongValue()-1),
+			(int)(ec.getScalarInput(rowUpper.getName(), rowUpper.getValueType(), rowUpper.isLiteral()).getLongValue()-1),
+			(int)(ec.getScalarInput(colLower.getName(), colLower.getValueType(), colLower.isLiteral()).getLongValue()-1),
+			(int)(ec.getScalarInput(colUpper.getName(), colUpper.getValueType(), colUpper.isLiteral()).getLongValue()-1));		
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java
index 7244938..a5bc299 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java
@@ -49,6 +49,18 @@ import jcuda.jcusparse.cusparsePointerMode;
 /**
  * Compressed Sparse Row (CSR) format for CUDA
  * Generalized matrix multiply is implemented for CSR format in the cuSparse library among other operations
+ * 
+ * Since we assume that the matrix is stored with zero-based indexing (i.e. CUSPARSE_INDEX_BASE_ZERO),
+ * the matrix
+ * 1.0 4.0 0.0 0.0 0.0 
+ * 0.0 2.0 3.0 0.0 0.0 
+ * 5.0 0.0 0.0 7.0 8.0 
+ * 0.0 0.0 9.0 0.0 6.0
+ * 
+ * is stored as
+ * val = 1.0 4.0 2.0 3.0 5.0 7.0 8.0 9.0 6.0 
+ * rowPtr = 0.0 2.0 4.0 7.0 9.0 
+ * colInd = 0.0 1.0 1.0 2.0 0.0 3.0 4.0 2.0 4.0
  */
 public class CSRPointer {
 
@@ -184,9 +196,9 @@ public class CSRPointer {
 		cudaMemcpy(r.colInd, Pointer.to(colInd), getIntSizeOf(nnz), cudaMemcpyHostToDevice);
 		cudaMemcpy(r.val, Pointer.to(values), getDoubleSizeOf(nnz), cudaMemcpyHostToDevice);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaToDevTime.addAndGet(System.nanoTime() - t0);
+			GPUStatistics.cudaToDevTime.add(System.nanoTime() - t0);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaToDevCount.addAndGet(3);
+			GPUStatistics.cudaToDevCount.add(3);
 	}
 
 	/**
@@ -208,9 +220,9 @@ public class CSRPointer {
 		cudaMemcpy(Pointer.to(colInd), r.colInd, getIntSizeOf(nnz), cudaMemcpyDeviceToHost);
 		cudaMemcpy(Pointer.to(values), r.val, getDoubleSizeOf(nnz), cudaMemcpyDeviceToHost);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime() - t0);
+			GPUStatistics.cudaFromDevTime.add(System.nanoTime() - t0);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaFromDevCount.addAndGet(3);
+			GPUStatistics.cudaFromDevCount.add(3);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
index f107f47..84d181b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
@@ -147,7 +147,7 @@ public class GPUContext {
 
 		if (DMLScript.STATISTICS)
 			GPUStatistics.cudaLibrariesInitTime = System.nanoTime() - start;
-		
+
 		LOG.info(" GPU memory - Total: " + (total[0] * (1e-6)) + " MB, Available: " + (free[0] * (1e-6)) + " MB on "
 				+ this);
 
@@ -269,7 +269,7 @@ public class GPUContext {
 				freeCUDASpaceMap.remove(size);
 			if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
 				GPUStatistics
-						.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_REUSE, System.nanoTime() - t0);
+				.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_REUSE, System.nanoTime() - t0);
 		} else {
 			LOG.trace(
 					"GPU : in allocate from instruction " + instructionName + ", allocating new block of size " + (size
@@ -280,9 +280,9 @@ public class GPUContext {
 			A = new Pointer();
 			cudaMalloc(A, size);
 			if (DMLScript.STATISTICS)
-				GPUStatistics.cudaAllocTime.getAndAdd(System.nanoTime() - t0);
+				GPUStatistics.cudaAllocTime.add(System.nanoTime() - t0);
 			if (DMLScript.STATISTICS)
-				GPUStatistics.cudaAllocCount.getAndAdd(statsCount);
+				GPUStatistics.cudaAllocCount.add(statsCount);
 			if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_ALLOCATE,
 						System.nanoTime() - t0);
@@ -298,9 +298,9 @@ public class GPUContext {
 		if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_SET_ZERO, end - t1);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaMemSet0Time.getAndAdd(end - t1);
+			GPUStatistics.cudaMemSet0Time.add(end - t1);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaMemSet0Count.getAndAdd(1);
+			GPUStatistics.cudaMemSet0Count.add(1);
 		cudaBlockSizeMap.put(A, size);
 		return A;
 
@@ -349,32 +349,32 @@ public class GPUContext {
 		long t0 = 0;
 		assert cudaBlockSizeMap.containsKey(
 				toFree) : "ERROR : Internal state corrupted, cache block size map is not aware of a block it trying to free up";
-		long size = cudaBlockSizeMap.get(toFree);
-		if (eager) {
-			LOG.trace("GPU : eagerly freeing cuda memory [ " + toFree + " ] for instruction " + instructionName + " on "
-					+ this);
-			if (DMLScript.STATISTICS)
-				t0 = System.nanoTime();
-			cudaFree(toFree);
-			cudaBlockSizeMap.remove(toFree);
-			if (DMLScript.STATISTICS)
-				GPUStatistics.cudaDeAllocTime.addAndGet(System.nanoTime() - t0);
-			if (DMLScript.STATISTICS)
-				GPUStatistics.cudaDeAllocCount.addAndGet(1);
-			if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
-				GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_CUDA_FREE,
-						System.nanoTime() - t0);
-		} else {
-			LOG.trace("GPU : lazily freeing cuda memory for instruction " + instructionName + " on " + this);
-			LinkedList<Pointer> freeList = freeCUDASpaceMap.get(size);
-			if (freeList == null) {
-				freeList = new LinkedList<Pointer>();
-				freeCUDASpaceMap.put(size, freeList);
-			}
-			if (freeList.contains(toFree))
-				throw new RuntimeException("GPU : Internal state corrupted, double free");
-			freeList.add(toFree);
-		}
+				long size = cudaBlockSizeMap.get(toFree);
+				if (eager) {
+					LOG.trace("GPU : eagerly freeing cuda memory [ " + toFree + " ] for instruction " + instructionName + " on "
+							+ this);
+					if (DMLScript.STATISTICS)
+						t0 = System.nanoTime();
+					cudaFree(toFree);
+					cudaBlockSizeMap.remove(toFree);
+					if (DMLScript.STATISTICS)
+						GPUStatistics.cudaDeAllocTime.add(System.nanoTime() - t0);
+					if (DMLScript.STATISTICS)
+						GPUStatistics.cudaDeAllocCount.add(1);
+					if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
+						GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_CUDA_FREE,
+								System.nanoTime() - t0);
+				} else {
+					LOG.trace("GPU : lazily freeing cuda memory for instruction " + instructionName + " on " + this);
+					LinkedList<Pointer> freeList = freeCUDASpaceMap.get(size);
+					if (freeList == null) {
+						freeList = new LinkedList<Pointer>();
+						freeCUDASpaceMap.put(size, freeList);
+					}
+					if (freeList.contains(toFree))
+						throw new RuntimeException("GPU : Internal state corrupted, double free");
+					freeList.add(toFree);
+				}
 	}
 
 	/**
@@ -426,7 +426,7 @@ public class GPUContext {
 	 */
 	protected void evict(String instructionName, final long neededSize) throws DMLRuntimeException {
 		LOG.trace("GPU : evict called from " + instructionName + " for size " + neededSize + " on " + this);
-		GPUStatistics.cudaEvictionCount.addAndGet(1);
+		GPUStatistics.cudaEvictionCount.add(1);
 		// Release the set of free blocks maintained in a GPUObject.freeCUDASpaceMap
 		// to free up space
 		LRUCacheMap<Long, LinkedList<Pointer>> lruCacheMap = freeCUDASpaceMap;

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
index 94ceb36..c3e23f3 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
@@ -32,9 +32,7 @@ import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost;
 import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice;
 
 import java.util.Arrays;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysml.api.DMLScript;
@@ -100,12 +98,12 @@ public class GPUObject {
 	/**
 	 * number of read/write locks on this object (this GPUObject is being used in a current instruction)
 	 */
-	protected AtomicInteger locks = new AtomicInteger(0);
+	protected AtomicLong locks = new AtomicLong();
 
 	/**
 	 * Timestamp, needed by {@link GPUContext#evict(long)}
 	 */
-	AtomicLong timestamp = new AtomicLong(0);
+	AtomicLong timestamp = new AtomicLong();
 
 	/**
 	 * Whether this block is in sparse format
@@ -131,7 +129,7 @@ public class GPUObject {
 			that.allocateTensorDescriptor(me.tensorShape[0], me.tensorShape[1], me.tensorShape[2], me.tensorShape[3]);
 		}
 		that.dirty = me.dirty;
-		that.locks = new AtomicInteger(me.locks.get());
+		that.locks = new AtomicLong(me.locks.get());
 		that.timestamp = new AtomicLong(me.timestamp.get());
 		that.isSparse = me.isSparse;
 
@@ -238,9 +236,9 @@ public class GPUObject {
 			t2 = System.nanoTime();
 		cudaMemcpy(Pointer.to(nnzC), nnzTotalDevHostPtr, getIntSizeOf(1), cudaMemcpyDeviceToHost);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime() - t2);
+			GPUStatistics.cudaFromDevTime.add(System.nanoTime() - t2);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaFromDevCount.addAndGet(1);
+			GPUStatistics.cudaFromDevCount.add(1);
 
 		if (nnzC[0] == -1) {
 			throw new DMLRuntimeException(
@@ -353,9 +351,9 @@ public class GPUObject {
 						cols));
 		// TODO: What if mat.getNnz() is -1 ?
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaDenseToSparseTime.addAndGet(System.nanoTime() - t0);
+			GPUStatistics.cudaDenseToSparseTime.add(System.nanoTime() - t0);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaDenseToSparseCount.addAndGet(1);
+			GPUStatistics.cudaDenseToSparseCount.add(1);
 	}
 
 	/**
@@ -430,9 +428,9 @@ public class GPUObject {
 		if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, end - start);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaSparseToDenseTime.addAndGet(end - start);
+			GPUStatistics.cudaSparseToDenseTime.add(end - start);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaSparseToDenseCount.addAndGet(1);
+			GPUStatistics.cudaSparseToDenseCount.add(1);
 	}
 
 	/**
@@ -560,8 +558,8 @@ public class GPUObject {
 		// If the fill value is 0, no need to call the special kernel, the allocate memsets the allocated region to 0
 		if (v != 0)
 			getGPUContext().getKernels()
-					.launchKernel("fill", ExecutionConfig.getConfigForSimpleVectorOperations(numElems),
-							getJcudaDenseMatrixPtr(), v, numElems);
+			.launchKernel("fill", ExecutionConfig.getConfigForSimpleVectorOperations(numElems),
+					getJcudaDenseMatrixPtr(), v, numElems);
 	}
 
 	/**
@@ -657,12 +655,12 @@ public class GPUObject {
 	 * @throws DMLRuntimeException if there is no locked GPU Object or if could not obtain a {@link GPUContext}
 	 */
 	private void updateReleaseLocks(int l) throws DMLRuntimeException {
-		int newLocks = locks.addAndGet(l);
+		int newLocks = (int) locks.addAndGet(l);
 		if (newLocks < 0) {
 			throw new CacheException("Internal state error : Invalid number of locks on a GPUObject");
 		}
 
-		LOG.trace("GPU : updateReleaseLocks, new number of locks is " + locks.get() + ", on " + this + ", GPUContext="
+		LOG.trace("GPU : updateReleaseLocks, new number of locks is " + newLocks + ", on " + this + ", GPUContext="
 				+ getGPUContext());
 		GPUContext.EvictionPolicy evictionPolicy = getGPUContext().evictionPolicy;
 		switch (evictionPolicy) {
@@ -802,18 +800,18 @@ public class GPUObject {
 					csrBlock = new SparseBlockCSR(toIntExact(mat.getNumRows()), cooBlock.rowIndexes(),
 							cooBlock.indexes(), cooBlock.values());
 					if (DMLScript.STATISTICS)
-						GPUStatistics.cudaSparseConversionTime.addAndGet(System.nanoTime() - t0);
+						GPUStatistics.cudaSparseConversionTime.add(System.nanoTime() - t0);
 					if (DMLScript.STATISTICS)
-						GPUStatistics.cudaSparseConversionCount.incrementAndGet();
+						GPUStatistics.cudaSparseConversionCount.increment();
 				} else if (block instanceof SparseBlockMCSR) {
 					if (DMLScript.STATISTICS)
 						t0 = System.nanoTime();
 					SparseBlockMCSR mcsrBlock = (SparseBlockMCSR) block;
 					csrBlock = new SparseBlockCSR(mcsrBlock.getRows(), toIntExact(mcsrBlock.size()));
 					if (DMLScript.STATISTICS)
-						GPUStatistics.cudaSparseConversionTime.addAndGet(System.nanoTime() - t0);
+						GPUStatistics.cudaSparseConversionTime.add(System.nanoTime() - t0);
 					if (DMLScript.STATISTICS)
-						GPUStatistics.cudaSparseConversionCount.incrementAndGet();
+						GPUStatistics.cudaSparseConversionCount.increment();
 				} else {
 					throw new DMLRuntimeException("Unsupported sparse matrix format for CUDA operations");
 				}
@@ -848,9 +846,9 @@ public class GPUObject {
 		mat.release();
 
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaToDevTime.addAndGet(System.nanoTime() - start);
+			GPUStatistics.cudaToDevTime.add(System.nanoTime() - start);
 		if (DMLScript.STATISTICS)
-			GPUStatistics.cudaToDevCount.addAndGet(1);
+			GPUStatistics.cudaToDevCount.add(1);
 	}
 
 	public static int toIntExact(long l) throws DMLRuntimeException {
@@ -882,9 +880,9 @@ public class GPUObject {
 			mat.release();
 
 			if (DMLScript.STATISTICS)
-				GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime() - start);
+				GPUStatistics.cudaFromDevTime.add(System.nanoTime() - start);
 			if (DMLScript.STATISTICS)
-				GPUStatistics.cudaFromDevCount.addAndGet(1);
+				GPUStatistics.cudaFromDevCount.add(1);
 		} else if (getJcudaSparseMatrixPtr() != null) {
 			if (!LibMatrixCUDA.isInSparseFormat(getGPUContext(), mat))
 				throw new DMLRuntimeException(
@@ -912,9 +910,9 @@ public class GPUObject {
 				mat.acquireModify(tmp);
 				mat.release();
 				if (DMLScript.STATISTICS)
-					GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime() - start);
+					GPUStatistics.cudaFromDevTime.add(System.nanoTime() - start);
 				if (DMLScript.STATISTICS)
-					GPUStatistics.cudaFromDevCount.addAndGet(1);
+					GPUStatistics.cudaFromDevCount.add(1);
 			}
 		} else {
 			throw new DMLRuntimeException(