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 2018/09/19 16:23:35 UTC

systemml git commit: [SYSTEMML-445] Dynamically decide whether to perform float-to-double conversion in the single precision mode on the host or device

Repository: systemml
Updated Branches:
  refs/heads/master 61139e400 -> 3fbfbaecb


[SYSTEMML-445] Dynamically decide whether to perform float-to-double conversion in the single precision mode on the host or device

- Fixed a int-to-long conversion bug in the shadow buffer.
- Updated javadocs for GPULazyCudaFreeMemoryManager.


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

Branch: refs/heads/master
Commit: 3fbfbaecb9d1e31341df8084ff28035bede47766
Parents: 61139e4
Author: Niketan Pansare <np...@us.ibm.com>
Authored: Wed Sep 19 09:19:30 2018 -0700
Committer: Niketan Pansare <np...@us.ibm.com>
Committed: Wed Sep 19 09:19:30 2018 -0700

----------------------------------------------------------------------
 .../context/GPULazyCudaFreeMemoryManager.java   | 51 +++++++++++++++++++-
 .../gpu/context/GPUMemoryManager.java           |  4 ++
 .../instructions/gpu/context/ShadowBuffer.java  | 18 +++++--
 .../matrix/data/CudaSupportFunctions.java       |  1 -
 .../SinglePrecisionCudaSupportFunctions.java    | 37 +++++++++-----
 .../apache/sysml/utils/PersistentLRUCache.java  | 15 +++---
 6 files changed, 100 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/3fbfbaec/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java
index ba98b3f..db21ae3 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java
@@ -45,6 +45,7 @@ public class GPULazyCudaFreeMemoryManager {
 	 */
 	private HashMap<Long, Set<Pointer>> rmvarGPUPointers = new HashMap<Long, Set<Pointer>>();
 	
+	
 	/**
 	 * Get any pointer of the given size from rmvar-ed pointers (applicable if eager cudaFree is set to false)
 	 * 
@@ -85,10 +86,17 @@ public class GPULazyCudaFreeMemoryManager {
 			GPUStatistics.maintainCPMiscTimes(opcode, instructionLevelTimer, System.nanoTime() - startTime);
 	}
 	
+	/**
+	 * 
+	 * @return set of all pointers managed by this memory manager.
+	 */
 	public Set<Pointer> getAllPointers() {
 		return rmvarGPUPointers.values().stream().flatMap(ptrs -> ptrs.stream()).collect(Collectors.toSet());
 	}
 	
+	/**
+	 * Frees up all the cached rmvar-ed pointers
+	 */
 	public void clearAll() {
 		Set<Pointer> toFree = new HashSet<Pointer>();
 		for(Set<Pointer> ptrs : rmvarGPUPointers.values()) {
@@ -100,9 +108,16 @@ public class GPULazyCudaFreeMemoryManager {
 		}
 	}
 	
+	/**
+	 * Helper method to get the rmvar pointer that is greater than equal to min size
+	 * 
+	 * @param opcode instruction name
+	 * @param minSize size in bytes
+	 * @return the rmvar pointer that is greater than equal to min size
+	 * @throws DMLRuntimeException if error
+	 */
 	public Pointer getRmvarPointerMinSize(String opcode, long minSize) throws DMLRuntimeException {
-		Optional<Long> toClear = rmvarGPUPointers.entrySet().stream().filter(e -> e.getValue().size() > 0).map(e -> e.getKey())
-				.filter(size -> size >= minSize).min((s1, s2) -> s1 < s2 ? -1 : 1);
+		Optional<Long> toClear = getRmvarSize(minSize);
 		if(toClear.isPresent()) {
 			boolean measureTime = opcode != null && ConfigurationManager.isFinegrainedStatistics();
 			long t0 = measureTime ?  System.nanoTime() : 0;
@@ -118,6 +133,38 @@ public class GPULazyCudaFreeMemoryManager {
 		return null;
 	}
 	
+	/**
+	 * Helper method to check if the lazy memory manager contains a pointer of the given size
+	 * 
+	 * @param opcode instruction name
+	 * @param size size in bytes
+	 * @return true if the lazy memory manager contains a pointer of the given size
+	 */
+	boolean contains(String opcode, long size) {
+		return rmvarGPUPointers.containsKey(size);
+	}
+	
+	/**
+	 * Helper method to check if the lazy memory manager contains a pointer >= minSize
+	 * 
+	 * @param opcode instruction name
+	 * @param minSize size in bytes
+	 * @return true if the lazy memory manager contains a pointer >= minSize
+	 */
+	boolean containsRmvarPointerMinSize(String opcode, long minSize) {
+		return getRmvarSize(minSize).isPresent();
+	}
+	
+	/**
+	 * Helper method to get the size of rmvar pointer that is greater than equal to min size
+	 *  
+	 * @param minSize size in bytes
+	 * @return size of rmvar pointer that is >= minSize
+	 */
+	private Optional<Long> getRmvarSize(long minSize) {
+		return rmvarGPUPointers.entrySet().stream().filter(e -> e.getValue().size() > 0).map(e -> e.getKey())
+				.filter(size -> size >= minSize).min((s1, s2) -> s1 < s2 ? -1 : 1);
+	}
 	
 	/**
 	 * Remove any pointer in the given hashmap

http://git-wip-us.apache.org/repos/asf/systemml/blob/3fbfbaec/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java
index 509aafe..033051a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java
@@ -241,6 +241,10 @@ public class GPUMemoryManager {
 	    }
 	}
 	
+	public boolean canAllocateWithoutEviction(String opcode, long size) {
+		return lazyCudaFreeMemoryManager.contains(opcode, size) || allocator.canAllocate(size) ||
+			lazyCudaFreeMemoryManager.containsRmvarPointerMinSize(opcode, size) ;
+	}
 	
 	/**
 	 * Allocate pointer of the given size in bytes.

http://git-wip-us.apache.org/repos/asf/systemml/blob/3fbfbaec/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java
index e122de2..a36e036 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java
@@ -74,8 +74,8 @@ public class ShadowBuffer {
 		long start = ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
 		int numElems = GPUObject.toIntExact(gpuObj.mat.getNumRows()*gpuObj.mat.getNumColumns());
 		shadowPointer = new float[numElems];
-		EVICTION_SHADOW_BUFFER_CURR_BYTES += shadowPointer.length*Sizeof.FLOAT;
-		cudaMemcpy(Pointer.to(shadowPointer), gpuObj.jcudaDenseMatrixPtr, numElems*LibMatrixCUDA.sizeOfDataType, jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost);
+		EVICTION_SHADOW_BUFFER_CURR_BYTES += getSizeOfFloat(shadowPointer.length);
+		cudaMemcpy(Pointer.to(shadowPointer), gpuObj.jcudaDenseMatrixPtr, getSizeOfDataType(numElems), jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost);
 		gpuObj.getGPUContext().cudaFreeHelper(instName, gpuObj.jcudaDenseMatrixPtr, true);
 		gpuObj.jcudaDenseMatrixPtr = null;
 		if (ConfigurationManager.isStatistics()) {
@@ -87,6 +87,14 @@ public class ShadowBuffer {
 		}
 	}
 	
+	private long getSizeOfFloat(long numElems) {
+		return numElems*Sizeof.FLOAT;
+	}
+	
+	private long getSizeOfDataType(long numElems) {
+		return numElems*LibMatrixCUDA.sizeOfDataType;
+	}
+	
 	/**
 	 * Move the data from shadow buffer to Matrix object
 	 */
@@ -117,7 +125,7 @@ public class ShadowBuffer {
 	 */
 	public void moveToDevice() {
 		long start = ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
-		long numBytes = shadowPointer.length*LibMatrixCUDA.sizeOfDataType;
+		long numBytes = getSizeOfDataType(shadowPointer.length);
 		gpuObj.jcudaDenseMatrixPtr = gpuObj.getGPUContext().allocate(null, numBytes);
 		cudaMemcpy(gpuObj.jcudaDenseMatrixPtr, Pointer.to(shadowPointer), numBytes, jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice);
 		clearShadowPointer();
@@ -137,7 +145,7 @@ public class ShadowBuffer {
 	 */
 	public boolean isEligibleForBuffering(boolean isEviction, boolean eagerDelete) {
 		if(LibMatrixCUDA.sizeOfDataType == jcuda.Sizeof.FLOAT && isEviction && eagerDelete && !gpuObj.isDensePointerNull()) {
-			int numBytes = GPUObject.toIntExact(gpuObj.mat.getNumRows()*gpuObj.mat.getNumColumns())*Sizeof.FLOAT;
+			long numBytes = getSizeOfFloat(gpuObj.mat.getNumRows()*gpuObj.mat.getNumColumns());
 			boolean ret = EVICTION_SHADOW_BUFFER_CURR_BYTES + numBytes <= EVICTION_SHADOW_BUFFER_MAX_BYTES;
 			if(!ret && !_warnedAboutShadowBuffer) {
 				LOG.warn("Shadow buffer is full, so using CP bufferpool instead. Consider increasing sysml.gpu.eviction.shadow.bufferSize.");
@@ -155,7 +163,7 @@ public class ShadowBuffer {
 	 */
 	public void clearShadowPointer() {
 		if(shadowPointer != null) {
-			EVICTION_SHADOW_BUFFER_CURR_BYTES -= shadowPointer.length*Sizeof.FLOAT;
+			EVICTION_SHADOW_BUFFER_CURR_BYTES -= getSizeOfFloat(shadowPointer.length);
 		}
 		shadowPointer = null;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/3fbfbaec/src/main/java/org/apache/sysml/runtime/matrix/data/CudaSupportFunctions.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/CudaSupportFunctions.java b/src/main/java/org/apache/sysml/runtime/matrix/data/CudaSupportFunctions.java
index 80e9bec..78793eb 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/CudaSupportFunctions.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/CudaSupportFunctions.java
@@ -55,7 +55,6 @@ import jcuda.Pointer;
  * 3. During SystemML initialization, the appropriate class implementing CudaKernels interface is set based on the configuration property sysml.dataType.
  */
 public interface CudaSupportFunctions {
-	public static boolean PERFORM_CONVERSION_ON_DEVICE = true;
 	public int cusparsecsrgemm(cusparseHandle handle, int transA, int transB, int m, int n, int k, 
 			cusparseMatDescr descrA, int nnzA, Pointer csrValA, Pointer csrRowPtrA, Pointer csrColIndA, 
 			cusparseMatDescr descrB, int nnzB, Pointer csrValB, Pointer csrRowPtrB, Pointer csrColIndB, 

http://git-wip-us.apache.org/repos/asf/systemml/blob/3fbfbaec/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java b/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java
index 044e943..dd23871 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java
@@ -178,17 +178,19 @@ public class SinglePrecisionCudaSupportFunctions implements CudaSupportFunctions
 		// However, the conversion requires an additional space to be allocated for the conversion, which can lead to infinite recursion 
 		// during eviction: `evict -> devictToHost -> float2double -> allocate -> ensureFreeSpace -> evict`. 
 		// To avoid this recursion, it is necessary to perform this conversion in host.
-		if(PERFORM_CONVERSION_ON_DEVICE && !isEviction) {
-			Pointer deviceDoubleData = gCtx.allocate(instName, ((long)dest.length)*Sizeof.DOUBLE);
+		if(gCtx.getMemoryManager().canAllocateWithoutEviction(instName, sizeOfDouble(dest.length)) && !isEviction) {
+			Pointer deviceDoubleData = gCtx.allocate(instName, sizeOfDouble(dest.length));
 			LibMatrixCUDA.float2double(gCtx, src, deviceDoubleData, dest.length);
-			cudaMemcpy(Pointer.to(dest), deviceDoubleData, ((long)dest.length)*Sizeof.DOUBLE, cudaMemcpyDeviceToHost);
+			cudaMemcpy(Pointer.to(dest), deviceDoubleData, sizeOfDouble(dest.length), cudaMemcpyDeviceToHost);
 			gCtx.cudaFreeHelper(instName, deviceDoubleData, gCtx.EAGER_CUDA_FREE);
 		}
 		else {
 			LOG.debug("Potential OOM: Allocated additional space on host in deviceToHost");
-			FloatBuffer floatData = ByteBuffer.allocateDirect(Sizeof.FLOAT*dest.length).order(ByteOrder.nativeOrder()).asFloatBuffer();
-			cudaMemcpy(Pointer.to(floatData), src, ((long)dest.length)*Sizeof.FLOAT, cudaMemcpyDeviceToHost);
-			LibMatrixNative.fromFloatBuffer(floatData, dest);
+			float[] floatData = new float[dest.length];
+			cudaMemcpy(Pointer.to(floatData), src, sizeOfFloat(dest.length), cudaMemcpyDeviceToHost);
+			for(int i = 0; i < dest.length; i++) {
+				dest[i] = floatData[i];
+			}
 		}
 		if(ConfigurationManager.isStatistics()) {
 			long totalTime = System.nanoTime() - t0;
@@ -204,16 +206,19 @@ public class SinglePrecisionCudaSupportFunctions implements CudaSupportFunctions
 		LOG.debug("Potential OOM: Allocated additional space in hostToDevice");
 		// TODO: Perform conversion on GPU using double2float and float2double kernels
 		long t0 = ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
-		if(PERFORM_CONVERSION_ON_DEVICE) {
-			Pointer deviceDoubleData = gCtx.allocate(instName, ((long)src.length)*Sizeof.DOUBLE);
-			cudaMemcpy(deviceDoubleData, Pointer.to(src), ((long)src.length)*Sizeof.DOUBLE, cudaMemcpyHostToDevice);
+		if(gCtx.getMemoryManager().canAllocateWithoutEviction(instName, sizeOfDouble(src.length))) {
+			Pointer deviceDoubleData = gCtx.allocate(instName, sizeOfDouble(src.length));
+			cudaMemcpy(deviceDoubleData, Pointer.to(src), sizeOfDouble(src.length), cudaMemcpyHostToDevice);
 			LibMatrixCUDA.double2float(gCtx, deviceDoubleData, dest, src.length);
 			gCtx.cudaFreeHelper(instName, deviceDoubleData, gCtx.EAGER_CUDA_FREE);
 		}
 		else {
-			FloatBuffer floatData = ByteBuffer.allocateDirect(Sizeof.FLOAT*src.length).order(ByteOrder.nativeOrder()).asFloatBuffer();
-			IntStream.range(0, src.length).parallel().forEach(i -> floatData.put(i, (float)src[i]));
-			cudaMemcpy(dest, Pointer.to(floatData), ((long)src.length)*Sizeof.FLOAT, cudaMemcpyHostToDevice);
+			LOG.debug("Potential OOM: Allocated additional space on host in hostToDevice");
+			float[] floatData = new float[src.length];
+			for(int i = 0; i < src.length; i++) {
+				floatData[i] = (float) src[i];
+			}
+			cudaMemcpy(dest, Pointer.to(floatData), sizeOfFloat(src.length), cudaMemcpyHostToDevice);
 		}
 		
 		if(ConfigurationManager.isStatistics()) {
@@ -224,4 +229,12 @@ public class SinglePrecisionCudaSupportFunctions implements CudaSupportFunctions
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_HOST_TO_DEVICE, totalTime);
 		}
 	}
+	
+	private long sizeOfFloat(long numElems) {
+		return Sizeof.FLOAT*numElems;
+	}
+	
+	private long sizeOfDouble(long numElems) {
+		return Sizeof.DOUBLE*numElems;
+	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/3fbfbaec/src/main/java/org/apache/sysml/utils/PersistentLRUCache.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/PersistentLRUCache.java b/src/main/java/org/apache/sysml/utils/PersistentLRUCache.java
index 22f74c6..bf356bb 100644
--- a/src/main/java/org/apache/sysml/utils/PersistentLRUCache.java
+++ b/src/main/java/org/apache/sysml/utils/PersistentLRUCache.java
@@ -78,7 +78,7 @@ import org.apache.sysml.runtime.util.FastBufferedDataOutputStream;
  * This class does not assume minimum capacity and hence only soft references.
  * 
  * To test this class, please use the below command:
- * java -cp systemml-*-standalone.jar:commons-lang3-3.8.jar org.apache.sysml.utils.PersistentLRUCache.
+ * java -cp systemml-1.3.0-SNAPSHOT-standalone.jar:commons-lang3-3.8.jar org.apache.sysml.utils.PersistentLRUCache
  */
 public class PersistentLRUCache extends LinkedHashMap<String, ValueWrapper> {
 	static final Log LOG = LogFactory.getLog(PersistentLRUCache.class.getName());
@@ -93,11 +93,14 @@ public class PersistentLRUCache extends LinkedHashMap<String, ValueWrapper> {
 	public static void main(String [] args) throws IOException {
 		org.apache.log4j.Logger.getRootLogger().setLevel(Level.DEBUG);
 		double numBytesInMB = 1e+7;
-		int numDoubleInMB = (int) (numBytesInMB / 8);
-		PersistentLRUCache cache = new PersistentLRUCache((long)(numBytesInMB*25));
-		for(int i = 0; i < 30; ++i) {
-			LOG.debug("Putting a double array of size 1MB.");
-			cache.put("file_" + i, new double[numDoubleInMB]);
+		int numDoubleIn50MB = (int) (50.0*numBytesInMB / 8.0);
+		long maxMemory = Runtime.getRuntime().maxMemory();
+		double multiplier = 2.0; // 0.3; // Use value > 1 to test GC and < 1 to test max capacity
+		PersistentLRUCache cache = new PersistentLRUCache((long)(maxMemory*multiplier));
+		long numIter = (long) ((3.0*maxMemory) / numBytesInMB);
+		for(long i = 0; i < numIter; ++i) {
+			LOG.debug("Putting a double array of size 50MB.");
+			cache.put("file_" + i, new double[numDoubleIn50MB]);
 		}
 		cache.clear();
 	}