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/10/20 18:09:29 UTC

[1/2] systemml git commit: [SYSTEMML-445] Support non-CuDNN GPU operator for LSTM forward and backward

Repository: systemml
Updated Branches:
  refs/heads/master ef842da9c -> bd34292d4


http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java
index 4ad4155..0424114 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java
@@ -38,6 +38,15 @@ import org.apache.sysml.runtime.util.DnnUtils;
 import org.apache.sysml.utils.GPUStatistics;
 
 public class DnnGPUInstruction extends GPUInstruction {
+	
+	public static enum LstmOperator {
+		CUDNN,
+		DENSE_NN,
+		NONE
+	}
+	
+	public static LstmOperator FORCED_LSTM_OP = LstmOperator.NONE;
+	
 	private CPOperand _input1;
 	private CPOperand _input2;
 	private CPOperand _input3;
@@ -638,43 +647,36 @@ public class DnnGPUInstruction extends GPUInstruction {
 		return (int)num;
 	}
 	
+	public static long getMemRequiredForCuDNNLSTMBackward(long N, long T, long M, long D, boolean return_sequences) {
+		double memRequired = (D+M)*4*M // sysmlWPointer
+				+ 2*(D+M+2)*(4*M) // cudnnWPointer and cudnnDwPointer
+				+ 3*N*T*D  // cudnnInput, cudnnDx and smlDx
+				+ 2*N*T*M // dy and yPointer
+				+ (return_sequences ? T*M : M); // dout
+		memRequired *= LibMatrixCUDA.sizeOfDataType;
+		// Assume the workspace to be proportional to cudnnWPointer (add 20% additional overhead for workspace)
+		memRequired += 1.2*(D+M+2)*(4*M)*LibMatrixCUDA.sizeOfDataType;
+		return (long)memRequired;
+	}
+	
 	private void processLstmBackwardInstruction(ExecutionContext ec) throws DMLRuntimeException {
 		MatrixObject out0 = getMatrixInputForGPUInstruction(ec, _input4.getName());
 		long M = out0.getNumColumns(); // hiddenSize .. since out0: (N, M)
+		long N1 = out0.getNumRows();
 		Pointer out0Pointer =  LibMatrixCUDA.getDensePointer(gCtx, out0, instName);
 		
 		MatrixObject W = getMatrixInputForGPUInstruction(ec, _input2.getName());
 		MatrixObject bias = getMatrixInputForGPUInstruction(ec, _input3.getName());
 		long numRowsW = W.getNumRows();
-		long D = numRowsW - M; // since W:(D+M, 4M) ... numFeatures 
-		Pointer sysmlWPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, W, instName, D+M, 4*M);
-		Pointer sysmlBiasPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, bias, instName, 1, 4*M);
-		Pointer cudnnWPointer = gCtx.allocate(instName, (D+M+2)*(4*M)*LibMatrixCUDA.sizeOfDataType);
-		LibMatrixCUDA.getCudaKernels(gCtx).launchKernel("prepare_lstm_weight",
-				ExecutionConfig.getConfigForSimpleVectorOperations(toInt((D+M+2)*(4*M))),
-				sysmlWPointer, sysmlBiasPointer, cudnnWPointer, D, M);
-		ec.releaseMatrixInputForGPUInstruction(_input2.getName());
-		ec.releaseMatrixInputForGPUInstruction(_input3.getName());
-		
-		
+		long D = numRowsW - M; // since W:(D+M, 4M) ... numFeatures
 		MatrixObject X = getMatrixInputForGPUInstruction(ec, _input1.getName());
-		Pointer xPointer = LibMatrixCUDA.getDensePointer(gCtx, X, instName); 
 		int N = toInt(X.getNumRows()); // batchSize .. since X:(N, T*D)
 		long numColsX = X.getNumColumns();
 		int T = toInt(numColsX/ D); // since X:(N, T*D) ... seqLength
-		Pointer cudnnInput = gCtx.allocate(instName, (N*T*D)*LibMatrixCUDA.sizeOfDataType);
-		LibMatrixCUDA.getCudaKernels(gCtx).launchKernel("prepare_lstm_input",
-				ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*T*D)),
-				xPointer, cudnnInput, N, D, T*D, N*T*D);
-		ec.releaseMatrixInputForGPUInstruction(_input1.getName());
-		
-		Pointer c0Pointer = LibMatrixCUDA.getDensePointer(gCtx, getMatrixInputForGPUInstruction(ec, _input5.getName()), instName);
 		boolean return_sequences = ec.getScalarInput(_input6.getName(), _input6.getValueType(), _input6.isLiteral()).getBooleanValue();
 		
-		// LibMatrixCuDNN.lstm(ec, gCtx, instName, 
-				// cudnnInput, cudnnWPointer, out0Pointer, c0Pointer, return_sequences, _output.getName(), _output2.getName(), N, M, D, T);
-				// String xName, Pointer hx, Pointer cx, Pointer wPointer, String doutName, String dcyName,  // input
-				// String dxName, String dwName, String dbName, String dhxName, String dcxName,  	// output
+		// long memRequired = getMemRequiredForCuDNNLSTMBackward(N, T, M, D, return_sequences);
+		 
 		String dxName = _output.getName();
 		String dwName = _output2.getName();
 		String dbName = _output3.getName();
@@ -682,12 +684,95 @@ public class DnnGPUInstruction extends GPUInstruction {
 		String dcxName = _output5.getName();
 		String doutName = _input7.getName();
 		String dcyName = _input8.getName();
-		LibMatrixCuDNN.lstmBackward(ec, gCtx, instName, 
-				cudnnInput, out0Pointer, c0Pointer, cudnnWPointer, doutName, dcyName,  // input
-				dxName, dwName, dbName, dhxName, dcxName, // output 
-				return_sequences, N, M, D, T);
-		gCtx.cudaFreeHelper(instName, cudnnWPointer, gCtx.EAGER_CUDA_FREE);
-		gCtx.cudaFreeHelper(instName, cudnnInput, gCtx.EAGER_CUDA_FREE);
+		
+		long memRequired = getMemRequiredForCuDNNLSTMBackward(N, T, M, D, return_sequences);
+		
+		boolean isWSparse = LibMatrixCUDA.isInSparseFormat(gCtx, W);
+		
+		
+		
+		if(FORCED_LSTM_OP == LstmOperator.CUDNN || 
+			N != N1 || // Use CuDNN operator when batch size of previous iteration is different that current iteration
+			(!isWSparse && // Don't use CuDNN kernel when w is sparse.
+			// When an operator is not forced, then prefer CuDNN kernel if it can fit in the GPU memory
+			FORCED_LSTM_OP == LstmOperator.NONE && gCtx.getMemoryManager().canAllocate(instName, memRequired))) {
+			// Use CuDNN LSTM kernel
+			Pointer sysmlWPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, W, instName, D+M, 4*M);
+			Pointer sysmlBiasPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, bias, instName, 1, 4*M);
+			Pointer cudnnWPointer = gCtx.allocate(instName, (D+M+2)*(4*M)*LibMatrixCUDA.sizeOfDataType);
+			LibMatrixCUDA.getCudaKernels(gCtx).launchKernel("prepare_lstm_weight",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt((D+M+2)*(4*M))),
+					sysmlWPointer, sysmlBiasPointer, cudnnWPointer, D, M);
+			ec.releaseMatrixInputForGPUInstruction(_input2.getName());
+			ec.releaseMatrixInputForGPUInstruction(_input3.getName());
+			Pointer xPointer = LibMatrixCUDA.getDensePointer(gCtx, X, instName); 
+			Pointer cudnnInput = gCtx.allocate(instName, (N*T*D)*LibMatrixCUDA.sizeOfDataType);
+			LibMatrixCUDA.getCudaKernels(gCtx).launchKernel("prepare_lstm_input",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*T*D)),
+					xPointer, cudnnInput, N, D, T*D, N*T*D);
+			ec.releaseMatrixInputForGPUInstruction(_input1.getName());
+			Pointer c0Pointer = LibMatrixCUDA.getDensePointer(gCtx, getMatrixInputForGPUInstruction(ec, _input5.getName()), instName);
+			LibMatrixCuDNN.cuDNNLstmBackward(ec, gCtx, instName, 
+					cudnnInput, out0Pointer, c0Pointer, cudnnWPointer, doutName, dcyName,  // input
+					dxName, dwName, dbName, dhxName, dcxName, // output 
+					return_sequences, N, M, D, T);
+			gCtx.cudaFreeHelper(instName, cudnnWPointer, gCtx.EAGER_CUDA_FREE);
+			gCtx.cudaFreeHelper(instName, cudnnInput, gCtx.EAGER_CUDA_FREE);
+		}
+		else {
+			if(N != N1) {
+				throw new DMLRuntimeException("Unsupported operation: The batch size of previous iteration " + N1 + 
+						" is different than the batch size of current iteration " + N);
+			}
+			
+			Pointer sysmlBiasPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, bias, instName, 1, 4*M);
+			Pointer xPointer = LibMatrixCUDA.getDensePointer(gCtx, X, instName); 
+			Pointer c0Pointer = LibMatrixCUDA.getDensePointer(gCtx, getMatrixInputForGPUInstruction(ec, _input5.getName()), instName);
+			
+			Pointer doutPointer = LibMatrixCuDNN.getDenseInputPointer(ec, gCtx, instName, doutName, N, return_sequences ? T*M : M);
+			Pointer dcyPointer = LibMatrixCuDNN.getDenseInputPointer(ec, gCtx, instName, dcyName, N, M);
+			
+			Pointer dxPointer = LibMatrixCuDNN.getDenseOutputPointer(ec, gCtx, instName, dxName, N, T*D);
+			Pointer dwPointer = LibMatrixCuDNN.getDenseOutputPointer(ec, gCtx, instName, dwName, D+M, 4*M);
+			Pointer dbPointer = LibMatrixCuDNN.getDenseOutputPointer(ec, gCtx, instName, dbName, 1, 4*M);
+			Pointer dhxPointer = LibMatrixCuDNN.getDenseOutputPointer(ec, gCtx, instName, dhxName, N, M);
+			Pointer dcxPointer = LibMatrixCuDNN.getDenseOutputPointer(ec, gCtx, instName, dcxName, N, M);
+			
+			// Donot skip cache as it is required in the backward pass
+			Pointer cache_out = gCtx.allocate(instName, T*N*M*LibMatrixCUDA.sizeOfDataType);
+			Pointer cache_c = gCtx.allocate(instName, T*N*M*LibMatrixCUDA.sizeOfDataType);
+			Pointer cache_ifog = gCtx.allocate(instName, T*N*4*M*LibMatrixCUDA.sizeOfDataType);
+			
+			Pointer cyPointer = gCtx.allocate(instName, N*M*LibMatrixCUDA.sizeOfDataType);
+			Pointer sysmlYPointer = gCtx.allocate(instName, (return_sequences ? N*(T*M) : N*M)*LibMatrixCUDA.sizeOfDataType);
+			LibMatrixCuDNN.nnLstm(ec, gCtx, instName, xPointer, W, sysmlBiasPointer, out0Pointer, 
+					c0Pointer, return_sequences, sysmlYPointer, cyPointer, 
+					cache_out, cache_c, cache_ifog, 
+					N, M,  D, T);
+			gCtx.cudaFreeHelper(instName, sysmlYPointer, gCtx.EAGER_CUDA_FREE);
+			gCtx.cudaFreeHelper(instName, cyPointer, gCtx.EAGER_CUDA_FREE);
+			
+			LibMatrixCuDNN.nnLstmBackward(ec, gCtx, instName,
+					xPointer, out0Pointer, c0Pointer, W, doutPointer, dcyPointer,  // input
+					cache_out, cache_c, cache_ifog,
+					dxPointer, dwPointer, dbPointer, dhxPointer, dcxPointer,  	// output
+					return_sequences, N, M, D, T);
+			
+			gCtx.cudaFreeHelper(instName, cache_out, gCtx.EAGER_CUDA_FREE);
+			gCtx.cudaFreeHelper(instName, cache_c, gCtx.EAGER_CUDA_FREE);
+			gCtx.cudaFreeHelper(instName, cache_ifog, gCtx.EAGER_CUDA_FREE);
+			ec.releaseMatrixInputForGPUInstruction(_input1.getName());
+			ec.releaseMatrixInputForGPUInstruction(_input2.getName()); // W
+			ec.releaseMatrixInputForGPUInstruction(_input3.getName()); // bias
+			ec.releaseMatrixInputForGPUInstruction(doutName);
+			ec.releaseMatrixInputForGPUInstruction(dcyName);
+			ec.releaseMatrixOutputForGPUInstruction(dxName);
+			ec.releaseMatrixOutputForGPUInstruction(dwName);
+			ec.releaseMatrixOutputForGPUInstruction(dbName);
+			ec.releaseMatrixOutputForGPUInstruction(dhxName);
+			ec.releaseMatrixOutputForGPUInstruction(dcxName);
+			
+		}
 		
 		// release inputs/outputs
 		ec.releaseMatrixInputForGPUInstruction(_input4.getName());
@@ -702,42 +787,79 @@ public class DnnGPUInstruction extends GPUInstruction {
 		// out: (N, T*M) or (N, M) ==> (T, M, N)
 		MatrixObject out0 = getMatrixInputForGPUInstruction(ec, _input4.getName());
 		long M = out0.getNumColumns(); // hiddenSize .. since out0: (N, M)
+		long N1 = out0.getNumRows();
 		Pointer out0Pointer =  LibMatrixCUDA.getDensePointer(gCtx, out0, instName);
 		
 		MatrixObject W = getMatrixInputForGPUInstruction(ec, _input2.getName());
 		MatrixObject bias = getMatrixInputForGPUInstruction(ec, _input3.getName());
 		long numRowsW = W.getNumRows();
 		long D = numRowsW - M; // since W:(D+M, 4M) ... numFeatures
-		
-		Pointer sysmlWPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, W, instName, D+M, 4*M);
-		Pointer sysmlBiasPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, bias, instName, 1, 4*M);
-		Pointer cudnnWPointer = gCtx.allocate(instName, (D+M+2)*(4*M)*LibMatrixCUDA.sizeOfDataType);
-		LibMatrixCUDA.getCudaKernels(gCtx).launchKernel("prepare_lstm_weight",
-				ExecutionConfig.getConfigForSimpleVectorOperations(toInt((D+M+2)*(4*M))),
-				sysmlWPointer, sysmlBiasPointer, cudnnWPointer, D, M);
-		ec.releaseMatrixInputForGPUInstruction(_input2.getName());
-		ec.releaseMatrixInputForGPUInstruction(_input3.getName());
-		
-		boolean return_sequences = ec.getScalarInput(_input6.getName(), _input6.getValueType(), _input6.isLiteral()).getBooleanValue();
-		
-		// Beause the matrices are released immediately, the output for transpose need not be taken into account
 		MatrixObject X = getMatrixInputForGPUInstruction(ec, _input1.getName());
-		Pointer xPointer = LibMatrixCUDA.getDensePointer(gCtx, X, instName); 
-		int N = toInt(X.getNumRows()); // batchSize .. since X:(N, T*D)
+		long N = X.getNumRows(); // batchSize .. since X:(N, T*D)
 		long numColsX = X.getNumColumns();
-		int T = toInt(numColsX/ D); // since X:(N, T*D) ... seqLength
-		Pointer cudnnInput = gCtx.allocate(instName, (N*T*D)*LibMatrixCUDA.sizeOfDataType);
-		LibMatrixCUDA.getCudaKernels(gCtx).launchKernel("prepare_lstm_input",
-				ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*T*D)),
-				xPointer, cudnnInput, N, D, T*D, N*T*D);
-		ec.releaseMatrixInputForGPUInstruction(_input1.getName());
+		long T = numColsX/D; // since X:(N, T*D) ... seqLength
+		boolean return_sequences = ec.getScalarInput(_input6.getName(), _input6.getValueType(), _input6.isLiteral()).getBooleanValue();
+		
+		long memRequired = getMemRequiredForCuDNNLSTMBackward(N, T, M, D, return_sequences);
 		
-		Pointer c0Pointer = LibMatrixCUDA.getDensePointer(gCtx, getMatrixInputForGPUInstruction(ec, _input5.getName()), instName); 
+		boolean isWSparse = LibMatrixCUDA.isInSparseFormat(gCtx, W);
 		
-		LibMatrixCuDNN.lstm(ec, gCtx, instName, cudnnInput, cudnnWPointer, out0Pointer, c0Pointer, return_sequences, _output.getName(), _output2.getName(), 
-				toInt(N), toInt(M), toInt(D), toInt(T));
-		gCtx.cudaFreeHelper(instName, cudnnWPointer, gCtx.EAGER_CUDA_FREE);
-		gCtx.cudaFreeHelper(instName, cudnnInput, gCtx.EAGER_CUDA_FREE);
+		if(FORCED_LSTM_OP == LstmOperator.CUDNN || 
+			N != N1 || // Use CuDNN operator when batch size of previous iteration is different that current iteration
+			(!isWSparse && // Don't use CuDNN kernel when w is sparse.
+			// When an operator is not forced, then prefer CuDNN kernel if it can fit in the GPU memory
+			FORCED_LSTM_OP == LstmOperator.NONE && gCtx.getMemoryManager().canAllocate(instName, memRequired))) {
+			Pointer sysmlWPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, W, instName, D+M, 4*M);
+			Pointer sysmlBiasPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, bias, instName, 1, 4*M);
+			Pointer cudnnWPointer = gCtx.allocate(instName, (D+M+2)*(4*M)*LibMatrixCUDA.sizeOfDataType);
+			LibMatrixCUDA.getCudaKernels(gCtx).launchKernel("prepare_lstm_weight",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt((D+M+2)*(4*M))),
+					sysmlWPointer, sysmlBiasPointer, cudnnWPointer, toInt(D), toInt(M));
+			ec.releaseMatrixInputForGPUInstruction(_input2.getName()); // W
+			ec.releaseMatrixInputForGPUInstruction(_input3.getName()); // bias
+			// Beause the matrices are released immediately, the output for transpose need not be taken into account
+			Pointer xPointer = LibMatrixCUDA.getDensePointer(gCtx, X, instName); 
+			Pointer cudnnInput = gCtx.allocate(instName, (N*T*D)*LibMatrixCUDA.sizeOfDataType);
+			LibMatrixCUDA.getCudaKernels(gCtx).launchKernel("prepare_lstm_input",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*T*D)),
+					xPointer, cudnnInput, toInt(N), toInt(D), toInt(T*D), toInt(N*T*D));
+			ec.releaseMatrixInputForGPUInstruction(_input1.getName());
+			Pointer c0Pointer = LibMatrixCUDA.getDensePointer(gCtx, getMatrixInputForGPUInstruction(ec, _input5.getName()), instName); 
+			LibMatrixCuDNN.cuDNNLstm(ec, gCtx, instName, cudnnInput, cudnnWPointer, out0Pointer, c0Pointer, return_sequences, _output.getName(), _output2.getName(), 
+					toInt(N), toInt(M), toInt(D), toInt(T));
+			gCtx.cudaFreeHelper(instName, cudnnWPointer, gCtx.EAGER_CUDA_FREE);
+			gCtx.cudaFreeHelper(instName, cudnnInput, gCtx.EAGER_CUDA_FREE);
+		}
+		else {
+			if(N != N1) {
+				throw new DMLRuntimeException("Unsupported operation: The batch size of previous iteration " + N1 + 
+						" is different than the batch size of current iteration " + N);
+			}
+			
+			Pointer sysmlBiasPointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, bias, instName, 1, 4*M);
+			Pointer xPointer = LibMatrixCUDA.getDensePointer(gCtx, X, instName); 
+			Pointer c0Pointer = LibMatrixCUDA.getDensePointer(gCtx, getMatrixInputForGPUInstruction(ec, _input5.getName()), instName);
+			Pointer sysmlYPointer = LibMatrixCuDNN.getDenseOutputPointer(ec, gCtx, instName, _output.getName(), N, 
+					return_sequences ? (T*M) : M);
+			Pointer cyPointer = LibMatrixCuDNN.getDenseOutputPointer(ec, gCtx, instName,  _output2.getName(), N, M);
+			
+			// Skip cache in forward for now. We can revisit this when we add stateful operators.
+			Pointer cache_out = null; // gCtx.allocate(instName, T*N*M*LibMatrixCUDA.sizeOfDataType);
+			Pointer cache_c = null;  // gCtx.allocate(instName, T*N*M*LibMatrixCUDA.sizeOfDataType);
+			Pointer cache_ifog = null; // gCtx.allocate(instName, T*N*4*M*LibMatrixCUDA.sizeOfDataType);
+			
+			LibMatrixCuDNN.nnLstm(ec, gCtx, instName, xPointer, W, sysmlBiasPointer, out0Pointer, 
+					c0Pointer, return_sequences, sysmlYPointer, cyPointer, 
+					cache_out, cache_c, cache_ifog, 
+					N, M,  D, T);
+			
+			// gCtx.cudaFreeHelper(instName, cache_out, gCtx.EAGER_CUDA_FREE);
+			// gCtx.cudaFreeHelper(instName, cache_c, gCtx.EAGER_CUDA_FREE);
+			// gCtx.cudaFreeHelper(instName, cache_ifog, gCtx.EAGER_CUDA_FREE);
+			ec.releaseMatrixInputForGPUInstruction(_input1.getName());
+			ec.releaseMatrixInputForGPUInstruction(_input2.getName()); // W
+			ec.releaseMatrixInputForGPUInstruction(_input3.getName()); // bias
+		}
 		
 		// release inputs/outputs
 		ec.releaseMatrixInputForGPUInstruction(_input4.getName());

http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/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 a08d4fd..6a04d97 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
@@ -224,6 +224,10 @@ public class GPUMemoryManager {
 			return "->" + stackTrace[index].getClassName() + "." + stackTrace[index].getMethodName() + "(" + stackTrace[index].getFileName() + ":" + stackTrace[index].getLineNumber() + ")";
 	}
 	
+	public boolean canAllocate(String opcode, long size) {
+		return allocator.canAllocate(size);
+	}
+	
 	
 	public boolean canAllocateWithoutEviction(String opcode, long size) {
 		return lazyCudaFreeMemoryManager.contains(opcode, size) || allocator.canAllocate(size) ||

http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java
index 00aa578..fd06578 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java
@@ -227,6 +227,23 @@ public class LibMatrixCUDA {
 				A, ret, numElems);
 		return ret;
 	}
+	
+	public static void printPointerForDebugging(Pointer ptr, int rows, int cols, String matName) {
+		if(sizeOfDataType == jcuda.Sizeof.DOUBLE) {
+			double[] devData = new double[rows*cols];
+			cudaMemcpy(Pointer.to(devData), ptr, rows*cols*sizeOfDataType, jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost);
+			System.out.println(matName + ":");
+			for(int i = 0; i < rows; i++) {
+				for(int j = 0; j < cols; j++) {
+					System.out.print(String.format("%.3f", devData[i*cols+j]) + " ");
+				}
+				System.out.println();
+			}
+		}
+		else {
+			throw new DMLRuntimeException("The method printPointerForDebugging is only supported for double precision.");
+		}
+	}
 
 	//********************************************************************/
 	//************************ End of UTILS ******************************/
@@ -1425,7 +1442,7 @@ public class LibMatrixCUDA {
 	 * @param isRightTransposed true if right matrix is transposed
 	 * @param op                operator
 	 */
-	private static void matrixMatrixOp(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2,
+	static void matrixMatrixOp(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2,
 			String outputName, boolean isLeftTransposed, boolean isRightTransposed, BinaryOperator op) {
 		if (ec.getGPUContext(0) != gCtx)
 			throw new DMLRuntimeException("GPU : Invalid internal state, the GPUContext set with the ExecutionContext is not the same used to run this LibMatrixCUDA function");
@@ -1502,7 +1519,7 @@ public class LibMatrixCUDA {
 	 * @param c						output matrix of size (maxRlen, maxClen) allocated on GPU
 	 * @param op					the operation to perform
 	 */
-	private static void matrixMatrixOp(GPUContext gCtx, String instName, Pointer a, Pointer b, int maxRlen, int maxClen, int vecStatusA, int vecStatusB, Pointer c, BinaryOperator op) {
+	static void matrixMatrixOp(GPUContext gCtx, String instName, Pointer a, Pointer b, int maxRlen, int maxClen, int vecStatusA, int vecStatusB, Pointer c, BinaryOperator op) {
 		if(LOG.isTraceEnabled()) {
 			LOG.trace("GPU : matrix_matrix_cellwise_op" + ", GPUContext=" + gCtx);
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java
index 8051cbc..413c550 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java
@@ -54,11 +54,14 @@ import org.apache.sysml.hops.OptimizerUtils;
 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.functionobjects.Plus;
 import org.apache.sysml.runtime.instructions.gpu.GPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.context.CSRPointer;
 import org.apache.sysml.runtime.instructions.gpu.context.ExecutionConfig;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
+import org.apache.sysml.runtime.matrix.data.LibMatrixCuMatMult.CuMatMultParameters;
 import org.apache.sysml.runtime.matrix.data.LibMatrixDNN.PoolingType;
+import org.apache.sysml.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysml.utils.GPUStatistics;
 import org.apache.sysml.utils.Statistics;
 
@@ -846,19 +849,231 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 		}
 	}
 	
-	static Pointer getDenseInputPointer(ExecutionContext ec, GPUContext gCtx, String instName, String inputName,
+	public static Pointer getDenseInputPointer(ExecutionContext ec, GPUContext gCtx, String instName, String inputName,
 			long numRows, long numCols) throws DMLRuntimeException {
 		MatrixObject output = ec.getMatrixInputForGPUInstruction(inputName, instName);
 		return LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, output, instName, numRows, numCols);
 	}
 	
-	static Pointer getDenseOutputPointer(ExecutionContext ec, GPUContext gCtx, String instName, String outputName,
+	public static Pointer getDenseOutputPointer(ExecutionContext ec, GPUContext gCtx, String instName, String outputName,
 			long numRows, long numCols) throws DMLRuntimeException {
 		MatrixObject output = ec.getMatrixObject(outputName);
 		getDenseMatrixOutputForGPUInstruction(ec, instName, outputName, numRows, numCols); // Allocated the dense output matrix
 		return getDensePointerForCuDNN(gCtx, output, instName, numRows, numCols);
 	}
 	
+	public static void nnLstmBackward(ExecutionContext ec, GPUContext gCtx, String instName,
+			Pointer X, Pointer out0, Pointer c0, MatrixObject W, Pointer dout, Pointer dc,  // input
+			Pointer cache_out, Pointer cache_c, Pointer cache_ifog,
+			Pointer dX, Pointer dW, Pointer db, Pointer dout0, Pointer dc0,  	// output
+			boolean return_sequences, long N, long M, long D, long T) throws DMLRuntimeException {
+		Pointer input = gCtx.allocate(instName, N*(D+M)*sizeOfDataType); 
+		Pointer difog_raw = gCtx.allocate(instName, N*4*M*sizeOfDataType);
+		Pointer dct = copy(gCtx, instName, dc, N*M);
+		Pointer dinput = gCtx.allocate(instName, N*(D+M)*sizeOfDataType); // (N, D+M)
+		Pointer tmpDb = gCtx.allocate(instName, 4*M*sizeOfDataType); // (1, 4M)
+		
+		// dW = dW + t(input) %*% difog_raw  # shape (D+M, 4M)
+		CuMatMultParameters param1 = new CuMatMultParameters(N, D+M,
+				N, 4*M, true, false, one(), one());
+		
+		// dinput = difog_raw %*% t(W)  # shape (N, D+M)
+		CuMatMultParameters param2 = new CuMatMultParameters(N, 4*M,
+				D+M, 4*M, false, true);
+		
+		CSRPointer wSparsePointer = null;
+		Pointer wDensePointer = null;
+		
+		// TODO: Only dense weight supported for now
+		boolean isWSparse = false; // isInSparseFormat(gCtx, W);
+		if(isWSparse)
+			wSparsePointer = W.getGPUObject(gCtx).getJcudaSparseMatrixPtr();
+		else
+			wDensePointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, W, instName, D+M, 4*M);
+		
+		Pointer dout_t = return_sequences ? gCtx.allocate(instName, N*M*sizeOfDataType) : copy(gCtx, instName, dout, N*M);
+		if(return_sequences) {
+			getCudaKernels(gCtx).launchKernel("initializeDoutWhenReturnSeq",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*M)),
+					dout, dout_t, T-1, toInt(M), toInt(T*M), toInt(N*M));
+		}
+		
+		for(int t = toInt(T); t >= 1; t--) {
+			// if (t == 1) { out_prev = out0; } else { out_prev = matrix(cache_out[t-1,], rows=N, cols=M) }
+			Pointer out_prev = (t == 1) ? out0 : cache_out.withByteOffset((t-2)*N*M*sizeOfDataType); // since read-only
+			
+			// X_t = X[,(t-1)*D+1:t*D]  # shape (N, D)
+			// input = cbind(X_t, out_prev)  # shape (N, D+M)
+			getCudaKernels(gCtx).launchKernel("prepareInputNNLstm",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*(D+M))),
+					X, out_prev, input, (t-1), toInt(M), toInt(D), toInt(T*D), toInt(D+M), toInt(N*(D+M)));
+			
+			// ct = matrix(cache_c[t,], rows=N, cols=M)  # shape (N, M)
+			Pointer ct = cache_c.withByteOffset((t-1)*N*M*sizeOfDataType); // since read-only
+			
+			// ifog = matrix(cache_ifog[t,], rows=N, cols=4*M)
+			Pointer ifog = cache_ifog.withByteOffset((t-1)*N*4*M*sizeOfDataType); // since read-only
+			
+			// i = ifog[,1:M]  # input gate, shape (N, M)
+			// f = ifog[,M+1:2*M]  # forget gate, shape (N, M)
+			// o = ifog[,2*M+1:3*M]  # output gate, shape (N, M)
+			// g = ifog[,3*M+1:4*M]  # g gate, shape (N, M)
+			// dct = dct + o*tanh::backward(dout_t, ct)  # shape (N, M)
+			// do = tanh::forward(ct) * dout_t  # output gate, shape (N, M)
+			// df = c_prev * dct  # forget gate, shape (N, M)
+			// dc_prev = f * dct  # shape (N, M)
+			// di = g * dct  # input gate, shape (N, M)
+			// dg = i * dct  # g gate, shape (N, M)
+			// di_raw = i * (1-i) * di
+			// df_raw = f * (1-f) * df
+			// do_raw = o * (1-o) * do
+			// dg_raw = (1-g^2) * dg
+			// difog_raw = cbind(di_raw, df_raw, do_raw, dg_raw)  # shape (N, 4M)
+			getCudaKernels(gCtx).launchKernel("computeDifog_raw",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*M)),
+					ifog, ct, dout_t, cache_c, c0, 
+					difog_raw, dct, dc0, // output
+					return_sequences ? 1 : 0, t-1, toInt(T), toInt(M), toInt(N*M));
+			
+			// dW = dW + t(input) %*% difog_raw  # shape (D+M, 4M)
+			LibMatrixCuMatMult.denseDenseMatMult(gCtx.getCublasHandle(), instName, dW, input, difog_raw, param1);
+			
+			// dinput = difog_raw %*% t(W)  # shape (N, D+M)
+			if(isWSparse) {
+				if(wSparsePointer.nnz == 0) {
+					cudaMemset(dinput, 0, N*(D+M)*sizeOfDataType);
+				}
+				else {
+					LibMatrixCuMatMult.denseSparseMatMult(gCtx.getCusparseHandle(), instName, dinput, difog_raw, wSparsePointer, param2);
+				}
+			}
+			else
+				LibMatrixCuMatMult.denseDenseMatMult(gCtx.getCublasHandle(), instName, dinput, difog_raw, wDensePointer, param2);
+			
+			// db = db + colSums(difog_raw)  # shape (1, 4M)
+			reduceCol(gCtx, instName, "reduce_col_sum", difog_raw, tmpDb, 1, toInt(4*M));
+			matrixMatrixOp(gCtx, instName, tmpDb, db, 1, toInt(4*M), VectorShape.NONE.code(), VectorShape.NONE.code(), db, 
+					new BinaryOperator(Plus.getPlusFnObject()));
+			
+			// jcuda.runtime.JCuda.cudaDeviceSynchronize();
+			
+			int size = toInt(Math.max(N*D, N*M));
+			getCudaKernels(gCtx).launchKernel("postProcessNNLstmBackward",
+					ExecutionConfig.getConfigForSimpleVectorOperations(size),
+					dinput, dout0, dout, dout_t, dX, return_sequences ? 1 : 0, t-1, N, D, M, 
+					toInt(N*D), toInt(N*M), toInt(T*D), toInt(T*M), toInt(D+M), size);
+			
+		}
+		
+		gCtx.cudaFreeHelper(instName, dout_t, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instName, input, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instName, difog_raw, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instName, dct, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instName, dinput, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instName, tmpDb, gCtx.EAGER_CUDA_FREE);
+		
+	}
+	
+	public static void nnLstm(ExecutionContext ec, GPUContext gCtx, String instName,
+			Pointer X,  MatrixObject W, Pointer b, Pointer out0, Pointer c0, boolean return_sequences,
+			Pointer out, Pointer c,  // output matrices
+			Pointer cache_out, Pointer cache_c, Pointer cache_ifog, // temporary workspace passed to the backward function
+			long N, long M, long D, long T) throws DMLRuntimeException {
+		boolean skipCache = cache_out == null || cache_c == null || cache_ifog == null;
+		
+		if( (skipCache && (cache_out != null || cache_c != null || cache_ifog != null)) || 
+			(!skipCache && (cache_out == null || cache_c == null || cache_ifog == null))) {
+			throw new DMLRuntimeException("Either all cache pointers should be null or all should be not null");
+		}
+		
+		// out_prev = out0
+		Pointer out_prev = copy(gCtx, instName, out0, N*M);
+		// c_prev = c0
+		Pointer c_prev = copy(gCtx, instName, c0, N*M);
+		// c = c_prev
+		cudaMemcpy(c, c_prev, N*M*sizeOfDataType, cudaMemcpyDeviceToDevice);
+		
+		Pointer input = gCtx.allocate(instName, N*(D+M)*sizeOfDataType);
+		Pointer ifog = gCtx.allocate(instName, N*4*M*sizeOfDataType);
+		
+		boolean isWSparse = isInSparseFormat(gCtx, W);
+		CSRPointer wSparsePointer = null;
+		Pointer wDensePointer = null;
+		if(isWSparse)
+			wSparsePointer = W.getGPUObject(gCtx).getJcudaSparseMatrixPtr();
+		else
+			wDensePointer = LibMatrixCuDNN.getDensePointerForCuDNN(gCtx, W, instName, D+M, 4*M);
+		
+		for(int t = 1; t <= T; t++) {
+			// X_t = X[,(t-1)*D+1:t*D]  # shape (N, D)
+			// input = cbind(X_t, out_prev)  # shape (N, D+M)
+			getCudaKernels(gCtx).launchKernel("prepareInputNNLstm",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*(D+M))),
+					X, out_prev, input, (t-1), toInt(M), toInt(D), toInt(T*D), toInt(D+M), toInt(N*(D+M)));
+			
+			// ifog = input %*% W
+			CuMatMultParameters param = new CuMatMultParameters(N, D+M,
+					D+M, 4*M, false, false);
+			if(isWSparse) {
+				if(wSparsePointer.nnz == 0) {
+					cudaMemset(ifog, 0, N*4*M*sizeOfDataType);
+				}
+				else {
+					LibMatrixCuMatMult.denseSparseMatMult(gCtx.getCusparseHandle(), instName, ifog, input, wSparsePointer, param);
+				}
+			}
+			else
+				LibMatrixCuMatMult.denseDenseMatMult(gCtx.getCublasHandle(), instName, ifog, input, wDensePointer, param);
+			
+			// ifog = ifog + b
+			// ifog[,1:3*M] = sigmoid::forward(ifog[,1:3*M])  # i,f,o gates squashed with sigmoid
+			// ifog[,3*M+1:4*M] = tanh::forward(ifog[,3*M+1:4*M])  # g gate squashed with tanh
+			getCudaKernels(gCtx).launchKernel("squashIFOG",
+					ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*4*M)),
+				ifog, b, toInt(M), toInt(N*4*M));
+			
+			
+			// c = ifog[,M+1:2*M]*c_prev + ifog[,1:M]*ifog[,3*M+1:4*M]
+			// out_t = ifog[,2*M+1:3*M] * tanh::forward(c)
+			// if (return_sequences) {
+			//   out[,(t-1)*M+1:t*M] = out_t
+			// }
+			// else {
+			//   out = out_t
+			// }
+			// out_prev = out_t
+			// c_prev = c
+			// cache_out[t,] = matrix(out_t, rows=1, cols=N*M)
+			// cache_c[t,] = matrix(c, rows=1, cols=N*M)
+			if(skipCache) {
+				getCudaKernels(gCtx).launchKernel("postProcessNNLstmForwardSkipCache",
+						ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*M)),
+					ifog, c,  out_prev, c_prev, out,
+					return_sequences ? 1 : 0, t-1, toInt(T), toInt(M), toInt(N*M));
+			}
+			else {
+				getCudaKernels(gCtx).launchKernel("postProcessNNLstmForward",
+						ExecutionConfig.getConfigForSimpleVectorOperations(toInt(N*M)),
+					ifog, c,  out_prev, c_prev, out, cache_out, cache_c,
+					return_sequences ? 1 : 0, t-1, toInt(T), toInt(M), toInt(N*M));
+				
+				// cache_ifog[t,] = matrix(ifog, rows=1, cols=N*4*M)  # reshape
+				cudaMemcpy(cache_ifog.withByteOffset((t-1)*N*4*M*sizeOfDataType), ifog, N*4*M*sizeOfDataType, cudaMemcpyDeviceToDevice);
+			}
+		}
+		
+		gCtx.cudaFreeHelper(instName, out_prev, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instName, c_prev, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instName, input, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instName, ifog, gCtx.EAGER_CUDA_FREE);
+	}
+	
+	private static Pointer copy(GPUContext gCtx, String instName, Pointer ptr, long numElems) {
+		Pointer ret = gCtx.allocate(instName, numElems*sizeOfDataType);
+		cudaMemcpy(ret, ptr, numElems*sizeOfDataType, cudaMemcpyDeviceToDevice);
+		return ret;
+	}
+	
 	/**
 	 * Computes the forward pass for an LSTM layer with M neurons.
 	 * The input data has N sequences of T examples, each with D features.
@@ -879,13 +1094,13 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 	 * @param T sequence length
 	 * @throws DMLRuntimeException if error
 	 */
-	public static void lstm(ExecutionContext ec, GPUContext gCtx, String instName,
+	public static void cuDNNLstm(ExecutionContext ec, GPUContext gCtx, String instName,
 			Pointer X,  Pointer wPointer, Pointer out0, Pointer c0, boolean return_sequences,
 			String outputName, String cyName, int N, int M, int D, int T) throws DMLRuntimeException {
-		singleLayerUnidirectionalRNNForward(ec, gCtx, instName, X, out0, c0, wPointer, outputName, cyName, "lstm", return_sequences, N, M, D, T);
+		cuDNNSingleLayerUnidirectionalRNNForward(ec, gCtx, instName, X, out0, c0, wPointer, outputName, cyName, "lstm", return_sequences, N, M, D, T);
 	}
 	
-	private static void singleLayerUnidirectionalRNNForward(ExecutionContext ec, GPUContext gCtx, String instName,
+	private static void cuDNNSingleLayerUnidirectionalRNNForward(ExecutionContext ec, GPUContext gCtx, String instName,
 			Pointer x, Pointer hx, Pointer cx, Pointer wPointer,  // input
 			String outputName, String cyName,  					 // output
 			String rnnMode, boolean return_sequences, int N, int M, int D, int T) throws DMLRuntimeException {
@@ -924,13 +1139,20 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 		gCtx.cudaFreeHelper(instName, cudnnYPointer, gCtx.EAGER_CUDA_FREE);
 	}
 	
-	public static void lstmBackward(ExecutionContext ec, GPUContext gCtx, String instName,
+	public static void cuDNNLstmBackward(ExecutionContext ec, GPUContext gCtx, String instName,
 			Pointer x, Pointer hx, Pointer cx, Pointer wPointer, String doutName, String dcyName,  // input
 			String dxName, String dwName, String dbName, String dhxName, String dcxName,  	// output
 			boolean return_sequences, long N, long M, long D, long T) throws DMLRuntimeException {
 		
 		if(LOG.isDebugEnabled()) {
-			long memRequired = (N*T*M + (return_sequences ? T*M : M) + N*T*M + 2*N*T*D + (D+M+2)*(4*M))*sizeOfDataType;
+			long memRequired = (D+M)*4*M // sysmlWPointer
+					+ 2*(D+M+2)*(4*M) // cudnnWPointer and cudnnDwPointer
+					+ 3*N*T*D  // cudnnInput, cudnnDx and smlDx
+					+ 2*N*T*M // dy and yPointer
+					+ (return_sequences ? T*M : M); // dout
+			memRequired *= LibMatrixCUDA.sizeOfDataType;
+			// Assume the workspace to be proportional to cudnnWPointer
+			// memRequired += (D+M+2)*(4*M)*LibMatrixCUDA.sizeOfDataType;
 			LOG.debug("Memory required for invoking lstmBackward is " + memRequired + " bytes + workspace + reserve space + memory for descriptors.");
 		}
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java
index 9833456..6dacf28 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java
@@ -45,9 +45,9 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 
 	private static final Log LOG = LogFactory.getLog(LibMatrixCuMatMult.class.getName());
 
-	private static class CuMatMultParameters {
+	public static class CuMatMultParameters {
 		/*
-		 * For the operation, C = op(A) %*% op(B), the below parameters are used
+		 * For the operation, C = alpha * op(A) %*% op(B) + beta*C, the below parameters are used
 		 * to invoke the corresponding kernels in CuBLAS and CuSPARSE.
 		 * 
 		 * All the below values have to be valid or else this class has to throw
@@ -68,8 +68,16 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 		public long rightNumCols; // number of cols of B
 		private boolean isLeftTransposed; // is op(A) = t(A)
 		private boolean isRightTransposed; // is op(B) = t(B)
+		private Pointer alpha = one();
+		private Pointer beta = zero();
 
 		public CuMatMultParameters(long leftNumRows1, long leftNumCols1, long rightNumRows1, long rightNumCols1,
+				boolean isLeftTransposed1, boolean isRightTransposed1, Pointer alpha1, Pointer beta1) {
+			this(leftNumRows1, leftNumCols1, rightNumRows1, rightNumCols1, isLeftTransposed1, isRightTransposed1);
+			alpha = alpha1;
+			beta = beta1;
+		}
+		public CuMatMultParameters(long leftNumRows1, long leftNumCols1, long rightNumRows1, long rightNumCols1,
 				boolean isLeftTransposed1, boolean isRightTransposed1) {
 			leftNumRows = leftNumRows1;
 			leftNumCols = leftNumCols1;
@@ -281,7 +289,7 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 			// Transpose: C = t(output)
 			long t0 = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 			cudaSupportFunctions.cublasgeam(gCtx.getCublasHandle(), cublasOperation.CUBLAS_OP_T, cublasOperation.CUBLAS_OP_T,
-					toInt(outCLen), toInt(outRLen), one(), output, toInt(outRLen), zero(), new Pointer(),
+					toInt(outCLen), toInt(outRLen), params.alpha, output, toInt(outRLen), params.beta, new Pointer(),
 					toInt(outRLen), C, toInt(outCLen));
 			if (!gCtx.EAGER_CUDA_FREE)
 				JCuda.cudaDeviceSynchronize();
@@ -310,7 +318,7 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 	 * @param param
 	 *            BLAS parameters
 	 */
-	private static void denseSparseMatMult(cusparseHandle handle, String instName, Pointer C, Pointer A, CSRPointer B,
+	static void denseSparseMatMult(cusparseHandle handle, String instName, Pointer C, Pointer A, CSRPointer B,
 			CuMatMultParameters param) {
 		long t0 = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 		String kernel = GPUInstruction.MISC_TIMER_SPARSE_MATRIX_DENSE_MATRIX_LIB;
@@ -322,8 +330,8 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 			int m = toInt(param.rightNumRows);
 			int n = toInt(param.rightNumCols);
 			int transa = reverseCusparseOp(cusparseOp(param.isLeftTransposed));
-			cudaSupportFunctions.cusparsecsrmv(handle, transa, m, n, toInt(B.nnz), one(), B.descr, B.val, B.rowPtr, B.colInd, A,
-					zero(), C);
+			cudaSupportFunctions.cusparsecsrmv(handle, transa, m, n, toInt(B.nnz), param.alpha, B.descr, B.val, B.rowPtr, B.colInd, A,
+					param.beta, C);
 			kernel = GPUInstruction.MISC_TIMER_SPARSE_MATRIX_DENSE_VECTOR_LIB;
 		} else {
 			int m = toInt(param.rightNumRows);
@@ -333,8 +341,8 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 			int transa = reverseCusparseOp(cusparseOp(param.isLeftTransposed));
 			int transb = cusparseOp(param.isRightTransposed);
 			LOG.debug(" GPU Sparse-Dense Matrix Multiply (rhs transpose) ");
-			cudaSupportFunctions.cusparsecsrmm2(handle, transa, transb, m, param.n, k, toInt(B.nnz), one(), B.descr, B.val,
-					B.rowPtr, B.colInd, A, param.ldb, zero(), C, param.ldc);
+			cudaSupportFunctions.cusparsecsrmm2(handle, transa, transb, m, param.n, k, toInt(B.nnz), param.alpha, B.descr, B.val,
+					B.rowPtr, B.colInd, A, param.ldb, param.beta, C, param.ldc);
 		}
 		if (ConfigurationManager.isFinegrainedStatistics())
 			GPUStatistics.maintainCPMiscTimes(instName, kernel, System.nanoTime() - t0);
@@ -359,7 +367,7 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 	 * @param param
 	 *            BLAS parameters
 	 */
-	private static void denseDenseMatMult(cublasHandle handle, String instName, Pointer C, Pointer A, Pointer B,
+	static void denseDenseMatMult(cublasHandle handle, String instName, Pointer C, Pointer A, Pointer B,
 			CuMatMultParameters param) {
 		long t0 = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 		String kernel = null;
@@ -388,19 +396,19 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 			transb = reverseCublasOp(transb);
 			int rightNumRows = (transb == CUSPARSE_OPERATION_TRANSPOSE) ? param.k : param.n;
 			int rightNumCols = (transb == CUSPARSE_OPERATION_TRANSPOSE) ? param.n : param.k;
-			cudaSupportFunctions.cublasgemv(handle, transb, rightNumRows, rightNumCols, one(), B, param.ldb, A, 1, zero(), C, 1);
+			cudaSupportFunctions.cublasgemv(handle, transb, rightNumRows, rightNumCols, param.alpha, B, param.ldb, A, 1, param.beta, C, 1);
 			kernel = GPUInstruction.MISC_TIMER_DENSE_VECTOR_DENSE_MATRIX_LIB;
 		} else if (param.n == 1) {
 			// Matrix-vector multiply
 			LOG.debug(" GPU Dense Matrix-Vector Multiply");
 			int leftNumRows = (transa == CUSPARSE_OPERATION_NON_TRANSPOSE) ? param.m : param.k;
 			int leftNumCols = (transa == CUSPARSE_OPERATION_NON_TRANSPOSE) ? param.k : param.m;
-			cudaSupportFunctions.cublasgemv(handle, transa, leftNumRows, leftNumCols, one(), A, param.lda, B, 1, zero(), C, 1);
+			cudaSupportFunctions.cublasgemv(handle, transa, leftNumRows, leftNumCols, param.alpha, A, param.lda, B, 1, param.beta, C, 1);
 			kernel = GPUInstruction.MISC_TIMER_DENSE_MATRIX_DENSE_VECTOR_LIB;
 		} else {
 			LOG.debug(" GPU Dense-Dense Matrix Multiply ");
-			cudaSupportFunctions.cublasgemm(handle, transa, transb, param.m, param.n, param.k, one(), A, param.lda, B, param.ldb,
-					zero(), C, param.ldc);
+			cudaSupportFunctions.cublasgemm(handle, transa, transb, param.m, param.n, param.k, param.alpha, A, param.lda, B, param.ldb,
+					param.beta, C, param.ldc);
 			kernel = GPUInstruction.MISC_TIMER_DENSE_MATRIX_DENSE_MATRIX_LIB;
 		}
 		if (ConfigurationManager.isFinegrainedStatistics())

http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/src/test/java/org/apache/sysml/test/gpu/LstmTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/gpu/LstmTest.java b/src/test/java/org/apache/sysml/test/gpu/LstmTest.java
new file mode 100644
index 0000000..47afe3a
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/gpu/LstmTest.java
@@ -0,0 +1,318 @@
+/*
+ * 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.test.gpu;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.sysml.runtime.instructions.gpu.DnnGPUInstruction;
+import org.apache.sysml.runtime.instructions.gpu.DnnGPUInstruction.LstmOperator;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * Tests lstm builtin function
+ */
+public class LstmTest extends GPUTests {
+
+	private final static String TEST_NAME = "LstmTests";
+	private final int seed = 42;
+	
+	private final static String builtinDML = "\"nn/layers/lstm_staging.dml\"";
+	private final static String nnDML = "\"nn/layers/lstm.dml\"";
+
+	@Override
+	public void setUp() {
+		super.setUp();
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(TEST_DIR, TEST_NAME);
+		getAndLoadTestConfiguration(TEST_NAME);
+	}
+
+	@Test
+	public void testLstmForward1() {
+		testLstmCuDNNWithNNBuiltinOperator(1, 1, 1, 1, "TRUE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward2() {
+		testLstmCuDNNWithNNBuiltinOperator(1, 1, 1, 1, "FALSE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward3() {
+		testLstmCuDNNWithNNBuiltinOperator(20, 13, 50, 10, "TRUE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward4() {
+		testLstmCuDNNWithNNBuiltinOperator(20, 13, 50, 10, "FALSE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward5() {
+		testLstmCuDNNWithNNBuiltinOperator(1, 3, 5, 1, "TRUE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward6() {
+		testLstmCuDNNWithNNBuiltinOperator(1, 3, 5, 1, "FALSE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward7() {
+		testLstmCuDNNWithNNBuiltinOperator(20, 13, 50, 10, "TRUE", 0.1);
+	}
+	
+	@Test
+	public void testLstmForward8() {
+		testLstmCuDNNWithNNBuiltinOperator(20, 13, 50, 10, "FALSE", 0.1);
+	}
+	
+	@Test
+	public void testLstmForward9() {
+		testLstmCuDNNWithNNLayer(1, 1, 1, 1, "TRUE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward10() {
+		testLstmCuDNNWithNNLayer(1, 1, 1, 1, "FALSE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward11() {
+		testLstmCuDNNWithNNLayer(20, 13, 50, 10, "TRUE", 0.9);
+	}
+	
+	@Test
+	public void testLstmForward12() {
+		testLstmCuDNNWithNNLayer(20, 13, 50, 10, "FALSE", 0.9);
+	}
+	
+	public void testLstmCuDNNWithNNBuiltinOperator(int N, int T, int D, int M, String returnSequences, double sparsity) {
+		String scriptStr = "source(" + builtinDML + ") as lstm;\n "
+				+ "[output, c] = lstm::forward(x, w, b, " + returnSequences + ", out0, c0)";
+		
+		HashMap<String, Object> inputs = new HashMap<>();
+		inputs.put("x", generateInputMatrix(spark, N, T*D, 0, 10, sparsity, seed));
+		inputs.put("w", generateInputMatrix(spark, D+M, 4*M, 0, 10, sparsity, seed));
+		inputs.put("b", generateInputMatrix(spark, 1, 4*M, 0, 10, sparsity, seed));
+		inputs.put("out0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		inputs.put("c0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		List<String> outputs = Arrays.asList("output", "c");
+		List<Object> outGPUWithCuDNN = null;
+		List<Object> outGPUWithNN = null;
+		synchronized (DnnGPUInstruction.FORCED_LSTM_OP) {
+			try {
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.CUDNN;
+				outGPUWithCuDNN = runOnGPU(spark, scriptStr, inputs, outputs);
+				inputs = new HashMap<>();
+				inputs.put("x", generateInputMatrix(spark, N, T*D, 0, 10, sparsity, seed));
+				inputs.put("w", generateInputMatrix(spark, D+M, 4*M, 0, 10, sparsity, seed));
+				inputs.put("b", generateInputMatrix(spark, 1, 4*M, 0, 10, sparsity, seed));
+				inputs.put("out0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+				inputs.put("c0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.DENSE_NN;
+				outGPUWithNN = runOnGPU(spark, scriptStr, inputs, outputs);
+			}
+			finally {
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.NONE;
+			}
+		}
+		assertEqualObjects(outGPUWithCuDNN.get(0), outGPUWithNN.get(0));
+		assertEqualObjects(outGPUWithCuDNN.get(1), outGPUWithNN.get(1));
+	}
+	
+	public void testLstmCuDNNWithNNLayer(int N, int T, int D, int M, String returnSequences, double sparsity) {
+		String scriptStr1 = "source(" + builtinDML + ") as lstm;\n "
+				+ "[output, c] = lstm::forward(x, w, b, " + returnSequences + ", out0, c0)";
+		String scriptStr2 = "source(" + nnDML + ") as lstm;\n "
+				+ "[output, c, cache_out, cache_c, cache_ifog] = lstm::forward(x, w, b, " 
+				+ T + ", " + D + ", " + returnSequences + ", out0, c0)";
+		
+		HashMap<String, Object> inputs = new HashMap<>();
+		inputs.put("x", generateInputMatrix(spark, N, T*D, 0, 10, sparsity, seed));
+		inputs.put("w", generateInputMatrix(spark, D+M, 4*M, 0, 10, sparsity, seed));
+		inputs.put("b", generateInputMatrix(spark, 1, 4*M, 0, 10, sparsity, seed));
+		inputs.put("out0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		inputs.put("c0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		List<String> outputs = Arrays.asList("output", "c");
+		List<Object> outGPUWithCuDNN = null;
+		List<Object> outCPUWithNN = null;
+		synchronized (DnnGPUInstruction.FORCED_LSTM_OP) {
+			try {
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.CUDNN;
+				outGPUWithCuDNN = runOnGPU(spark, scriptStr1, inputs, outputs);
+				outCPUWithNN = runOnCPU(spark, scriptStr2, inputs, outputs);
+			}
+			finally {
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.NONE;
+			}
+		}
+		assertEqualObjects(outGPUWithCuDNN.get(0), outCPUWithNN.get(0));
+		assertEqualObjects(outGPUWithCuDNN.get(1), outCPUWithNN.get(1));
+	}
+	
+	@Test
+	public void testLstmBackward1() {
+		testLstmBackwardCuDNNWithNNBuiltinOperator(1, 1, 1, 1, "TRUE", 0.9, 0.9);
+	}
+	
+	@Test
+	public void testLstmBackward2() {
+		testLstmBackwardCuDNNWithNNBuiltinOperator(1, 1, 1, 1, "FALSE", 0.9, 0.9);
+	}
+	
+	@Test
+	public void testLstmBackward3() {
+		testLstmBackwardCuDNNWithNNBuiltinOperator(20, 13, 50, 10, "TRUE", 0.9, 0.9);
+	}
+	
+	@Test
+	public void testLstmBackward4() {
+		testLstmBackwardCuDNNWithNNBuiltinOperator(20, 13, 50, 10, "FALSE", 0.9, 0.9);
+	}
+	
+//	@Test
+//	public void testLstmBackward5() {
+//		testLstmBackwardCuDNNWithNNBuiltinOperator(20, 13, 50, 10, "TRUE", 0.9, 0.1);
+//	}
+//	
+//	@Test
+//	public void testLstmBackward6() {
+//		testLstmBackwardCuDNNWithNNBuiltinOperator(20, 13, 50, 10, "FALSE", 0.9, 0.1);
+//	}
+	
+	
+	@Test
+	public void testLstmBackward7() {
+		testLstmBackwardCuDNNWithNNLayer(1, 1, 1, 1, "TRUE", 0.9, 0.9);
+	}
+	
+	@Test
+	public void testLstmBackward8() {
+		testLstmBackwardCuDNNWithNNLayer(1, 1, 1, 1, "FALSE", 0.9, 0.9);
+	}
+	
+	@Test
+	public void testLstmBackward9() {
+		testLstmBackwardCuDNNWithNNLayer(20, 13, 50, 10, "TRUE", 0.9, 0.9);
+	}
+	
+	@Test
+	public void testLstmBackward10() {
+		testLstmBackwardCuDNNWithNNLayer(20, 13, 50, 10, "FALSE", 0.9, 0.9);
+	}
+	
+//	@Test
+//	public void testLstmBackward11() {
+//		testLstmBackwardCuDNNWithNNLayer(20, 13, 50, 10, "TRUE", 0.9, 0.1);
+//	}
+//	
+//	@Test
+//	public void testLstmBackward12() {
+//		testLstmBackwardCuDNNWithNNLayer(20, 13, 50, 10, "FALSE", 0.9, 0.1);
+//	}
+	
+	public void testLstmBackwardCuDNNWithNNBuiltinOperator(int N, int T, int D, int M, String returnSequences, double sparsity, 
+			double weightSparsity) {
+		boolean returnSequences1 = returnSequences.equals("TRUE");
+				
+		String scriptStr = "source(" + builtinDML + ") as lstm;\n "
+				+ "[dX, dW, db, dout0, dc0] = lstm::backward(dout, dc, x, w, b, " + returnSequences + ", out0, c0);";
+		
+		HashMap<String, Object> inputs = new HashMap<>();
+		inputs.put("dout", generateInputMatrix(spark, N, returnSequences1 ? T*M : M, 0, 10, sparsity, seed));
+		inputs.put("dc", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		inputs.put("x", generateInputMatrix(spark, N, T*D, 0, 10, sparsity, seed));
+		inputs.put("w", generateInputMatrix(spark, D+M, 4*M, 0, 10, weightSparsity, seed));
+		inputs.put("b", generateInputMatrix(spark, 1, 4*M, 0, 10, sparsity, seed));
+		inputs.put("out0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		inputs.put("c0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		List<String> outputs = Arrays.asList("dX", "dW", "db", "dout0", "dc0");
+		List<Object> outGPUWithCuDNN = null;
+		List<Object> outGPUWithNN = null;
+		synchronized (DnnGPUInstruction.FORCED_LSTM_OP) {
+			try {
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.CUDNN;
+				outGPUWithCuDNN = runOnGPU(spark, scriptStr, inputs, outputs);
+				inputs = new HashMap<>();
+				inputs.put("dout", generateInputMatrix(spark, N, returnSequences1 ? T*M : M, 0, 10, sparsity, seed));
+				inputs.put("dc", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+				inputs.put("x", generateInputMatrix(spark, N, T*D, 0, 10, sparsity, seed));
+				inputs.put("w", generateInputMatrix(spark, D+M, 4*M, 0, 10, weightSparsity, seed));
+				inputs.put("b", generateInputMatrix(spark, 1, 4*M, 0, 10, sparsity, seed));
+				inputs.put("out0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+				inputs.put("c0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.DENSE_NN;
+				outGPUWithNN = runOnGPU(spark, scriptStr, inputs, outputs);
+			}
+			finally {
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.NONE;
+			}
+		}
+		assertEqualObjects(outGPUWithCuDNN.get(0), outGPUWithNN.get(0));
+		assertEqualObjects(outGPUWithCuDNN.get(1), outGPUWithNN.get(1));
+		assertEqualObjects(outGPUWithCuDNN.get(2), outGPUWithNN.get(2));
+		assertEqualObjects(outGPUWithCuDNN.get(3), outGPUWithNN.get(3));
+		assertEqualObjects(outGPUWithCuDNN.get(4), outGPUWithNN.get(4));
+	}
+	
+	public void testLstmBackwardCuDNNWithNNLayer(int N, int T, int D, int M, String returnSequences, double sparsity,
+			double weightSparsity) {
+		boolean returnSequences1 = returnSequences.equals("TRUE");
+		
+		String scriptStr1 = "source(" + builtinDML + ") as lstm;\n "
+				+ "[dX, dW, db, dout0, dc0] = lstm::backward(dout, dc, x, w, b, " + returnSequences + ", out0, c0);";
+		String scriptStr2 = "source(" + nnDML + ") as lstm;\n "
+				+ "[output, c, cache_out, cache_c, cache_ifog] = lstm::forward(x, w, b, " 
+				+ T + ", " + D + ", " + returnSequences + ", out0, c0); \n"
+				+ "[dX, dW, db, dout0, dc0] = lstm::backward(dout, dc, x, w, b, " 
+				+ T + ", " + D + ", " + returnSequences + ", out0, c0, cache_out, cache_c, cache_ifog);";
+		
+		HashMap<String, Object> inputs = new HashMap<>();
+		inputs.put("dout", generateInputMatrix(spark, N, returnSequences1 ? T*M : M, 0, 10, sparsity, seed));
+		inputs.put("dc", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		inputs.put("x", generateInputMatrix(spark, N, T*D, 0, 10, sparsity, seed));
+		inputs.put("w", generateInputMatrix(spark, D+M, 4*M, 0, 10, weightSparsity, seed));
+		inputs.put("b", generateInputMatrix(spark, 1, 4*M, 0, 10, sparsity, seed));
+		inputs.put("out0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		inputs.put("c0", generateInputMatrix(spark, N, M, 0, 10, sparsity, seed));
+		List<String> outputs = Arrays.asList("dX", "dW", "db", "dout0", "dc0");
+		List<Object> outGPUWithCuDNN = null;
+		List<Object> outCPUWithNN = null;
+		synchronized (DnnGPUInstruction.FORCED_LSTM_OP) {
+			try {
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.CUDNN;
+				outGPUWithCuDNN = runOnGPU(spark, scriptStr1, inputs, outputs);
+			}
+			finally {
+				DnnGPUInstruction.FORCED_LSTM_OP = LstmOperator.NONE;
+			}
+			outCPUWithNN = runOnCPU(spark, scriptStr2, inputs, outputs);
+		}
+		assertEqualObjects(outGPUWithCuDNN.get(0), outCPUWithNN.get(0));
+		assertEqualObjects(outGPUWithCuDNN.get(1), outCPUWithNN.get(1));
+		assertEqualObjects(outGPUWithCuDNN.get(2), outCPUWithNN.get(2));
+		assertEqualObjects(outGPUWithCuDNN.get(3), outCPUWithNN.get(3));
+		assertEqualObjects(outGPUWithCuDNN.get(4), outCPUWithNN.get(4));
+	}
+}


[2/2] systemml git commit: [SYSTEMML-445] Support non-CuDNN GPU operator for LSTM forward and backward

Posted by ni...@apache.org.
[SYSTEMML-445] Support non-CuDNN GPU operator for LSTM forward and backward

- Added corresponding GPU tests that compare the result of CuDNN operator with the newly added operator. Also, the results are compared with DML-bodied LSTM implementation in the nn layer.
- The LSTM forward operator support sparse weights.
- Sparse support for LSTM backward is disabled in the initial implementation.
- Unnecessary intermediates are removed from lstm.dml
- Extended LibMatrixCuMatMult to support arbitrary alpha and beta during matrix multiplication.


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

Branch: refs/heads/master
Commit: bd34292d4e521ffaa5118f89ab9350ffe4e89af0
Parents: ef842da
Author: Niketan Pansare <np...@us.ibm.com>
Authored: Sat Oct 20 11:03:53 2018 -0700
Committer: Niketan Pansare <np...@us.ibm.com>
Committed: Sat Oct 20 11:08:11 2018 -0700

----------------------------------------------------------------------
 scripts/nn/layers/lstm.dml                      |    1 -
 src/main/cpp/kernels/SystemML.cu                |  315 +++
 src/main/cpp/kernels/SystemML.ptx               | 2074 +++++++++++++++++-
 .../instructions/gpu/DnnGPUInstruction.java     |  232 +-
 .../gpu/context/GPUMemoryManager.java           |    4 +
 .../runtime/matrix/data/LibMatrixCUDA.java      |   21 +-
 .../runtime/matrix/data/LibMatrixCuDNN.java     |  236 +-
 .../runtime/matrix/data/LibMatrixCuMatMult.java |   34 +-
 .../org/apache/sysml/test/gpu/LstmTest.java     |  318 +++
 9 files changed, 3130 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/scripts/nn/layers/lstm.dml
----------------------------------------------------------------------
diff --git a/scripts/nn/layers/lstm.dml b/scripts/nn/layers/lstm.dml
index 44942d2..0b0016b 100644
--- a/scripts/nn/layers/lstm.dml
+++ b/scripts/nn/layers/lstm.dml
@@ -182,7 +182,6 @@ backward = function(matrix[double] dout, matrix[double] dc,
   for (iter in 1:T) {  # each timestep in reverse order
     X_t = X[,(t-1)*D+1:t*D]  # shape (N, D)
     dout_t = dout[,(t-1)*M+1:t*M]  # shape (N, M)
-    out_t = matrix(cache_out[t,], rows=N, cols=M)  # shape (N, M)
     ct = matrix(cache_c[t,], rows=N, cols=M)  # shape (N, M)
     if (t == 1) {
       out_prev = out0  # shape (N, M)

http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/src/main/cpp/kernels/SystemML.cu
----------------------------------------------------------------------
diff --git a/src/main/cpp/kernels/SystemML.cu b/src/main/cpp/kernels/SystemML.cu
index 26d7f43..ab5f326 100644
--- a/src/main/cpp/kernels/SystemML.cu
+++ b/src/main/cpp/kernels/SystemML.cu
@@ -2406,3 +2406,318 @@ extern "C" __global__ void backward_dgamma_tmp_f(double *ema_mean, double *dout,
 	int N, int C, int HW, int CHW, int NCHW) {
   backward_dgamma_tmp(ema_mean, dout, X, ema_var, ret, N, C, HW, CHW, NCHW);
 }
+
+
+// Performs the operation:
+// X_t = X[,(t-1)*D+1:t*D]  # shape (N, D)
+// ret = cbind(X_t, out_prev)  # shape (N, D+M)
+// size => N*(D+M)
+template <typename T>
+__device__ void prepareInputNNLstm(T *X, T* out_prev, T *ret, int t, int M, int D, int TD, int DPlusM, unsigned int size) {
+  int index = blockIdx.x * blockDim.x + threadIdx.x;
+  if (index < size) {
+    int n = index / DPlusM;
+  	int iy = index % DPlusM;
+    if(iy < D) {
+    	ret[index] = X[n*TD + t*D + iy];
+    }
+    else {
+    	ret[index] = out_prev[n*M + (iy-D)];
+    }
+  }
+}
+
+extern "C" __global__ void prepareInputNNLstm_d(double *X, double* out_prev, double *ret, int t, int M, int D, int TD, int DPlusM, unsigned int size) {
+  prepareInputNNLstm(X, out_prev, ret, t, M, D, TD, DPlusM, size);
+}
+
+extern "C" __global__ void prepareInputNNLstm_f(float *X, float* out_prev, float *ret, int t, int M, int D, int TD, int DPlusM, unsigned int size) {
+  prepareInputNNLstm(X, out_prev, ret, t, M, D, TD, DPlusM, size);
+}
+
+
+// Performs the operations:
+// ifog = ifog + b
+// ifog[,1:3*M] = sigmoid::forward(ifog[,1:3*M])  # i,f,o gates squashed with sigmoid
+// ifog[,3*M+1:4*M] = tanh::forward(ifog[,3*M+1:4*M])  # g gate squashed with tanh
+template <typename T>
+__device__ void squashIFOG(T *ifog, T *b, int M, unsigned int size) {
+  int index = blockIdx.x * blockDim.x + threadIdx.x;
+  if (index < size) {
+    int M4 = M*4;
+  	int n = index / M4;
+  	int iy = index % M4; 
+	T ifogVal = ifog[index] + b[iy];
+	if(iy < M*3) {
+		ifogVal = 0.5 * tanh(0.5 * ifogVal) + 0.5; // sigmoid
+	}
+	else {
+		ifogVal = tanh(ifogVal);
+	}
+	ifog[index] = ifogVal;
+  }
+}
+
+extern "C" __global__ void squashIFOG_d(double *ifog, double *b, int M, unsigned int size) {
+  squashIFOG(ifog, b, M, size);
+}
+
+extern "C" __global__ void squashIFOG_f(float *ifog, float *b, int M, unsigned int size) {
+  squashIFOG(ifog, b, M, size);
+}
+
+// c = ifog[,M+1:2*M]*c_prev + ifog[,1:M]*ifog[,3*M+1:4*M]
+// out_t = ifog[,2*M+1:3*M] * tanh::forward(c)
+// if (return_sequences) {
+//   out[,(t-1)*M+1:t*M] = out_t
+// }
+// else {
+//   out = out_t
+// }
+// out_prev = out_t
+// c_prev = c
+// cache_out[t,] = matrix(out_t, rows=1, cols=N*M)
+// cache_c[t,] = matrix(c, rows=1, cols=N*M) 
+template <typename T>
+__device__ void postProcessNNLstmForward(T *ifog, 
+	T *c,  T* out_prev, T* c_prev, 
+	T *out, T *cache_out, T *cache_c,
+	int return_sequences, int t, int T1, int M,
+	unsigned int NM) { 
+	int index = blockIdx.x * blockDim.x + threadIdx.x;
+  if (index < NM) {
+    int M4 = M*4;
+  	int n = index / M;
+  	int m = index % M;
+  	int m4 = m*4;
+  	T iGate = ifog[n*M4 + m]; 		// ifog[,1:M]
+  	T fGate = ifog[n*M4 + M + m];  // ifog[,M+1:2*M]
+  	T oGate = ifog[n*M4 + M*2 + m]; // ifog[,2*M+1:3*M]
+  	T gGate = ifog[n*M4 + M*3 + m]; // ifog[,3*M+1:4*M]
+  	T cVal = fGate*c_prev[index] + iGate*gGate;
+  	T out_tVal = oGate*tanh(cVal);
+  	int outIndex = return_sequences == 0 ? index : (n*T1*M + t*M + m);
+  	int cacheIndex = t*NM + index;
+  	
+  	c[index] = cVal;
+  	out_prev[index] = out_tVal;
+  	c_prev[index] = cVal;
+  	cache_out[cacheIndex] = out_tVal;
+  	cache_c[cacheIndex] = cVal;
+  	out[outIndex] = out_tVal;
+  }
+}
+
+extern "C" __global__ void postProcessNNLstmForward_d(double *ifog, 
+	double *c, double *out_prev, double *c_prev, 
+	double *out, double *cache_out, double *cache_c,
+	int return_sequences, int t, int T1, int M,
+	unsigned int NM) { 
+	postProcessNNLstmForward(ifog, c, out_prev, c_prev, out, cache_out, cache_c, return_sequences, t, T1, M, NM);
+}
+
+extern "C" __global__ void postProcessNNLstmForward_f(float *ifog, 
+	float *c, float *out_prev, float *c_prev, 
+	float *out, float *cache_out, float *cache_c,
+	int return_sequences, int t, int T1, int M,
+	unsigned int NM) { 
+	postProcessNNLstmForward(ifog, c, out_prev, c_prev, out, cache_out, cache_c, return_sequences, t, T1, M, NM);
+}
+
+
+// c = ifog[,M+1:2*M]*c_prev + ifog[,1:M]*ifog[,3*M+1:4*M]
+// out_t = ifog[,2*M+1:3*M] * tanh::forward(c)
+// if (return_sequences) {
+//   out[,(t-1)*M+1:t*M] = out_t
+// }
+// else {
+//   out = out_t
+// }
+// out_prev = out_t
+// c_prev = c
+template <typename T>
+__device__ void postProcessNNLstmForwardSkipCache(T *ifog, 
+	T *c,  T* out_prev, T* c_prev, 
+	T *out, 
+	int return_sequences, int t, int T1, int M,
+	unsigned int NM) { 
+	int index = blockIdx.x * blockDim.x + threadIdx.x;
+  if (index < NM) {
+    int M4 = M*4;
+  	int n = index / M;
+  	int m = index % M;
+  	int m4 = m*4;
+  	T iGate = ifog[n*M4 + m]; 		// ifog[,1:M]
+  	T fGate = ifog[n*M4 + M + m];  // ifog[,M+1:2*M]
+  	T oGate = ifog[n*M4 + M*2 + m]; // ifog[,2*M+1:3*M]
+  	T gGate = ifog[n*M4 + M*3 + m]; // ifog[,3*M+1:4*M]
+  	T cVal = fGate*c_prev[index] + iGate*gGate;
+  	T out_tVal = oGate*tanh(cVal);
+  	int outIndex = return_sequences == 0 ? index : (n*T1*M + t*M + m);
+  	int cacheIndex = t*NM + index;
+  	
+  	c[index] = cVal;
+  	out_prev[index] = out_tVal;
+  	c_prev[index] = cVal;
+  	out[outIndex] = out_tVal;
+  }
+}
+
+extern "C" __global__ void postProcessNNLstmForwardSkipCache_d(double *ifog, 
+	double *c, double *out_prev, double *c_prev, 
+	double *out, 
+	int return_sequences, int t, int T1, int M,
+	unsigned int NM) { 
+	postProcessNNLstmForwardSkipCache(ifog, c, out_prev, c_prev, out, return_sequences, t, T1, M, NM);
+}
+
+extern "C" __global__ void postProcessNNLstmForwardSkipCache_f(float *ifog, 
+	float *c, float *out_prev, float *c_prev, 
+	float *out, 
+	int return_sequences, int t, int T1, int M,
+	unsigned int NM) { 
+	postProcessNNLstmForwardSkipCache(ifog, c, out_prev, c_prev, out, return_sequences, t, T1, M, NM);
+}
+
+template <typename T>
+__device__ void initializeDoutWhenReturnSeq(T *dout, T *dout_t, int t, int M, int TM, unsigned int NM) {
+	int index = blockIdx.x * blockDim.x + threadIdx.x;
+  	if (index < NM) {
+  		int n = index / M;
+  		int m = index % M;
+		dout_t[index] = dout[n*TM + t*M + m];
+	}
+}
+
+extern "C" __global__ void initializeDoutWhenReturnSeq_d(double *dout, double  *dout_t, int t, int M, int TM, unsigned int NM) {
+	initializeDoutWhenReturnSeq(dout, dout_t, t, M, TM, NM);
+}
+
+extern "C" __global__ void initializeDoutWhenReturnSeq_f(float *dout, float *dout_t, int t, int M, int TM, unsigned int NM) {
+	initializeDoutWhenReturnSeq(dout, dout_t, t, M, TM, NM);
+}
+
+
+// Performs the operation
+// i = ifog[,1:M]  # input gate, shape (N, M)
+// f = ifog[,M+1:2*M]  # forget gate, shape (N, M)
+// o = ifog[,2*M+1:3*M]  # output gate, shape (N, M)
+// g = ifog[,3*M+1:4*M]  # g gate, shape (N, M)
+// dct = dct + o*tanh::backward(dout_t, ct)  # shape (N, M)
+// do = tanh::forward(ct) * dout_t  # output gate, shape (N, M)
+// df = c_prev * dct  # forget gate, shape (N, M)
+// dc_prev = f * dct  # shape (N, M)
+// di = g * dct  # input gate, shape (N, M)
+// dg = i * dct  # g gate, shape (N, M)
+// di_raw = i * (1-i) * di
+// df_raw = f * (1-f) * df
+// do_raw = o * (1-o) * do
+// dg_raw = (1-g^2) * dg
+// difog_raw = cbind(di_raw, df_raw, do_raw, dg_raw)  # shape (N, 4M)
+template <typename T>
+__device__ void computeDifog_raw(T *ifog, T *ct, T *dout_t, T *cache_c, T *c0, 
+	T *difog_raw, T *dct, T *dc0, // output
+	int return_sequences, int t, int T1, int M, unsigned int NM) {
+  int index = blockIdx.x * blockDim.x + threadIdx.x;
+  if (index < NM) {
+  	int M4 = M*4;
+  	int n = index / M;
+  	int m = index % M;
+  	
+  	T dout_tVal = dout_t[index];
+  	
+  	T i = ifog[n*M4 + m];
+  	T f = ifog[n*M4 + M + m];
+  	T o = ifog[n*M4 + M*2 + m];
+  	T g = ifog[n*M4 + M*3 + m];
+  	
+  	T ctVal = ct[index];
+  	
+  	// if (t == 1) 
+  	// 	 c_prev = c0  # shape (N, M)
+  	// else
+  	//   c_prev = matrix(cache_c[t-1,], rows=N, cols=M)  # shape (N, M)
+  	T c_prevVal = (t==0) ? c0[index] : cache_c[(t-1)*NM + index];
+  	
+  	// dct = dct + o*tanh::backward(dout_t, ct)
+  	T tmp = tanh(ctVal);
+  	T dctVal = dct[index] + o*((1-tmp*tmp) * dout_tVal);
+  	
+  	T dc_prevVal = f * dctVal;
+  	
+	T do1 = tanh(ctVal) * dout_tVal;
+	T df = c_prevVal * dctVal;
+  	T di = g * dctVal;
+  	T dg = i * dctVal;
+  	
+  	if (t == 0) {
+  		dc0[index] = dc_prevVal;
+  		dct[index] = dctVal;
+  	}
+  	else {
+  		dct[index] = dc_prevVal;
+  	}
+  	difog_raw[n*M4 + m] = i * (1-i) * di; // di_raw
+  	difog_raw[n*M4 + M + m] = f * (1-f) * df; // df_raw
+  	difog_raw[n*M4 + M*2 + m] = o * (1-o) * do1; // do_raw
+  	difog_raw[n*M4 + M*3 + m] = (1-g*g) * dg; // dg_raw
+  }
+}
+
+extern "C" __global__ void computeDifog_raw_d(double *ifog, double *ct, double *dout_t, double *cache_c, double *c0, 
+	double *difog_raw, double *dct, double *dc0, // output
+	int return_sequences, int t, int T1, int M, unsigned int NM) {
+	computeDifog_raw(ifog, ct, dout_t, cache_c, c0, 
+		difog_raw, dct, dc0, // output
+		return_sequences, t, T1, M, NM);
+}
+
+extern "C" __global__ void computeDifog_raw_f(float *ifog, float *ct, float *dout_t, float *cache_c, float *c0, 
+	float *difog_raw, float *dct, float *dc0, // output
+	int return_sequences, int t, int T1, int M, unsigned int NM) {
+	computeDifog_raw(ifog, ct, dout_t, cache_c, c0, 
+		difog_raw, dct, dc0, // output
+		return_sequences, t, T1, M, NM);
+}
+
+template <typename T>
+__device__ void postProcessNNLstmBackward(T *dinput, T *dout0, T* dout, T * dout_t, T *dX, int return_sequences, int t, int N, int D, int M, 
+	int ND, int NM, int TD, int TM, int DPlusM, unsigned int size) {
+  int index = blockIdx.x * blockDim.x + threadIdx.x;
+  if (index < ND) {
+  	int n = index / D;
+  	int d = index % D;
+  	// dX[,(t-1)*D+1:t*D] = dinput[,1:D] // dinput is of shape (N, D+M)
+  	dX[n*TD + t*D + d] = dinput[n*DPlusM + d];
+  }
+  if (index < NM) {
+  	int n = index / M;
+  	int m = index % M;
+  	// dout_prev = dinput[,D+1:D+M]
+  	T dout_prev = dinput[n*DPlusM + D + m];
+  	if(t == 0) {
+  		// dout0 = dout_prev
+  		dout0[index] = dout_prev;
+  	}
+  	else if(return_sequences != 0) {
+  		// dout_t =  dout[,(t-2)*M+1:(t-1)*M] + dout_prev
+  		dout_t[index] = dout[n*TM + (t-1)*M + m] + dout_prev;
+  	}
+  	else {
+  		// dout_t = dout_prev
+  		dout_t[index] = dout_prev;
+  	}
+  }
+}
+
+extern "C" __global__ void postProcessNNLstmBackward_d(double *dinput, double *dout0, double *dout, double *dout_t, double *dX, int return_sequences, int t, int N, int D, int M, 
+	int ND, int NM, int TD, int TM, int DPlusM, unsigned int size) {
+	postProcessNNLstmBackward(dinput, dout0, dout, dout_t, dX, return_sequences, t, N, D, M, 
+		ND, NM, TD, TM, DPlusM, size);
+}
+
+extern "C" __global__ void postProcessNNLstmBackward_f(float *dinput, float *dout0, float *dout, float *dout_t, float *dX, int return_sequences, int t, int N, int D, int M, 
+	int ND, int NM, int TD, int TM, int DPlusM, unsigned int size) {
+	postProcessNNLstmBackward(dinput, dout0, dout, dout_t, dX, return_sequences, t, N, D, M, 
+		ND, NM, TD, TM, DPlusM, size);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/bd34292d/src/main/cpp/kernels/SystemML.ptx
----------------------------------------------------------------------
diff --git a/src/main/cpp/kernels/SystemML.ptx b/src/main/cpp/kernels/SystemML.ptx
index 3043373..bf40fb9 100644
--- a/src/main/cpp/kernels/SystemML.ptx
+++ b/src/main/cpp/kernels/SystemML.ptx
@@ -15218,12 +15218,2032 @@ BB125_2:
 	ret;
 }
 
+	// .globl	prepareInputNNLstm_d
+.visible .entry prepareInputNNLstm_d(
+	.param .u64 prepareInputNNLstm_d_param_0,
+	.param .u64 prepareInputNNLstm_d_param_1,
+	.param .u64 prepareInputNNLstm_d_param_2,
+	.param .u32 prepareInputNNLstm_d_param_3,
+	.param .u32 prepareInputNNLstm_d_param_4,
+	.param .u32 prepareInputNNLstm_d_param_5,
+	.param .u32 prepareInputNNLstm_d_param_6,
+	.param .u32 prepareInputNNLstm_d_param_7,
+	.param .u32 prepareInputNNLstm_d_param_8
+)
+{
+	.reg .pred 	%p<3>;
+	.reg .b32 	%r<18>;
+	.reg .f64 	%fd<3>;
+	.reg .b64 	%rd<13>;
+
+
+	ld.param.u64 	%rd2, [prepareInputNNLstm_d_param_0];
+	ld.param.u64 	%rd3, [prepareInputNNLstm_d_param_1];
+	ld.param.u64 	%rd4, [prepareInputNNLstm_d_param_2];
+	ld.param.u32 	%r4, [prepareInputNNLstm_d_param_3];
+	ld.param.u32 	%r5, [prepareInputNNLstm_d_param_4];
+	ld.param.u32 	%r6, [prepareInputNNLstm_d_param_5];
+	ld.param.u32 	%r7, [prepareInputNNLstm_d_param_6];
+	ld.param.u32 	%r8, [prepareInputNNLstm_d_param_7];
+	ld.param.u32 	%r9, [prepareInputNNLstm_d_param_8];
+	mov.u32 	%r10, %ntid.x;
+	mov.u32 	%r11, %ctaid.x;
+	mov.u32 	%r12, %tid.x;
+	mad.lo.s32 	%r1, %r10, %r11, %r12;
+	setp.ge.u32	%p1, %r1, %r9;
+	@%p1 bra 	BB126_4;
+
+	cvta.to.global.u64 	%rd5, %rd4;
+	div.s32 	%r2, %r1, %r8;
+	rem.s32 	%r3, %r1, %r8;
+	setp.lt.s32	%p2, %r3, %r6;
+	mul.wide.s32 	%rd6, %r1, 8;
+	add.s64 	%rd1, %rd5, %rd6;
+	@%p2 bra 	BB126_3;
+	bra.uni 	BB126_2;
+
+BB126_3:
+	cvta.to.global.u64 	%rd10, %rd2;
+	mul.lo.s32 	%r15, %r6, %r4;
+	mad.lo.s32 	%r16, %r2, %r7, %r15;
+	add.s32 	%r17, %r16, %r3;
+	mul.wide.s32 	%rd11, %r17, 8;
+	add.s64 	%rd12, %rd10, %rd11;
+	ld.global.f64 	%fd2, [%rd12];
+	st.global.f64 	[%rd1], %fd2;
+	bra.uni 	BB126_4;
+
+BB126_2:
+	cvta.to.global.u64 	%rd7, %rd3;
+	sub.s32 	%r13, %r3, %r6;
+	mad.lo.s32 	%r14, %r2, %r5, %r13;
+	mul.wide.s32 	%rd8, %r14, 8;
+	add.s64 	%rd9, %rd7, %rd8;
+	ld.global.f64 	%fd1, [%rd9];
+	st.global.f64 	[%rd1], %fd1;
+
+BB126_4:
+	ret;
+}
+
+	// .globl	prepareInputNNLstm_f
+.visible .entry prepareInputNNLstm_f(
+	.param .u64 prepareInputNNLstm_f_param_0,
+	.param .u64 prepareInputNNLstm_f_param_1,
+	.param .u64 prepareInputNNLstm_f_param_2,
+	.param .u32 prepareInputNNLstm_f_param_3,
+	.param .u32 prepareInputNNLstm_f_param_4,
+	.param .u32 prepareInputNNLstm_f_param_5,
+	.param .u32 prepareInputNNLstm_f_param_6,
+	.param .u32 prepareInputNNLstm_f_param_7,
+	.param .u32 prepareInputNNLstm_f_param_8
+)
+{
+	.reg .pred 	%p<3>;
+	.reg .f32 	%f<3>;
+	.reg .b32 	%r<18>;
+	.reg .b64 	%rd<13>;
+
+
+	ld.param.u64 	%rd2, [prepareInputNNLstm_f_param_0];
+	ld.param.u64 	%rd3, [prepareInputNNLstm_f_param_1];
+	ld.param.u64 	%rd4, [prepareInputNNLstm_f_param_2];
+	ld.param.u32 	%r4, [prepareInputNNLstm_f_param_3];
+	ld.param.u32 	%r5, [prepareInputNNLstm_f_param_4];
+	ld.param.u32 	%r6, [prepareInputNNLstm_f_param_5];
+	ld.param.u32 	%r7, [prepareInputNNLstm_f_param_6];
+	ld.param.u32 	%r8, [prepareInputNNLstm_f_param_7];
+	ld.param.u32 	%r9, [prepareInputNNLstm_f_param_8];
+	mov.u32 	%r10, %ntid.x;
+	mov.u32 	%r11, %ctaid.x;
+	mov.u32 	%r12, %tid.x;
+	mad.lo.s32 	%r1, %r10, %r11, %r12;
+	setp.ge.u32	%p1, %r1, %r9;
+	@%p1 bra 	BB127_4;
+
+	cvta.to.global.u64 	%rd5, %rd4;
+	div.s32 	%r2, %r1, %r8;
+	rem.s32 	%r3, %r1, %r8;
+	setp.lt.s32	%p2, %r3, %r6;
+	mul.wide.s32 	%rd6, %r1, 4;
+	add.s64 	%rd1, %rd5, %rd6;
+	@%p2 bra 	BB127_3;
+	bra.uni 	BB127_2;
+
+BB127_3:
+	cvta.to.global.u64 	%rd10, %rd2;
+	mul.lo.s32 	%r15, %r6, %r4;
+	mad.lo.s32 	%r16, %r2, %r7, %r15;
+	add.s32 	%r17, %r16, %r3;
+	mul.wide.s32 	%rd11, %r17, 4;
+	add.s64 	%rd12, %rd10, %rd11;
+	ld.global.f32 	%f2, [%rd12];
+	st.global.f32 	[%rd1], %f2;
+	bra.uni 	BB127_4;
+
+BB127_2:
+	cvta.to.global.u64 	%rd7, %rd3;
+	sub.s32 	%r13, %r3, %r6;
+	mad.lo.s32 	%r14, %r2, %r5, %r13;
+	mul.wide.s32 	%rd8, %r14, 4;
+	add.s64 	%rd9, %rd7, %rd8;
+	ld.global.f32 	%f1, [%rd9];
+	st.global.f32 	[%rd1], %f1;
+
+BB127_4:
+	ret;
+}
+
+	// .globl	squashIFOG_d
+.visible .entry squashIFOG_d(
+	.param .u64 squashIFOG_d_param_0,
+	.param .u64 squashIFOG_d_param_1,
+	.param .u32 squashIFOG_d_param_2,
+	.param .u32 squashIFOG_d_param_3
+)
+{
+	.reg .pred 	%p<7>;
+	.reg .b32 	%r<32>;
+	.reg .f64 	%fd<152>;
+	.reg .b64 	%rd<9>;
+
+
+	ld.param.u64 	%rd2, [squashIFOG_d_param_0];
+	ld.param.u64 	%rd3, [squashIFOG_d_param_1];
+	ld.param.u32 	%r6, [squashIFOG_d_param_2];
+	ld.param.u32 	%r7, [squashIFOG_d_param_3];
+	mov.u32 	%r8, %ctaid.x;
+	mov.u32 	%r9, %ntid.x;
+	mov.u32 	%r10, %tid.x;
+	mad.lo.s32 	%r1, %r9, %r8, %r10;
+	setp.ge.u32	%p1, %r1, %r7;
+	@%p1 bra 	BB128_11;
+
+	cvta.to.global.u64 	%rd4, %rd2;
+	shl.b32 	%r11, %r6, 2;
+	rem.s32 	%r12, %r1, %r11;
+	mul.wide.s32 	%rd5, %r1, 8;
+	add.s64 	%rd1, %rd4, %rd5;
+	cvta.to.global.u64 	%rd6, %rd3;
+	mul.wide.s32 	%rd7, %r12, 8;
+	add.s64 	%rd8, %rd6, %rd7;
+	ld.global.f64 	%fd14, [%rd8];
+	ld.global.f64 	%fd15, [%rd1];
+	add.f64 	%fd1, %fd15, %fd14;
+	mul.lo.s32 	%r13, %r6, 3;
+	setp.lt.s32	%p2, %r12, %r13;
+	@%p2 bra 	BB128_6;
+	bra.uni 	BB128_2;
+
+BB128_6:
+	mul.f64 	%fd7, %fd1, 0d3FE0000000000000;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r4}, %fd7;
+	}
+	and.b32  	%r5, %r4, 2147483647;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r23, %temp}, %fd7;
+	}
+	mov.b64 	%fd8, {%r23, %r5};
+	setp.ltu.f64	%p5, %fd8, 0d3FE1C7A398201CD6;
+	@%p5 bra 	BB128_8;
+	bra.uni 	BB128_7;
+
+BB128_8:
+	mul.f64 	%fd127, %fd7, %fd7;
+	mov.f64 	%fd128, 0dBF2B9093D89F0E23;
+	mov.f64 	%fd129, 0d3F0ABFFC9B5786C4;
+	fma.rn.f64 	%fd130, %fd129, %fd127, %fd128;
+	mov.f64 	%fd131, 0d3F42FA2744C30B61;
+	fma.rn.f64 	%fd132, %fd130, %fd127, %fd131;
+	mov.f64 	%fd133, 0dBF57CF3B9C1E491D;
+	fma.rn.f64 	%fd134, %fd132, %fd127, %fd133;
+	mov.f64 	%fd135, 0d3F6D6C61D450119A;
+	fma.rn.f64 	%fd136, %fd134, %fd127, %fd135;
+	mov.f64 	%fd137, 0dBF8226DDD44294F5;
+	fma.rn.f64 	%fd138, %fd136, %fd127, %fd137;
+	mov.f64 	%fd139, 0d3F9664F45C2B04A6;
+	fma.rn.f64 	%fd140, %fd138, %fd127, %fd139;
+	mov.f64 	%fd141, 0dBFABA1BA1AD70754;
+	fma.rn.f64 	%fd142, %fd140, %fd127, %fd141;
+	mov.f64 	%fd143, 0d3FC111111110295E;
+	fma.rn.f64 	%fd144, %fd142, %fd127, %fd143;
+	mov.f64 	%fd145, 0dBFD555555555549F;
+	fma.rn.f64 	%fd146, %fd144, %fd127, %fd145;
+	mul.f64 	%fd147, %fd127, %fd146;
+	fma.rn.f64 	%fd150, %fd147, %fd7, %fd7;
+	bra.uni 	BB128_9;
+
+BB128_2:
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r2}, %fd1;
+	}
+	and.b32  	%r3, %r2, 2147483647;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r14, %temp}, %fd1;
+	}
+	mov.b64 	%fd2, {%r14, %r3};
+	setp.ltu.f64	%p3, %fd2, 0d3FE1C7A398201CD6;
+	@%p3 bra 	BB128_4;
+	bra.uni 	BB128_3;
+
+BB128_4:
+	mul.f64 	%fd61, %fd1, %fd1;
+	mov.f64 	%fd62, 0dBF2B9093D89F0E23;
+	mov.f64 	%fd63, 0d3F0ABFFC9B5786C4;
+	fma.rn.f64 	%fd64, %fd63, %fd61, %fd62;
+	mov.f64 	%fd65, 0d3F42FA2744C30B61;
+	fma.rn.f64 	%fd66, %fd64, %fd61, %fd65;
+	mov.f64 	%fd67, 0dBF57CF3B9C1E491D;
+	fma.rn.f64 	%fd68, %fd66, %fd61, %fd67;
+	mov.f64 	%fd69, 0d3F6D6C61D450119A;
+	fma.rn.f64 	%fd70, %fd68, %fd61, %fd69;
+	mov.f64 	%fd71, 0dBF8226DDD44294F5;
+	fma.rn.f64 	%fd72, %fd70, %fd61, %fd71;
+	mov.f64 	%fd73, 0d3F9664F45C2B04A6;
+	fma.rn.f64 	%fd74, %fd72, %fd61, %fd73;
+	mov.f64 	%fd75, 0dBFABA1BA1AD70754;
+	fma.rn.f64 	%fd76, %fd74, %fd61, %fd75;
+	mov.f64 	%fd77, 0d3FC111111110295E;
+	fma.rn.f64 	%fd78, %fd76, %fd61, %fd77;
+	mov.f64 	%fd79, 0dBFD555555555549F;
+	fma.rn.f64 	%fd80, %fd78, %fd61, %fd79;
+	mul.f64 	%fd81, %fd61, %fd80;
+	fma.rn.f64 	%fd149, %fd81, %fd1, %fd1;
+	bra.uni 	BB128_5;
+
+BB128_7:
+	add.f64 	%fd82, %fd8, %fd8;
+	mov.f64 	%fd83, 0d4338000000000000;
+	mov.f64 	%fd84, 0d3FF71547652B82FE;
+	fma.rn.f64 	%fd85, %fd82, %fd84, %fd83;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r24, %temp}, %fd85;
+	}
+	mov.f64 	%fd86, 0dC338000000000000;
+	add.rn.f64 	%fd87, %fd85, %fd86;
+	mov.f64 	%fd88, 0dBFE62E42FEFA39EF;
+	fma.rn.f64 	%fd89, %fd87, %fd88, %fd82;
+	mov.f64 	%fd90, 0dBC7ABC9E3B39803F;
+	fma.rn.f64 	%fd91, %fd87, %fd90, %fd89;
+	mov.f64 	%fd92, 0d3E5AF86D8EBD13CD;
+	mov.f64 	%fd93, 0d3E21F4076ACD15B6;
+	fma.rn.f64 	%fd94, %fd93, %fd91, %fd92;
+	mov.f64 	%fd95, 0d3E927E5092BA033D;
+	fma.rn.f64 	%fd96, %fd94, %fd91, %fd95;
+	mov.f64 	%fd97, 0d3EC71DDE6C5F9DA1;
+	fma.rn.f64 	%fd98, %fd96, %fd91, %fd97;
+	mov.f64 	%fd99, 0d3EFA01A018D034E6;
+	fma.rn.f64 	%fd100, %fd98, %fd91, %fd99;
+	mov.f64 	%fd101, 0d3F2A01A01B3B6940;
+	fma.rn.f64 	%fd102, %fd100, %fd91, %fd101;
+	mov.f64 	%fd103, 0d3F56C16C16C1B5DD;
+	fma.rn.f64 	%fd104, %fd102, %fd91, %fd103;
+	mov.f64 	%fd105, 0d3F8111111110F74D;
+	fma.rn.f64 	%fd106, %fd104, %fd91, %fd105;
+	mov.f64 	%fd107, 0d3FA555555555554D;
+	fma.rn.f64 	%fd108, %fd106, %fd91, %fd107;
+	mov.f64 	%fd109, 0d3FC5555555555557;
+	fma.rn.f64 	%fd110, %fd108, %fd91, %fd109;
+	mov.f64 	%fd111, 0d3FE0000000000000;
+	fma.rn.f64 	%fd112, %fd110, %fd91, %fd111;
+	mul.f64 	%fd113, %fd91, %fd112;
+	fma.rn.f64 	%fd114, %fd113, %fd91, %fd91;
+	shl.b32 	%r25, %r24, 20;
+	add.s32 	%r26, %r25, 1072693248;
+	mov.u32 	%r27, 0;
+	mov.b64 	%fd115, {%r27, %r26};
+	fma.rn.f64 	%fd116, %fd114, %fd115, %fd115;
+	add.f64 	%fd117, %fd116, 0d3FF0000000000000;
+	rcp.approx.ftz.f64 	%fd118, %fd117;
+	neg.f64 	%fd119, %fd117;
+	mov.f64 	%fd120, 0d3FF0000000000000;
+	fma.rn.f64 	%fd121, %fd119, %fd118, %fd120;
+	fma.rn.f64 	%fd122, %fd121, %fd121, %fd121;
+	fma.rn.f64 	%fd123, %fd122, %fd118, %fd118;
+	neg.f64 	%fd124, %fd123;
+	mov.f64 	%fd125, 0d4000000000000000;
+	fma.rn.f64 	%fd126, %fd125, %fd124, %fd120;
+	setp.gt.u32	%p6, %r5, 1077936127;
+	selp.f64	%fd150, 0d3FF0000000000000, %fd126, %p6;
+
+BB128_9:
+	and.b32  	%r28, %r4, -2147483648;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r29}, %fd150;
+	}
+	or.b32  	%r30, %r29, %r28;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r31, %temp}, %fd150;
+	}
+	mov.b64 	%fd148, {%r31, %r30};
+	fma.rn.f64 	%fd151, %fd148, 0d3FE0000000000000, 0d3FE0000000000000;
+	bra.uni 	BB128_10;
+
+BB128_3:
+	add.f64 	%fd16, %fd2, %fd2;
+	mov.f64 	%fd17, 0d4338000000000000;
+	mov.f64 	%fd18, 0d3FF71547652B82FE;
+	fma.rn.f64 	%fd19, %fd16, %fd18, %fd17;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r15, %temp}, %fd19;
+	}
+	mov.f64 	%fd20, 0dC338000000000000;
+	add.rn.f64 	%fd21, %fd19, %fd20;
+	mov.f64 	%fd22, 0dBFE62E42FEFA39EF;
+	fma.rn.f64 	%fd23, %fd21, %fd22, %fd16;
+	mov.f64 	%fd24, 0dBC7ABC9E3B39803F;
+	fma.rn.f64 	%fd25, %fd21, %fd24, %fd23;
+	mov.f64 	%fd26, 0d3E5AF86D8EBD13CD;
+	mov.f64 	%fd27, 0d3E21F4076ACD15B6;
+	fma.rn.f64 	%fd28, %fd27, %fd25, %fd26;
+	mov.f64 	%fd29, 0d3E927E5092BA033D;
+	fma.rn.f64 	%fd30, %fd28, %fd25, %fd29;
+	mov.f64 	%fd31, 0d3EC71DDE6C5F9DA1;
+	fma.rn.f64 	%fd32, %fd30, %fd25, %fd31;
+	mov.f64 	%fd33, 0d3EFA01A018D034E6;
+	fma.rn.f64 	%fd34, %fd32, %fd25, %fd33;
+	mov.f64 	%fd35, 0d3F2A01A01B3B6940;
+	fma.rn.f64 	%fd36, %fd34, %fd25, %fd35;
+	mov.f64 	%fd37, 0d3F56C16C16C1B5DD;
+	fma.rn.f64 	%fd38, %fd36, %fd25, %fd37;
+	mov.f64 	%fd39, 0d3F8111111110F74D;
+	fma.rn.f64 	%fd40, %fd38, %fd25, %fd39;
+	mov.f64 	%fd41, 0d3FA555555555554D;
+	fma.rn.f64 	%fd42, %fd40, %fd25, %fd41;
+	mov.f64 	%fd43, 0d3FC5555555555557;
+	fma.rn.f64 	%fd44, %fd42, %fd25, %fd43;
+	mov.f64 	%fd45, 0d3FE0000000000000;
+	fma.rn.f64 	%fd46, %fd44, %fd25, %fd45;
+	mul.f64 	%fd47, %fd25, %fd46;
+	fma.rn.f64 	%fd48, %fd47, %fd25, %fd25;
+	shl.b32 	%r16, %r15, 20;
+	add.s32 	%r17, %r16, 1072693248;
+	mov.u32 	%r18, 0;
+	mov.b64 	%fd49, {%r18, %r17};
+	fma.rn.f64 	%fd50, %fd48, %fd49, %fd49;
+	add.f64 	%fd51, %fd50, 0d3FF0000000000000;
+	rcp.approx.ftz.f64 	%fd52, %fd51;
+	neg.f64 	%fd53, %fd51;
+	mov.f64 	%fd54, 0d3FF0000000000000;
+	fma.rn.f64 	%fd55, %fd53, %fd52, %fd54;
+	fma.rn.f64 	%fd56, %fd55, %fd55, %fd55;
+	fma.rn.f64 	%fd57, %fd56, %fd52, %fd52;
+	neg.f64 	%fd58, %fd57;
+	mov.f64 	%fd59, 0d4000000000000000;
+	fma.rn.f64 	%fd60, %fd59, %fd58, %fd54;
+	setp.gt.u32	%p4, %r3, 1077936127;
+	selp.f64	%fd149, 0d3FF0000000000000, %fd60, %p4;
+
+BB128_5:
+	and.b32  	%r19, %r2, -2147483648;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r20}, %fd149;
+	}
+	or.b32  	%r21, %r20, %r19;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r22, %temp}, %fd149;
+	}
+	mov.b64 	%fd151, {%r22, %r21};
+
+BB128_10:
+	st.global.f64 	[%rd1], %fd151;
+
+BB128_11:
+	ret;
+}
+
+	// .globl	squashIFOG_f
+.visible .entry squashIFOG_f(
+	.param .u64 squashIFOG_f_param_0,
+	.param .u64 squashIFOG_f_param_1,
+	.param .u32 squashIFOG_f_param_2,
+	.param .u32 squashIFOG_f_param_3
+)
+{
+	.reg .pred 	%p<8>;
+	.reg .f32 	%f<36>;
+	.reg .b32 	%r<26>;
+	.reg .f64 	%fd<76>;
+	.reg .b64 	%rd<9>;
+
+
+	ld.param.u64 	%rd2, [squashIFOG_f_param_0];
+	ld.param.u64 	%rd3, [squashIFOG_f_param_1];
+	ld.param.u32 	%r4, [squashIFOG_f_param_2];
+	ld.param.u32 	%r5, [squashIFOG_f_param_3];
+	mov.u32 	%r6, %ctaid.x;
+	mov.u32 	%r7, %ntid.x;
+	mov.u32 	%r8, %tid.x;
+	mad.lo.s32 	%r1, %r7, %r6, %r8;
+	setp.ge.u32	%p1, %r1, %r5;
+	@%p1 bra 	BB129_10;
+
+	cvta.to.global.u64 	%rd4, %rd2;
+	shl.b32 	%r9, %r4, 2;
+	rem.s32 	%r10, %r1, %r9;
+	mul.wide.s32 	%rd5, %r1, 4;
+	add.s64 	%rd1, %rd4, %rd5;
+	cvta.to.global.u64 	%rd6, %rd3;
+	mul.wide.s32 	%rd7, %r10, 4;
+	add.s64 	%rd8, %rd6, %rd7;
+	ld.global.f32 	%f7, [%rd8];
+	ld.global.f32 	%f8, [%rd1];
+	add.f32 	%f1, %f8, %f7;
+	mul.lo.s32 	%r11, %r4, 3;
+	setp.lt.s32	%p2, %r10, %r11;
+	@%p2 bra 	BB129_5;
+	bra.uni 	BB129_2;
+
+BB129_5:
+	cvt.f64.f32	%fd6, %f1;
+	mul.f64 	%fd1, %fd6, 0d3FE0000000000000;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r2}, %fd1;
+	}
+	and.b32  	%r3, %r2, 2147483647;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r17, %temp}, %fd1;
+	}
+	mov.b64 	%fd2, {%r17, %r3};
+	setp.ltu.f64	%p6, %fd2, 0d3FE1C7A398201CD6;
+	@%p6 bra 	BB129_7;
+	bra.uni 	BB129_6;
+
+BB129_7:
+	mul.f64 	%fd52, %fd1, %fd1;
+	mov.f64 	%fd53, 0dBF2B9093D89F0E23;
+	mov.f64 	%fd54, 0d3F0ABFFC9B5786C4;
+	fma.rn.f64 	%fd55, %fd54, %fd52, %fd53;
+	mov.f64 	%fd56, 0d3F42FA2744C30B61;
+	fma.rn.f64 	%fd57, %fd55, %fd52, %fd56;
+	mov.f64 	%fd58, 0dBF57CF3B9C1E491D;
+	fma.rn.f64 	%fd59, %fd57, %fd52, %fd58;
+	mov.f64 	%fd60, 0d3F6D6C61D450119A;
+	fma.rn.f64 	%fd61, %fd59, %fd52, %fd60;
+	mov.f64 	%fd62, 0dBF8226DDD44294F5;
+	fma.rn.f64 	%fd63, %fd61, %fd52, %fd62;
+	mov.f64 	%fd64, 0d3F9664F45C2B04A6;
+	fma.rn.f64 	%fd65, %fd63, %fd52, %fd64;
+	mov.f64 	%fd66, 0dBFABA1BA1AD70754;
+	fma.rn.f64 	%fd67, %fd65, %fd52, %fd66;
+	mov.f64 	%fd68, 0d3FC111111110295E;
+	fma.rn.f64 	%fd69, %fd67, %fd52, %fd68;
+	mov.f64 	%fd70, 0dBFD555555555549F;
+	fma.rn.f64 	%fd71, %fd69, %fd52, %fd70;
+	mul.f64 	%fd72, %fd52, %fd71;
+	fma.rn.f64 	%fd75, %fd72, %fd1, %fd1;
+	bra.uni 	BB129_8;
+
+BB129_2:
+	abs.f32 	%f2, %f1;
+	setp.ltu.f32	%p3, %f2, 0f3F0CCCCD;
+	@%p3 bra 	BB129_4;
+	bra.uni 	BB129_3;
+
+BB129_4:
+	mul.f32 	%f24, %f1, %f1;
+	mov.f32 	%f25, 0fBD57BE66;
+	mov.f32 	%f26, 0f3C86A81B;
+	fma.rn.f32 	%f27, %f26, %f24, %f25;
+	mov.f32 	%f28, 0f3E08677B;
+	fma.rn.f32 	%f29, %f27, %f24, %f28;
+	mov.f32 	%f30, 0fBEAAAA29;
+	fma.rn.f32 	%f31, %f29, %f24, %f30;
+	mul.f32 	%f32, %f24, %f31;
+	fma.rn.f32 	%f33, %f32, %f1, %f1;
+	add.f32 	%f34, %f1, %f1;
+	setp.eq.f32	%p5, %f1, 0f00000000;
+	selp.f32	%f35, %f34, %f33, %p5;
+	bra.uni 	BB129_9;
+
+BB129_6:
+	add.f64 	%fd7, %fd2, %fd2;
+	mov.f64 	%fd8, 0d4338000000000000;
+	mov.f64 	%fd9, 0d3FF71547652B82FE;
+	fma.rn.f64 	%fd10, %fd7, %fd9, %fd8;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r18, %temp}, %fd10;
+	}
+	mov.f64 	%fd11, 0dC338000000000000;
+	add.rn.f64 	%fd12, %fd10, %fd11;
+	mov.f64 	%fd13, 0dBFE62E42FEFA39EF;
+	fma.rn.f64 	%fd14, %fd12, %fd13, %fd7;
+	mov.f64 	%fd15, 0dBC7ABC9E3B39803F;
+	fma.rn.f64 	%fd16, %fd12, %fd15, %fd14;
+	mov.f64 	%fd17, 0d3E5AF86D8EBD13CD;
+	mov.f64 	%fd18, 0d3E21F4076ACD15B6;
+	fma.rn.f64 	%fd19, %fd18, %fd16, %fd17;
+	mov.f64 	%fd20, 0d3E927E5092BA033D;
+	fma.rn.f64 	%fd21, %fd19, %fd16, %fd20;
+	mov.f64 	%fd22, 0d3EC71DDE6C5F9DA1;
+	fma.rn.f64 	%fd23, %fd21, %fd16, %fd22;
+	mov.f64 	%fd24, 0d3EFA01A018D034E6;
+	fma.rn.f64 	%fd25, %fd23, %fd16, %fd24;
+	mov.f64 	%fd26, 0d3F2A01A01B3B6940;
+	fma.rn.f64 	%fd27, %fd25, %fd16, %fd26;
+	mov.f64 	%fd28, 0d3F56C16C16C1B5DD;
+	fma.rn.f64 	%fd29, %fd27, %fd16, %fd28;
+	mov.f64 	%fd30, 0d3F8111111110F74D;
+	fma.rn.f64 	%fd31, %fd29, %fd16, %fd30;
+	mov.f64 	%fd32, 0d3FA555555555554D;
+	fma.rn.f64 	%fd33, %fd31, %fd16, %fd32;
+	mov.f64 	%fd34, 0d3FC5555555555557;
+	fma.rn.f64 	%fd35, %fd33, %fd16, %fd34;
+	mov.f64 	%fd36, 0d3FE0000000000000;
+	fma.rn.f64 	%fd37, %fd35, %fd16, %fd36;
+	mul.f64 	%fd38, %fd16, %fd37;
+	fma.rn.f64 	%fd39, %fd38, %fd16, %fd16;
+	shl.b32 	%r19, %r18, 20;
+	add.s32 	%r20, %r19, 1072693248;
+	mov.u32 	%r21, 0;
+	mov.b64 	%fd40, {%r21, %r20};
+	fma.rn.f64 	%fd41, %fd39, %fd40, %fd40;
+	add.f64 	%fd42, %fd41, 0d3FF0000000000000;
+	rcp.approx.ftz.f64 	%fd43, %fd42;
+	neg.f64 	%fd44, %fd42;
+	mov.f64 	%fd45, 0d3FF0000000000000;
+	fma.rn.f64 	%fd46, %fd44, %fd43, %fd45;
+	fma.rn.f64 	%fd47, %fd46, %fd46, %fd46;
+	fma.rn.f64 	%fd48, %fd47, %fd43, %fd43;
+	neg.f64 	%fd49, %fd48;
+	mov.f64 	%fd50, 0d4000000000000000;
+	fma.rn.f64 	%fd51, %fd50, %fd49, %fd45;
+	setp.gt.u32	%p7, %r3, 1077936127;
+	selp.f64	%fd75, 0d3FF0000000000000, %fd51, %p7;
+
+BB129_8:
+	and.b32  	%r22, %r2, -2147483648;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r23}, %fd75;
+	}
+	or.b32  	%r24, %r23, %r22;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r25, %temp}, %fd75;
+	}
+	mov.b64 	%fd73, {%r25, %r24};
+	fma.rn.f64 	%fd74, %fd73, 0d3FE0000000000000, 0d3FE0000000000000;
+	cvt.rn.f32.f64	%f35, %fd74;
+	bra.uni 	BB129_9;
+
+BB129_3:
+	add.f32 	%f11, %f2, %f2;
+	mul.f32 	%f12, %f11, 0f3FB8AA3B;
+	cvt.rzi.f32.f32	%f13, %f12;
+	mov.f32 	%f14, 0fBF317200;
+	fma.rn.f32 	%f15, %f13, %f14, %f11;
+	mov.f32 	%f16, 0fB5BFBE8E;
+	fma.rn.f32 	%f17, %f13, %f16, %f15;
+	mul.f32 	%f18, %f17, 0f3FB8AA3B;
+	ex2.approx.ftz.f32 	%f19, %f18;
+	ex2.approx.f32 	%f20, %f13;
+	mov.f32 	%f21, 0f3F800000;
+	fma.rn.f32 	%f10, %f19, %f20, %f21;
+	// inline asm
+	rcp.approx.ftz.f32 %f9,%f10;
+	// inline asm
+	mov.f32 	%f22, 0fC0000000;
+	fma.rn.f32 	%f23, %f9, %f22, %f21;
+	mov.b32 	 %r12, %f23;
+	setp.ltu.f32	%p4, %f2, 0f42B00000;
+	selp.b32	%r13, %r12, 1065353216, %p4;
+	mov.b32 	 %r14, %f1;
+	and.b32  	%r15, %r14, -2147483648;
+	or.b32  	%r16, %r13, %r15;
+	mov.b32 	 %f35, %r16;
+
+BB129_9:
+	st.global.f32 	[%rd1], %f35;
+
+BB129_10:
+	ret;
+}
+
+	// .globl	postProcessNNLstmForward_d
+.visible .entry postProcessNNLstmForward_d(
+	.param .u64 postProcessNNLstmForward_d_param_0,
+	.param .u64 postProcessNNLstmForward_d_param_1,
+	.param .u64 postProcessNNLstmForward_d_param_2,
+	.param .u64 postProcessNNLstmForward_d_param_3,
+	.param .u64 postProcessNNLstmForward_d_param_4,
+	.param .u64 postProcessNNLstmForward_d_param_5,
+	.param .u64 postProcessNNLstmForward_d_param_6,
+	.param .u32 postProcessNNLstmForward_d_param_7,
+	.param .u32 postProcessNNLstmForward_d_param_8,
+	.param .u32 postProcessNNLstmForward_d_param_9,
+	.param .u32 postProcessNNLstmForward_d_param_10,
+	.param .u32 postProcessNNLstmForward_d_param_11
+)
+{
+	.reg .pred 	%p<5>;
+	.reg .b32 	%r<41>;
+	.reg .f64 	%fd<81>;
+	.reg .b64 	%rd<34>;
+
+
+	ld.param.u64 	%rd2, [postProcessNNLstmForward_d_param_0];
+	ld.param.u64 	%rd5, [postProcessNNLstmForward_d_param_3];
+	ld.param.u64 	%rd6, [postProcessNNLstmForward_d_param_4];
+	ld.param.u64 	%rd8, [postProcessNNLstmForward_d_param_6];
+	ld.param.u32 	%r8, [postProcessNNLstmForward_d_param_10];
+	ld.param.u32 	%r9, [postProcessNNLstmForward_d_param_11];
+	mov.u32 	%r10, %ntid.x;
+	mov.u32 	%r11, %ctaid.x;
+	mov.u32 	%r12, %tid.x;
+	mad.lo.s32 	%r1, %r10, %r11, %r12;
+	setp.ge.u32	%p1, %r1, %r9;
+	@%p1 bra 	BB130_5;
+
+	cvta.to.global.u64 	%rd9, %rd2;
+	cvta.to.global.u64 	%rd10, %rd5;
+	div.s32 	%r2, %r1, %r8;
+	shl.b32 	%r13, %r8, 2;
+	rem.s32 	%r3, %r1, %r8;
+	mad.lo.s32 	%r14, %r2, %r13, %r3;
+	mul.wide.s32 	%rd11, %r14, 8;
+	add.s64 	%rd12, %rd9, %rd11;
+	shl.b32 	%r15, %r8, 3;
+	cvt.s64.s32	%rd13, %r15;
+	add.s64 	%rd14, %rd12, %rd13;
+	add.s64 	%rd15, %rd14, %rd13;
+	ld.global.f64 	%fd1, [%rd15];
+	add.s64 	%rd16, %rd15, %rd13;
+	mul.wide.s32 	%rd17, %r1, 8;
+	add.s64 	%rd1, %rd10, %rd17;
+	ld.global.f64 	%fd7, [%rd1];
+	ld.global.f64 	%fd8, [%rd14];
+	mul.f64 	%fd9, %fd8, %fd7;
+	ld.global.f64 	%fd10, [%rd16];
+	ld.global.f64 	%fd11, [%rd12];
+	fma.rn.f64 	%fd2, %fd11, %fd10, %fd9;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r4}, %fd2;
+	}
+	and.b32  	%r16, %r4, 2147483647;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r17, %temp}, %fd2;
+	}
+	mov.b64 	%fd3, {%r17, %r16};
+	setp.ltu.f64	%p2, %fd3, 0d3FE1C7A398201CD6;
+	@%p2 bra 	BB130_3;
+	bra.uni 	BB130_2;
+
+BB130_3:
+	mul.f64 	%fd57, %fd2, %fd2;
+	mov.f64 	%fd58, 0dBF2B9093D89F0E23;
+	mov.f64 	%fd59, 0d3F0ABFFC9B5786C4;
+	fma.rn.f64 	%fd60, %fd59, %fd57, %fd58;
+	mov.f64 	%fd61, 0d3F42FA2744C30B61;
+	fma.rn.f64 	%fd62, %fd60, %fd57, %fd61;
+	mov.f64 	%fd63, 0dBF57CF3B9C1E491D;
+	fma.rn.f64 	%fd64, %fd62, %fd57, %fd63;
+	mov.f64 	%fd65, 0d3F6D6C61D450119A;
+	fma.rn.f64 	%fd66, %fd64, %fd57, %fd65;
+	mov.f64 	%fd67, 0dBF8226DDD44294F5;
+	fma.rn.f64 	%fd68, %fd66, %fd57, %fd67;
+	mov.f64 	%fd69, 0d3F9664F45C2B04A6;
+	fma.rn.f64 	%fd70, %fd68, %fd57, %fd69;
+	mov.f64 	%fd71, 0dBFABA1BA1AD70754;
+	fma.rn.f64 	%fd72, %fd70, %fd57, %fd71;
+	mov.f64 	%fd73, 0d3FC111111110295E;
+	fma.rn.f64 	%fd74, %fd72, %fd57, %fd73;
+	mov.f64 	%fd75, 0dBFD555555555549F;
+	fma.rn.f64 	%fd76, %fd74, %fd57, %fd75;
+	mul.f64 	%fd77, %fd57, %fd76;
+	fma.rn.f64 	%fd80, %fd77, %fd2, %fd2;
+	bra.uni 	BB130_4;
+
+BB130_2:
+	add.f64 	%fd12, %fd3, %fd3;
+	mov.f64 	%fd13, 0d4338000000000000;
+	mov.f64 	%fd14, 0d3FF71547652B82FE;
+	fma.rn.f64 	%fd15, %fd12, %fd14, %fd13;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r18, %temp}, %fd15;
+	}
+	mov.f64 	%fd16, 0dC338000000000000;
+	add.rn.f64 	%fd17, %fd15, %fd16;
+	mov.f64 	%fd18, 0dBFE62E42FEFA39EF;
+	fma.rn.f64 	%fd19, %fd17, %fd18, %fd12;
+	mov.f64 	%fd20, 0dBC7ABC9E3B39803F;
+	fma.rn.f64 	%fd21, %fd17, %fd20, %fd19;
+	mov.f64 	%fd22, 0d3E5AF86D8EBD13CD;
+	mov.f64 	%fd23, 0d3E21F4076ACD15B6;
+	fma.rn.f64 	%fd24, %fd23, %fd21, %fd22;
+	mov.f64 	%fd25, 0d3E927E5092BA033D;
+	fma.rn.f64 	%fd26, %fd24, %fd21, %fd25;
+	mov.f64 	%fd27, 0d3EC71DDE6C5F9DA1;
+	fma.rn.f64 	%fd28, %fd26, %fd21, %fd27;
+	mov.f64 	%fd29, 0d3EFA01A018D034E6;
+	fma.rn.f64 	%fd30, %fd28, %fd21, %fd29;
+	mov.f64 	%fd31, 0d3F2A01A01B3B6940;
+	fma.rn.f64 	%fd32, %fd30, %fd21, %fd31;
+	mov.f64 	%fd33, 0d3F56C16C16C1B5DD;
+	fma.rn.f64 	%fd34, %fd32, %fd21, %fd33;
+	mov.f64 	%fd35, 0d3F8111111110F74D;
+	fma.rn.f64 	%fd36, %fd34, %fd21, %fd35;
+	mov.f64 	%fd37, 0d3FA555555555554D;
+	fma.rn.f64 	%fd38, %fd36, %fd21, %fd37;
+	mov.f64 	%fd39, 0d3FC5555555555557;
+	fma.rn.f64 	%fd40, %fd38, %fd21, %fd39;
+	mov.f64 	%fd41, 0d3FE0000000000000;
+	fma.rn.f64 	%fd42, %fd40, %fd21, %fd41;
+	mul.f64 	%fd43, %fd21, %fd42;
+	fma.rn.f64 	%fd44, %fd43, %fd21, %fd21;
+	shl.b32 	%r19, %r18, 20;
+	add.s32 	%r20, %r19, 1072693248;
+	mov.u32 	%r21, 0;
+	mov.b64 	%fd45, {%r21, %r20};
+	fma.rn.f64 	%fd46, %fd44, %fd45, %fd45;
+	add.f64 	%fd47, %fd46, 0d3FF0000000000000;
+	rcp.approx.ftz.f64 	%fd48, %fd47;
+	neg.f64 	%fd49, %fd47;
+	mov.f64 	%fd50, 0d3FF0000000000000;
+	fma.rn.f64 	%fd51, %fd49, %fd48, %fd50;
+	fma.rn.f64 	%fd52, %fd51, %fd51, %fd51;
+	fma.rn.f64 	%fd53, %fd52, %fd48, %fd48;
+	neg.f64 	%fd54, %fd53;
+	mov.f64 	%fd55, 0d4000000000000000;
+	fma.rn.f64 	%fd56, %fd55, %fd54, %fd50;
+	setp.gt.u32	%p3, %r16, 1077936127;
+	selp.f64	%fd80, 0d3FF0000000000000, %fd56, %p3;
+
+BB130_4:
+	ld.param.u64 	%rd33, [postProcessNNLstmForward_d_param_5];
+	ld.param.u64 	%rd32, [postProcessNNLstmForward_d_param_2];
+	ld.param.u64 	%rd31, [postProcessNNLstmForward_d_param_1];
+	ld.param.u32 	%r40, [postProcessNNLstmForward_d_param_11];
+	ld.param.u32 	%r39, [postProcessNNLstmForward_d_param_7];
+	ld.param.u32 	%r38, [postProcessNNLstmForward_d_param_8];
+	ld.param.u32 	%r37, [postProcessNNLstmForward_d_param_9];
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r36}, %fd2;
+	}
+	ld.param.u32 	%r35, [postProcessNNLstmForward_d_param_10];
+	and.b32  	%r23, %r36, -2147483648;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r24}, %fd80;
+	}
+	or.b32  	%r25, %r24, %r23;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r26, %temp}, %fd80;
+	}
+	mov.b64 	%fd78, {%r26, %r25};
+	mul.f64 	%fd79, %fd1, %fd78;
+	mad.lo.s32 	%r27, %r2, %r37, %r38;
+	mad.lo.s32 	%r28, %r27, %r35, %r3;
+	setp.eq.s32	%p4, %r39, 0;
+	selp.b32	%r33, %r1, %r28, %p4;
+	mad.lo.s32 	%r34, %r40, %r38, %r1;
+	cvta.to.global.u64 	%rd18, %rd31;
+	add.s64 	%rd20, %rd18, %rd17;
+	st.global.f64 	[%rd20], %fd2;
+	cvta.to.global.u64 	%rd21, %rd32;
+	add.s64 	%rd22, %rd21, %rd17;
+	st.global.f64 	[%rd22], %fd79;
+	st.global.f64 	[%rd1], %fd2;
+	cvta.to.global.u64 	%rd23, %rd33;
+	mul.wide.s32 	%rd24, %r34, 8;
+	add.s64 	%rd25, %rd23, %rd24;
+	st.global.f64 	[%rd25], %fd79;
+	cvta.to.global.u64 	%rd26, %rd8;
+	add.s64 	%rd27, %rd26, %rd24;
+	st.global.f64 	[%rd27], %fd2;
+	cvta.to.global.u64 	%rd28, %rd6;
+	mul.wide.s32 	%rd29, %r33, 8;
+	add.s64 	%rd30, %rd28, %rd29;
+	st.global.f64 	[%rd30], %fd79;
+
+BB130_5:
+	ret;
+}
+
+	// .globl	postProcessNNLstmForward_f
+.visible .entry postProcessNNLstmForward_f(
+	.param .u64 postProcessNNLstmForward_f_param_0,
+	.param .u64 postProcessNNLstmForward_f_param_1,
+	.param .u64 postProcessNNLstmForward_f_param_2,
+	.param .u64 postProcessNNLstmForward_f_param_3,
+	.param .u64 postProcessNNLstmForward_f_param_4,
+	.param .u64 postProcessNNLstmForward_f_param_5,
+	.param .u64 postProcessNNLstmForward_f_param_6,
+	.param .u32 postProcessNNLstmForward_f_param_7,
+	.param .u32 postProcessNNLstmForward_f_param_8,
+	.param .u32 postProcessNNLstmForward_f_param_9,
+	.param .u32 postProcessNNLstmForward_f_param_10,
+	.param .u32 postProcessNNLstmForward_f_param_11
+)
+{
+	.reg .pred 	%p<6>;
+	.reg .f32 	%f<40>;
+	.reg .b32 	%r<23>;
+	.reg .b64 	%rd<31>;
+
+
+	ld.param.u64 	%rd2, [postProcessNNLstmForward_f_param_0];
+	ld.param.u64 	%rd3, [postProcessNNLstmForward_f_param_1];
+	ld.param.u64 	%rd4, [postProcessNNLstmForward_f_param_2];
+	ld.param.u64 	%rd5, [postProcessNNLstmForward_f_param_3];
+	ld.param.u64 	%rd6, [postProcessNNLstmForward_f_param_4];
+	ld.param.u64 	%rd7, [postProcessNNLstmForward_f_param_5];
+	ld.param.u64 	%rd8, [postProcessNNLstmForward_f_param_6];
+	ld.param.u32 	%r4, [postProcessNNLstmForward_f_param_7];
+	ld.param.u32 	%r5, [postProcessNNLstmForward_f_param_8];
+	ld.param.u32 	%r6, [postProcessNNLstmForward_f_param_9];
+	ld.param.u32 	%r7, [postProcessNNLstmForward_f_param_10];
+	ld.param.u32 	%r8, [postProcessNNLstmForward_f_param_11];
+	mov.u32 	%r9, %ntid.x;
+	mov.u32 	%r10, %ctaid.x;
+	mov.u32 	%r11, %tid.x;
+	mad.lo.s32 	%r1, %r9, %r10, %r11;
+	setp.ge.u32	%p1, %r1, %r8;
+	@%p1 bra 	BB131_5;
+
+	cvta.to.global.u64 	%rd9, %rd2;
+	cvta.to.global.u64 	%rd10, %rd5;
+	div.s32 	%r2, %r1, %r7;
+	rem.s32 	%r3, %r1, %r7;
+	shl.b32 	%r12, %r7, 2;
+	mad.lo.s32 	%r13, %r2, %r12, %r3;
+	mul.wide.s32 	%rd11, %r13, 4;
+	add.s64 	%rd12, %rd9, %rd11;
+	cvt.s64.s32	%rd13, %r12;
+	add.s64 	%rd14, %rd12, %rd13;
+	add.s64 	%rd15, %rd14, %rd13;
+	ld.global.f32 	%f1, [%rd15];
+	add.s64 	%rd16, %rd15, %rd13;
+	mul.wide.s32 	%rd17, %r1, 4;
+	add.s64 	%rd1, %rd10, %rd17;
+	ld.global.f32 	%f7, [%rd1];
+	ld.global.f32 	%f8, [%rd14];
+	mul.f32 	%f9, %f8, %f7;
+	ld.global.f32 	%f10, [%rd16];
+	ld.global.f32 	%f11, [%rd12];
+	fma.rn.f32 	%f2, %f11, %f10, %f9;
+	abs.f32 	%f3, %f2;
+	setp.ltu.f32	%p2, %f3, 0f3F0CCCCD;
+	@%p2 bra 	BB131_3;
+	bra.uni 	BB131_2;
+
+BB131_3:
+	mul.f32 	%f27, %f2, %f2;
+	mov.f32 	%f28, 0fBD57BE66;
+	mov.f32 	%f29, 0f3C86A81B;
+	fma.rn.f32 	%f30, %f29, %f27, %f28;
+	mov.f32 	%f31, 0f3E08677B;
+	fma.rn.f32 	%f32, %f30, %f27, %f31;
+	mov.f32 	%f33, 0fBEAAAA29;
+	fma.rn.f32 	%f34, %f32, %f27, %f33;
+	mul.f32 	%f35, %f27, %f34;
+	fma.rn.f32 	%f36, %f35, %f2, %f2;
+	add.f32 	%f37, %f2, %f2;
+	setp.eq.f32	%p4, %f2, 0f00000000;
+	selp.f32	%f39, %f37, %f36, %p4;
+	bra.uni 	BB131_4;
+
+BB131_2:
+	add.f32 	%f14, %f3, %f3;
+	mul.f32 	%f15, %f14, 0f3FB8AA3B;
+	cvt.rzi.f32.f32	%f16, %f15;
+	mov.f32 	%f17, 0fBF317200;
+	fma.rn.f32 	%f18, %f16, %f17, %f14;
+	mov.f32 	%f19, 0fB5BFBE8E;
+	fma.rn.f32 	%f20, %f16, %f19, %f18;
+	mul.f32 	%f21, %f20, 0f3FB8AA3B;
+	ex2.approx.ftz.f32 	%f22, %f21;
+	ex2.approx.f32 	%f23, %f16;
+	mov.f32 	%f24, 0f3F800000;
+	fma.rn.f32 	%f13, %f22, %f23, %f24;
+	// inline asm
+	rcp.approx.ftz.f32 %f12,%f13;
+	// inline asm
+	mov.f32 	%f25, 0fC0000000;
+	fma.rn.f32 	%f26, %f12, %f25, %f24;
+	mov.b32 	 %r14, %f26;
+	setp.ltu.f32	%p3, %f3, 0f42B00000;
+	selp.b32	%r15, %r14, 1065353216, %p3;
+	mov.b32 	 %r16, %f2;
+	and.b32  	%r17, %r16, -2147483648;
+	or.b32  	%r18, %r15, %r17;
+	mov.b32 	 %f39, %r18;
+
+BB131_4:
+	mad.lo.s32 	%r19, %r2, %r6, %r5;
+	mad.lo.s32 	%r20, %r19, %r7, %r3;
+	setp.eq.s32	%p5, %r4, 0;
+	selp.b32	%r21, %r1, %r20, %p5;
+	mad.lo.s32 	%r22, %r8, %r5, %r1;
+	cvta.to.global.u64 	%rd18, %rd3;
+	add.s64 	%rd20, %rd18, %rd17;
+	st.global.f32 	[%rd20], %f2;
+	cvta.to.global.u64 	%rd21, %rd4;
+	add.s64 	%rd22, %rd21, %rd17;
+	mul.f32 	%f38, %f1, %f39;
+	st.global.f32 	[%rd22], %f38;
+	st.global.f32 	[%rd1], %f2;
+	cvta.to.global.u64 	%rd23, %rd7;
+	mul.wide.s32 	%rd24, %r22, 4;
+	add.s64 	%rd25, %rd23, %rd24;
+	st.global.f32 	[%rd25], %f38;
+	cvta.to.global.u64 	%rd26, %rd8;
+	add.s64 	%rd27, %rd26, %rd24;
+	st.global.f32 	[%rd27], %f2;
+	cvta.to.global.u64 	%rd28, %rd6;
+	mul.wide.s32 	%rd29, %r21, 4;
+	add.s64 	%rd30, %rd28, %rd29;
+	st.global.f32 	[%rd30], %f38;
+
+BB131_5:
+	ret;
+}
+
+	// .globl	postProcessNNLstmForwardSkipCache_d
+.visible .entry postProcessNNLstmForwardSkipCache_d(
+	.param .u64 postProcessNNLstmForwardSkipCache_d_param_0,
+	.param .u64 postProcessNNLstmForwardSkipCache_d_param_1,
+	.param .u64 postProcessNNLstmForwardSkipCache_d_param_2,
+	.param .u64 postProcessNNLstmForwardSkipCache_d_param_3,
+	.param .u64 postProcessNNLstmForwardSkipCache_d_param_4,
+	.param .u32 postProcessNNLstmForwardSkipCache_d_param_5,
+	.param .u32 postProcessNNLstmForwardSkipCache_d_param_6,
+	.param .u32 postProcessNNLstmForwardSkipCache_d_param_7,
+	.param .u32 postProcessNNLstmForwardSkipCache_d_param_8,
+	.param .u32 postProcessNNLstmForwardSkipCache_d_param_9
+)
+{
+	.reg .pred 	%p<5>;
+	.reg .b32 	%r<35>;
+	.reg .f64 	%fd<81>;
+	.reg .b64 	%rd<25>;
+
+
+	ld.param.u64 	%rd2, [postProcessNNLstmForwardSkipCache_d_param_0];
+	ld.param.u64 	%rd4, [postProcessNNLstmForwardSkipCache_d_param_2];
+	ld.param.u64 	%rd5, [postProcessNNLstmForwardSkipCache_d_param_3];
+	ld.param.u64 	%rd6, [postProcessNNLstmForwardSkipCache_d_param_4];
+	ld.param.u32 	%r8, [postProcessNNLstmForwardSkipCache_d_param_8];
+	ld.param.u32 	%r9, [postProcessNNLstmForwardSkipCache_d_param_9];
+	mov.u32 	%r10, %ntid.x;
+	mov.u32 	%r11, %ctaid.x;
+	mov.u32 	%r12, %tid.x;
+	mad.lo.s32 	%r1, %r10, %r11, %r12;
+	setp.ge.u32	%p1, %r1, %r9;
+	@%p1 bra 	BB132_5;
+
+	cvta.to.global.u64 	%rd7, %rd2;
+	cvta.to.global.u64 	%rd8, %rd5;
+	div.s32 	%r2, %r1, %r8;
+	shl.b32 	%r13, %r8, 2;
+	rem.s32 	%r3, %r1, %r8;
+	mad.lo.s32 	%r14, %r2, %r13, %r3;
+	mul.wide.s32 	%rd9, %r14, 8;
+	add.s64 	%rd10, %rd7, %rd9;
+	shl.b32 	%r15, %r8, 3;
+	cvt.s64.s32	%rd11, %r15;
+	add.s64 	%rd12, %rd10, %rd11;
+	add.s64 	%rd13, %rd12, %rd11;
+	ld.global.f64 	%fd1, [%rd13];
+	add.s64 	%rd14, %rd13, %rd11;
+	mul.wide.s32 	%rd15, %r1, 8;
+	add.s64 	%rd1, %rd8, %rd15;
+	ld.global.f64 	%fd7, [%rd1];
+	ld.global.f64 	%fd8, [%rd12];
+	mul.f64 	%fd9, %fd8, %fd7;
+	ld.global.f64 	%fd10, [%rd14];
+	ld.global.f64 	%fd11, [%rd10];
+	fma.rn.f64 	%fd2, %fd11, %fd10, %fd9;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r4}, %fd2;
+	}
+	and.b32  	%r16, %r4, 2147483647;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r17, %temp}, %fd2;
+	}
+	mov.b64 	%fd3, {%r17, %r16};
+	setp.ltu.f64	%p2, %fd3, 0d3FE1C7A398201CD6;
+	@%p2 bra 	BB132_3;
+	bra.uni 	BB132_2;
+
+BB132_3:
+	mul.f64 	%fd57, %fd2, %fd2;
+	mov.f64 	%fd58, 0dBF2B9093D89F0E23;
+	mov.f64 	%fd59, 0d3F0ABFFC9B5786C4;
+	fma.rn.f64 	%fd60, %fd59, %fd57, %fd58;
+	mov.f64 	%fd61, 0d3F42FA2744C30B61;
+	fma.rn.f64 	%fd62, %fd60, %fd57, %fd61;
+	mov.f64 	%fd63, 0dBF57CF3B9C1E491D;
+	fma.rn.f64 	%fd64, %fd62, %fd57, %fd63;
+	mov.f64 	%fd65, 0d3F6D6C61D450119A;
+	fma.rn.f64 	%fd66, %fd64, %fd57, %fd65;
+	mov.f64 	%fd67, 0dBF8226DDD44294F5;
+	fma.rn.f64 	%fd68, %fd66, %fd57, %fd67;
+	mov.f64 	%fd69, 0d3F9664F45C2B04A6;
+	fma.rn.f64 	%fd70, %fd68, %fd57, %fd69;
+	mov.f64 	%fd71, 0dBFABA1BA1AD70754;
+	fma.rn.f64 	%fd72, %fd70, %fd57, %fd71;
+	mov.f64 	%fd73, 0d3FC111111110295E;
+	fma.rn.f64 	%fd74, %fd72, %fd57, %fd73;
+	mov.f64 	%fd75, 0dBFD555555555549F;
+	fma.rn.f64 	%fd76, %fd74, %fd57, %fd75;
+	mul.f64 	%fd77, %fd57, %fd76;
+	fma.rn.f64 	%fd80, %fd77, %fd2, %fd2;
+	bra.uni 	BB132_4;
+
+BB132_2:
+	add.f64 	%fd12, %fd3, %fd3;
+	mov.f64 	%fd13, 0d4338000000000000;
+	mov.f64 	%fd14, 0d3FF71547652B82FE;
+	fma.rn.f64 	%fd15, %fd12, %fd14, %fd13;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r18, %temp}, %fd15;
+	}
+	mov.f64 	%fd16, 0dC338000000000000;
+	add.rn.f64 	%fd17, %fd15, %fd16;
+	mov.f64 	%fd18, 0dBFE62E42FEFA39EF;
+	fma.rn.f64 	%fd19, %fd17, %fd18, %fd12;
+	mov.f64 	%fd20, 0dBC7ABC9E3B39803F;
+	fma.rn.f64 	%fd21, %fd17, %fd20, %fd19;
+	mov.f64 	%fd22, 0d3E5AF86D8EBD13CD;
+	mov.f64 	%fd23, 0d3E21F4076ACD15B6;
+	fma.rn.f64 	%fd24, %fd23, %fd21, %fd22;
+	mov.f64 	%fd25, 0d3E927E5092BA033D;
+	fma.rn.f64 	%fd26, %fd24, %fd21, %fd25;
+	mov.f64 	%fd27, 0d3EC71DDE6C5F9DA1;
+	fma.rn.f64 	%fd28, %fd26, %fd21, %fd27;
+	mov.f64 	%fd29, 0d3EFA01A018D034E6;
+	fma.rn.f64 	%fd30, %fd28, %fd21, %fd29;
+	mov.f64 	%fd31, 0d3F2A01A01B3B6940;
+	fma.rn.f64 	%fd32, %fd30, %fd21, %fd31;
+	mov.f64 	%fd33, 0d3F56C16C16C1B5DD;
+	fma.rn.f64 	%fd34, %fd32, %fd21, %fd33;
+	mov.f64 	%fd35, 0d3F8111111110F74D;
+	fma.rn.f64 	%fd36, %fd34, %fd21, %fd35;
+	mov.f64 	%fd37, 0d3FA555555555554D;
+	fma.rn.f64 	%fd38, %fd36, %fd21, %fd37;
+	mov.f64 	%fd39, 0d3FC5555555555557;
+	fma.rn.f64 	%fd40, %fd38, %fd21, %fd39;
+	mov.f64 	%fd41, 0d3FE0000000000000;
+	fma.rn.f64 	%fd42, %fd40, %fd21, %fd41;
+	mul.f64 	%fd43, %fd21, %fd42;
+	fma.rn.f64 	%fd44, %fd43, %fd21, %fd21;
+	shl.b32 	%r19, %r18, 20;
+	add.s32 	%r20, %r19, 1072693248;
+	mov.u32 	%r21, 0;
+	mov.b64 	%fd45, {%r21, %r20};
+	fma.rn.f64 	%fd46, %fd44, %fd45, %fd45;
+	add.f64 	%fd47, %fd46, 0d3FF0000000000000;
+	rcp.approx.ftz.f64 	%fd48, %fd47;
+	neg.f64 	%fd49, %fd47;
+	mov.f64 	%fd50, 0d3FF0000000000000;
+	fma.rn.f64 	%fd51, %fd49, %fd48, %fd50;
+	fma.rn.f64 	%fd52, %fd51, %fd51, %fd51;
+	fma.rn.f64 	%fd53, %fd52, %fd48, %fd48;
+	neg.f64 	%fd54, %fd53;
+	mov.f64 	%fd55, 0d4000000000000000;
+	fma.rn.f64 	%fd56, %fd55, %fd54, %fd50;
+	setp.gt.u32	%p3, %r16, 1077936127;
+	selp.f64	%fd80, 0d3FF0000000000000, %fd56, %p3;
+
+BB132_4:
+	ld.param.u64 	%rd24, [postProcessNNLstmForwardSkipCache_d_param_1];
+	ld.param.u32 	%r34, [postProcessNNLstmForwardSkipCache_d_param_5];
+	ld.param.u32 	%r33, [postProcessNNLstmForwardSkipCache_d_param_6];
+	ld.param.u32 	%r32, [postProcessNNLstmForwardSkipCache_d_param_7];
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r31}, %fd2;
+	}
+	ld.param.u32 	%r30, [postProcessNNLstmForwardSkipCache_d_param_8];
+	and.b32  	%r23, %r31, -2147483648;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r24}, %fd80;
+	}
+	or.b32  	%r25, %r24, %r23;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r26, %temp}, %fd80;
+	}
+	mov.b64 	%fd78, {%r26, %r25};
+	mul.f64 	%fd79, %fd1, %fd78;
+	mad.lo.s32 	%r27, %r2, %r32, %r33;
+	mad.lo.s32 	%r28, %r27, %r30, %r3;
+	setp.eq.s32	%p4, %r34, 0;
+	selp.b32	%r29, %r1, %r28, %p4;
+	cvta.to.global.u64 	%rd16, %rd24;
+	add.s64 	%rd18, %rd16, %rd15;
+	st.global.f64 	[%rd18], %fd2;
+	cvta.to.global.u64 	%rd19, %rd4;
+	add.s64 	%rd20, %rd19, %rd15;
+	st.global.f64 	[%rd20], %fd79;
+	st.global.f64 	[%rd1], %fd2;
+	cvta.to.global.u64 	%rd21, %rd6;
+	mul.wide.s32 	%rd22, %r29, 8;
+	add.s64 	%rd23, %rd21, %rd22;
+	st.global.f64 	[%rd23], %fd79;
+
+BB132_5:
+	ret;
+}
+
+	// .globl	postProcessNNLstmForwardSkipCache_f
+.visible .entry postProcessNNLstmForwardSkipCache_f(
+	.param .u64 postProcessNNLstmForwardSkipCache_f_param_0,
+	.param .u64 postProcessNNLstmForwardSkipCache_f_param_1,
+	.param .u64 postProcessNNLstmForwardSkipCache_f_param_2,
+	.param .u64 postProcessNNLstmForwardSkipCache_f_param_3,
+	.param .u64 postProcessNNLstmForwardSkipCache_f_param_4,
+	.param .u32 postProcessNNLstmForwardSkipCache_f_param_5,
+	.param .u32 postProcessNNLstmForwardSkipCache_f_param_6,
+	.param .u32 postProcessNNLstmForwardSkipCache_f_param_7,
+	.param .u32 postProcessNNLstmForwardSkipCache_f_param_8,
+	.param .u32 postProcessNNLstmForwardSkipCache_f_param_9
+)
+{
+	.reg .pred 	%p<6>;
+	.reg .f32 	%f<40>;
+	.reg .b32 	%r<22>;
+	.reg .b64 	%rd<24>;
+
+
+	ld.param.u64 	%rd2, [postProcessNNLstmForwardSkipCache_f_param_0];
+	ld.param.u64 	%rd3, [postProcessNNLstmForwardSkipCache_f_param_1];
+	ld.param.u64 	%rd4, [postProcessNNLstmForwardSkipCache_f_param_2];
+	ld.param.u64 	%rd5, [postProcessNNLstmForwardSkipCache_f_param_3];
+	ld.param.u64 	%rd6, [postProcessNNLstmForwardSkipCache_f_param_4];
+	ld.param.u32 	%r4, [postProcessNNLstmForwardSkipCache_f_param_5];
+	ld.param.u32 	%r5, [postProcessNNLstmForwardSkipCache_f_param_6];
+	ld.param.u32 	%r6, [postProcessNNLstmForwardSkipCache_f_param_7];
+	ld.param.u32 	%r7, [postProcessNNLstmForwardSkipCache_f_param_8];
+	ld.param.u32 	%r8, [postProcessNNLstmForwardSkipCache_f_param_9];
+	mov.u32 	%r9, %ntid.x;
+	mov.u32 	%r10, %ctaid.x;
+	mov.u32 	%r11, %tid.x;
+	mad.lo.s32 	%r1, %r9, %r10, %r11;
+	setp.ge.u32	%p1, %r1, %r8;
+	@%p1 bra 	BB133_5;
+
+	cvta.to.global.u64 	%rd7, %rd2;
+	cvta.to.global.u64 	%rd8, %rd5;
+	div.s32 	%r2, %r1, %r7;
+	rem.s32 	%r3, %r1, %r7;
+	shl.b32 	%r12, %r7, 2;
+	mad.lo.s32 	%r13, %r2, %r12, %r3;
+	mul.wide.s32 	%rd9, %r13, 4;
+	add.s64 	%rd10, %rd7, %rd9;
+	cvt.s64.s32	%rd11, %r12;
+	add.s64 	%rd12, %rd10, %rd11;
+	add.s64 	%rd13, %rd12, %rd11;
+	ld.global.f32 	%f1, [%rd13];
+	add.s64 	%rd14, %rd13, %rd11;
+	mul.wide.s32 	%rd15, %r1, 4;
+	add.s64 	%rd1, %rd8, %rd15;
+	ld.global.f32 	%f7, [%rd1];
+	ld.global.f32 	%f8, [%rd12];
+	mul.f32 	%f9, %f8, %f7;
+	ld.global.f32 	%f10, [%rd14];
+	ld.global.f32 	%f11, [%rd10];
+	fma.rn.f32 	%f2, %f11, %f10, %f9;
+	abs.f32 	%f3, %f2;
+	setp.ltu.f32	%p2, %f3, 0f3F0CCCCD;
+	@%p2 bra 	BB133_3;
+	bra.uni 	BB133_2;
+
+BB133_3:
+	mul.f32 	%f27, %f2, %f2;
+	mov.f32 	%f28, 0fBD57BE66;
+	mov.f32 	%f29, 0f3C86A81B;
+	fma.rn.f32 	%f30, %f29, %f27, %f28;
+	mov.f32 	%f31, 0f3E08677B;
+	fma.rn.f32 	%f32, %f30, %f27, %f31;
+	mov.f32 	%f33, 0fBEAAAA29;
+	fma.rn.f32 	%f34, %f32, %f27, %f33;
+	mul.f32 	%f35, %f27, %f34;
+	fma.rn.f32 	%f36, %f35, %f2, %f2;
+	add.f32 	%f37, %f2, %f2;
+	setp.eq.f32	%p4, %f2, 0f00000000;
+	selp.f32	%f39, %f37, %f36, %p4;
+	bra.uni 	BB133_4;
+
+BB133_2:
+	add.f32 	%f14, %f3, %f3;
+	mul.f32 	%f15, %f14, 0f3FB8AA3B;
+	cvt.rzi.f32.f32	%f16, %f15;
+	mov.f32 	%f17, 0fBF317200;
+	fma.rn.f32 	%f18, %f16, %f17, %f14;
+	mov.f32 	%f19, 0fB5BFBE8E;
+	fma.rn.f32 	%f20, %f16, %f19, %f18;
+	mul.f32 	%f21, %f20, 0f3FB8AA3B;
+	ex2.approx.ftz.f32 	%f22, %f21;
+	ex2.approx.f32 	%f23, %f16;
+	mov.f32 	%f24, 0f3F800000;
+	fma.rn.f32 	%f13, %f22, %f23, %f24;
+	// inline asm
+	rcp.approx.ftz.f32 %f12,%f13;
+	// inline asm
+	mov.f32 	%f25, 0fC0000000;
+	fma.rn.f32 	%f26, %f12, %f25, %f24;
+	mov.b32 	 %r14, %f26;
+	setp.ltu.f32	%p3, %f3, 0f42B00000;
+	selp.b32	%r15, %r14, 1065353216, %p3;
+	mov.b32 	 %r16, %f2;
+	and.b32  	%r17, %r16, -2147483648;
+	or.b32  	%r18, %r15, %r17;
+	mov.b32 	 %f39, %r18;
+
+BB133_4:
+	cvta.to.global.u64 	%rd16, %rd4;
+	cvta.to.global.u64 	%rd17, %rd6;
+	mad.lo.s32 	%r19, %r2, %r6, %r5;
+	mad.lo.s32 	%r20, %r19, %r7, %r3;
+	setp.eq.s32	%p5, %r4, 0;
+	selp.b32	%r21, %r1, %r20, %p5;
+	cvta.to.global.u64 	%rd18, %rd3;
+	add.s64 	%rd20, %rd18, %rd15;
+	st.global.f32 	[%rd20], %f2;
+	add.s64 	%rd21, %rd16, %rd15;
+	mul.f32 	%f38, %f1, %f39;
+	st.global.f32 	[%rd21], %f38;
+	st.global.f32 	[%rd1], %f2;
+	mul.wide.s32 	%rd22, %r21, 4;
+	add.s64 	%rd23, %rd17, %rd22;
+	st.global.f32 	[%rd23], %f38;
+
+BB133_5:
+	ret;
+}
+
+	// .globl	initializeDoutWhenReturnSeq_d
+.visible .entry initializeDoutWhenReturnSeq_d(
+	.param .u64 initializeDoutWhenReturnSeq_d_param_0,
+	.param .u64 initializeDoutWhenReturnSeq_d_param_1,
+	.param .u32 initializeDoutWhenReturnSeq_d_param_2,
+	.param .u32 initializeDoutWhenReturnSeq_d_param_3,
+	.param .u32 initializeDoutWhenReturnSeq_d_param_4,
+	.param .u32 initializeDoutWhenReturnSeq_d_param_5
+)
+{
+	.reg .pred 	%p<2>;
+	.reg .b32 	%r<14>;
+	.reg .f64 	%fd<2>;
+	.reg .b64 	%rd<9>;
+
+
+	ld.param.u64 	%rd1, [initializeDoutWhenReturnSeq_d_param_0];
+	ld.param.u64 	%rd2, [initializeDoutWhenReturnSeq_d_param_1];
+	ld.param.u32 	%r2, [initializeDoutWhenReturnSeq_d_param_2];
+	ld.param.u32 	%r3, [initializeDoutWhenReturnSeq_d_param_3];
+	ld.param.u32 	%r4, [initializeDoutWhenReturnSeq_d_param_4];
+	ld.param.u32 	%r5, [initializeDoutWhenReturnSeq_d_param_5];
+	mov.u32 	%r6, %ctaid.x;
+	mov.u32 	%r7, %ntid.x;
+	mov.u32 	%r8, %tid.x;
+	mad.lo.s32 	%r1, %r7, %r6, %r8;
+	setp.ge.u32	%p1, %r1, %r5;
+	@%p1 bra 	BB134_2;
+
+	cvta.to.global.u64 	%rd3, %rd1;
+	div.s32 	%r9, %r1, %r3;
+	mul.lo.s32 	%r10, %r3, %r2;
+	mad.lo.s32 	%r11, %r9, %r4, %r10;
+	rem.s32 	%r12, %r1, %r3;
+	add.s32 	%r13, %r11, %r12;
+	mul.wide.s32 	%rd4, %r13, 8;
+	add.s64 	%rd5, %rd3, %rd4;
+	ld.global.f64 	%fd1, [%rd5];
+	cvta.to.global.u64 	%rd6, %rd2;
+	mul.wide.s32 	%rd7, %r1, 8;
+	add.s64 	%rd8, %rd6, %rd7;
+	st.global.f64 	[%rd8], %fd1;
+
+BB134_2:
+	ret;
+}
+
+	// .globl	initializeDoutWhenReturnSeq_f
+.visible .entry initializeDoutWhenReturnSeq_f(
+	.param .u64 initializeDoutWhenReturnSeq_f_param_0,
+	.param .u64 initializeDoutWhenReturnSeq_f_param_1,
+	.param .u32 initializeDoutWhenReturnSeq_f_param_2,
+	.param .u32 initializeDoutWhenReturnSeq_f_param_3,
+	.param .u32 initializeDoutWhenReturnSeq_f_param_4,
+	.param .u32 initializeDoutWhenReturnSeq_f_param_5
+)
+{
+	.reg .pred 	%p<2>;
+	.reg .f32 	%f<2>;
+	.reg .b32 	%r<14>;
+	.reg .b64 	%rd<9>;
+
+
+	ld.param.u64 	%rd1, [initializeDoutWhenReturnSeq_f_param_0];
+	ld.param.u64 	%rd2, [initializeDoutWhenReturnSeq_f_param_1];
+	ld.param.u32 	%r2, [initializeDoutWhenReturnSeq_f_param_2];
+	ld.param.u32 	%r3, [initializeDoutWhenReturnSeq_f_param_3];
+	ld.param.u32 	%r4, [initializeDoutWhenReturnSeq_f_param_4];
+	ld.param.u32 	%r5, [initializeDoutWhenReturnSeq_f_param_5];
+	mov.u32 	%r6, %ctaid.x;
+	mov.u32 	%r7, %ntid.x;
+	mov.u32 	%r8, %tid.x;
+	mad.lo.s32 	%r1, %r7, %r6, %r8;
+	setp.ge.u32	%p1, %r1, %r5;
+	@%p1 bra 	BB135_2;
+
+	cvta.to.global.u64 	%rd3, %rd1;
+	div.s32 	%r9, %r1, %r3;
+	mul.lo.s32 	%r10, %r3, %r2;
+	mad.lo.s32 	%r11, %r9, %r4, %r10;
+	rem.s32 	%r12, %r1, %r3;
+	add.s32 	%r13, %r11, %r12;
+	mul.wide.s32 	%rd4, %r13, 4;
+	add.s64 	%rd5, %rd3, %rd4;
+	ld.global.f32 	%f1, [%rd5];
+	cvta.to.global.u64 	%rd6, %rd2;
+	mul.wide.s32 	%rd7, %r1, 4;
+	add.s64 	%rd8, %rd6, %rd7;
+	st.global.f32 	[%rd8], %f1;
+
+BB135_2:
+	ret;
+}
+
+	// .globl	computeDifog_raw_d
+.visible .entry computeDifog_raw_d(
+	.param .u64 computeDifog_raw_d_param_0,
+	.param .u64 computeDifog_raw_d_param_1,
+	.param .u64 computeDifog_raw_d_param_2,
+	.param .u64 computeDifog_raw_d_param_3,
+	.param .u64 computeDifog_raw_d_param_4,
+	.param .u64 computeDifog_raw_d_param_5,
+	.param .u64 computeDifog_raw_d_param_6,
+	.param .u64 computeDifog_raw_d_param_7,
+	.param .u32 computeDifog_raw_d_param_8,
+	.param .u32 computeDifog_raw_d_param_9,
+	.param .u32 computeDifog_raw_d_param_10,
+	.param .u32 computeDifog_raw_d_param_11,
+	.param .u32 computeDifog_raw_d_param_12
+)
+{
+	.reg .pred 	%p<6>;
+	.reg .b32 	%r<49>;
+	.reg .f64 	%fd<106>;
+	.reg .b64 	%rd<51>;
+
+
+	ld.param.u64 	%rd6, [computeDifog_raw_d_param_0];
+	ld.param.u64 	%rd7, [computeDifog_raw_d_param_1];
+	ld.param.u64 	%rd8, [computeDifog_raw_d_param_2];
+	ld.param.u64 	%rd9, [computeDifog_raw_d_param_3];
+	ld.param.u64 	%rd10, [computeDifog_raw_d_param_4];
+	ld.param.u32 	%r3, [computeDifog_raw_d_param_9];
+	ld.param.u32 	%r4, [computeDifog_raw_d_param_11];
+	ld.param.u32 	%r5, [computeDifog_raw_d_param_12];
+	mov.u32 	%r6, %ntid.x;
+	mov.u32 	%r7, %ctaid.x;
+	mov.u32 	%r8, %tid.x;
+	mad.lo.s32 	%r1, %r6, %r7, %r8;
+	setp.ge.u32	%p1, %r1, %r5;
+	@%p1 bra 	BB136_10;
+
+	cvta.to.global.u64 	%rd14, %rd6;
+	cvta.to.global.u64 	%rd15, %rd8;
+	cvt.s64.s32	%rd1, %r1;
+	mul.wide.s32 	%rd16, %r1, 8;
+	add.s64 	%rd17, %rd15, %rd16;
+	ld.global.f64 	%fd1, [%rd17];
+	div.s32 	%r9, %r1, %r4;
+	shl.b32 	%r10, %r4, 2;
+	rem.s32 	%r11, %r1, %r4;
+	mad.lo.s32 	%r12, %r9, %r10, %r11;
+	mul.wide.s32 	%rd18, %r12, 8;
+	add.s64 	%rd19, %rd14, %rd18;
+	ld.global.f64 	%fd2, [%rd19];
+	mul.wide.s32 	%rd20, %r4, 8;
+	add.s64 	%rd21, %rd19, %rd20;
+	ld.global.f64 	%fd3, [%rd21];
+	add.s64 	%rd22, %rd21, %rd20;
+	ld.global.f64 	%fd4, [%rd22];
+	add.s64 	%rd23, %rd22, %rd20;
+	ld.global.f64 	%fd5, [%rd23];
+	cvta.to.global.u64 	%rd24, %rd7;
+	add.s64 	%rd25, %rd24, %rd16;
+	ld.global.f64 	%fd6, [%rd25];
+	setp.eq.s32	%p2, %r3, 0;
+	@%p2 bra 	BB136_3;
+
+	cvta.to.global.u64 	%rd26, %rd9;
+	add.s32 	%r13, %r3, -1;
+	mad.lo.s32 	%r14, %r13, %r5, %r1;
+	mul.wide.u32 	%rd27, %r14, 8;
+	add.s64 	%rd50, %rd26, %rd27;
+	bra.uni 	BB136_4;
+
+BB136_3:
+	cvta.to.global.u64 	%rd28, %rd10;
+	shl.b64 	%rd29, %rd1, 3;
+	add.s64 	%rd50, %rd28, %rd29;
+
+BB136_4:
+	ld.global.f64 	%fd7, [%rd50];
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r2}, %fd6;
+	}
+	and.b32  	%r15, %r2, 2147483647;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r16, %temp}, %fd6;
+	}
+	mov.b64 	%fd8, {%r16, %r15};
+	setp.ltu.f64	%p3, %fd8, 0d3FE1C7A398201CD6;
+	@%p3 bra 	BB136_6;
+	bra.uni 	BB136_5;
+
+BB136_6:
+	mul.f64 	%fd61, %fd6, %fd6;
+	mov.f64 	%fd62, 0dBF2B9093D89F0E23;
+	mov.f64 	%fd63, 0d3F0ABFFC9B5786C4;
+	fma.rn.f64 	%fd64, %fd63, %fd61, %fd62;
+	mov.f64 	%fd65, 0d3F42FA2744C30B61;
+	fma.rn.f64 	%fd66, %fd64, %fd61, %fd65;
+	mov.f64 	%fd67, 0dBF57CF3B9C1E491D;
+	fma.rn.f64 	%fd68, %fd66, %fd61, %fd67;
+	mov.f64 	%fd69, 0d3F6D6C61D450119A;
+	fma.rn.f64 	%fd70, %fd68, %fd61, %fd69;
+	mov.f64 	%fd71, 0dBF8226DDD44294F5;
+	fma.rn.f64 	%fd72, %fd70, %fd61, %fd71;
+	mov.f64 	%fd73, 0d3F9664F45C2B04A6;
+	fma.rn.f64 	%fd74, %fd72, %fd61, %fd73;
+	mov.f64 	%fd75, 0dBFABA1BA1AD70754;
+	fma.rn.f64 	%fd76, %fd74, %fd61, %fd75;
+	mov.f64 	%fd77, 0d3FC111111110295E;
+	fma.rn.f64 	%fd78, %fd76, %fd61, %fd77;
+	mov.f64 	%fd79, 0dBFD555555555549F;
+	fma.rn.f64 	%fd80, %fd78, %fd61, %fd79;
+	mul.f64 	%fd81, %fd61, %fd80;
+	fma.rn.f64 	%fd104, %fd81, %fd6, %fd6;
+	bra.uni 	BB136_7;
+
+BB136_5:
+	add.f64 	%fd16, %fd8, %fd8;
+	mov.f64 	%fd17, 0d4338000000000000;
+	mov.f64 	%fd18, 0d3FF71547652B82FE;
+	fma.rn.f64 	%fd19, %fd16, %fd18, %fd17;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r17, %temp}, %fd19;
+	}
+	mov.f64 	%fd20, 0dC338000000000000;
+	add.rn.f64 	%fd21, %fd19, %fd20;
+	mov.f64 	%fd22, 0dBFE62E42FEFA39EF;
+	fma.rn.f64 	%fd23, %fd21, %fd22, %fd16;
+	mov.f64 	%fd24, 0dBC7ABC9E3B39803F;
+	fma.rn.f64 	%fd25, %fd21, %fd24, %fd23;
+	mov.f64 	%fd26, 0d3E5AF86D8EBD13CD;
+	mov.f64 	%fd27, 0d3E21F4076ACD15B6;
+	fma.rn.f64 	%fd28, %fd27, %fd25, %fd26;
+	mov.f64 	%fd29, 0d3E927E5092BA033D;
+	fma.rn.f64 	%fd30, %fd28, %fd25, %fd29;
+	mov.f64 	%fd31, 0d3EC71DDE6C5F9DA1;
+	fma.rn.f64 	%fd32, %fd30, %fd25, %fd31;
+	mov.f64 	%fd33, 0d3EFA01A018D034E6;
+	fma.rn.f64 	%fd34, %fd32, %fd25, %fd33;
+	mov.f64 	%fd35, 0d3F2A01A01B3B6940;
+	fma.rn.f64 	%fd36, %fd34, %fd25, %fd35;
+	mov.f64 	%fd37, 0d3F56C16C16C1B5DD;
+	fma.rn.f64 	%fd38, %fd36, %fd25, %fd37;
+	mov.f64 	%fd39, 0d3F8111111110F74D;
+	fma.rn.f64 	%fd40, %fd38, %fd25, %fd39;
+	mov.f64 	%fd41, 0d3FA555555555554D;
+	fma.rn.f64 	%fd42, %fd40, %fd25, %fd41;
+	mov.f64 	%fd43, 0d3FC5555555555557;
+	fma.rn.f64 	%fd44, %fd42, %fd25, %fd43;
+	mov.f64 	%fd45, 0d3FE0000000000000;
+	fma.rn.f64 	%fd46, %fd44, %fd25, %fd45;
+	mul.f64 	%fd47, %fd25, %fd46;
+	fma.rn.f64 	%fd48, %fd47, %fd25, %fd25;
+	shl.b32 	%r18, %r17, 20;
+	add.s32 	%r19, %r18, 1072693248;
+	mov.u32 	%r20, 0;
+	mov.b64 	%fd49, {%r20, %r19};
+	fma.rn.f64 	%fd50, %fd48, %fd49, %fd49;
+	add.f64 	%fd51, %fd50, 0d3FF0000000000000;
+	rcp.approx.ftz.f64 	%fd52, %fd51;
+	neg.f64 	%fd53, %fd51;
+	mov.f64 	%fd54, 0d3FF0000000000000;
+	fma.rn.f64 	%fd55, %fd53, %fd52, %fd54;
+	fma.rn.f64 	%fd56, %fd55, %fd55, %fd55;
+	fma.rn.f64 	%fd57, %fd56, %fd52, %fd52;
+	neg.f64 	%fd58, %fd57;
+	mov.f64 	%fd59, 0d4000000000000000;
+	fma.rn.f64 	%fd60, %fd59, %fd58, %fd54;
+	setp.gt.u32	%p4, %r15, 1077936127;
+	selp.f64	%fd104, 0d3FF0000000000000, %fd60, %p4;
+
+BB136_7:
+	ld.param.u64 	%rd49, [computeDifog_raw_d_param_6];
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r39}, %fd6;
+	}
+	ld.param.u32 	%r38, [computeDifog_raw_d_param_9];
+	and.b32  	%r22, %r39, -2147483648;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%temp, %r23}, %fd104;
+	}
+	or.b32  	%r24, %r23, %r22;
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r25, %temp}, %fd104;
+	}
+	mov.b64 	%fd82, {%r25, %r24};
+	cvta.to.global.u64 	%rd30, %rd49;
+	add.s64 	%rd32, %rd30, %rd16;
+	mul.f64 	%fd83, %fd82, %fd82;
+	mov.f64 	%fd84, 0d3FF0000000000000;
+	sub.f64 	%fd85, %fd84, %fd83;
+	mul.f64 	%fd86, %fd1, %fd85;
+	ld.global.f64 	%fd87, [%rd32];
+	fma.rn.f64 	%fd12, %fd4, %fd86, %fd87;
+	mul.f64 	%fd105, %fd3, %fd12;
+	mul.f64 	%fd14, %fd1, %fd82;
+	setp.ne.s32	%p5, %r38, 0;
+	@%p5 bra 	BB136_9;
+
+	ld.param.u64 	%rd48, [computeDifog_raw_d_param_7];
+	cvta.to.global.u64 	%rd33, %rd48;
+	add.s64 	%rd35, %rd33, %rd16;
+	st.global.f64 	[%rd35], %fd105;
+	mov.f64 	%fd105, %fd12;
+
+BB136_9:
+	mov.u32 	%r48, %tid.x;
+	mov.u32 	%r47, %ctaid.x;
+	mov.u32 	%r46, %ntid.x;
+	ld.param.u32 	%r45, [computeDifog_raw_d_param_11];
+	mad.lo.s32 	%r44, %r46, %r47, %r48;
+	rem.s32 	%r43, %r44, %r45;
+	shl.b32 	%r42, %r45, 2;
+	div.s32 	%r41, %r44, %r45;
+	mad.lo.s32 	%r40, %r41, %r42, %r43;
+	cvt.s64.s32	%rd47, %r40;
+	ld.param.u64 	%rd46, [computeDifog_raw_d_param_5];
+	mul.f64 	%fd88, %fd7, %fd12;
+	mul.f64 	%fd89, %fd5, %fd12;
+	mul.f64 	%fd90, %fd2, %fd12;
+	st.global.f64 	[%rd32], %fd105;
+	sub.f64 	%fd92, %fd84, %fd2;
+	mul.f64 	%fd93, %fd2, %fd92;
+	mul.f64 	%fd94, %fd93, %fd89;
+	cvta.to.global.u64 	%rd39, %rd46;
+	shl.b64 	%rd40, %rd47, 3;
+	add.s64 	%rd41, %rd39, %rd40;
+	st.global.f64 	[%rd41], %fd94;
+	sub.f64 	%fd95, %fd84, %fd3;
+	mul.f64 	%fd96, %fd3, %fd95;
+	mul.f64 	%fd97, %fd96, %fd88;
+	add.s64 	%rd43, %rd41, %rd20;
+	st.global.f64 	[%rd43], %fd97;
+	sub.f64 	%fd98, %fd84, %fd4;
+	mul.f64 	%fd99, %fd4, %fd98;
+	mul.f64 	%fd100, %fd99, %fd14;
+	add.s64 	%rd44, %rd43, %rd20;
+	st.global.f64 	[%rd44], %fd100;
+	mul.f64 	%fd101, %fd5, %fd5;
+	sub.f64 	%fd102, %fd84, %fd101;
+	mul.f64 	%fd103, %fd102, %fd90;
+	add.s64 	%rd45, %rd44, %rd20;
+	st.global.f64 	[%rd45], %fd103;
+
+BB136_10:
+	ret;
+}
+
+	// .globl	computeDifog_raw_f
+.visible .entry computeDifog_raw_f(
+	.param .u64 computeDifog_raw_f_param_0,
+	.param .u64 computeDifog_raw_f_param_1,
+	.param .u64 computeDifog_raw_f_param_2,
+	.param .u64 computeDifog_raw_f_param_3,
+	.param .u64 computeDifog_raw_f_param_4,
+	.param .u64 computeDifog_raw_f_param_5,
+	.param .u64 computeDifog_raw_f_param_6,
+	.param .u64 computeDifog_raw_f_param_7,
+	.param .u32 computeDifog_raw_f_param_8,
+	.param .u32 computeDifog_raw_f_param_9,
+	.param .u32 computeDifog_raw_f_param_10,
+	.param .u32 computeDifog_raw_f_param_11,
+	.param .u32 computeDifog_raw_f_param_12
+)
+{
+	.reg .pred 	%p<7>;
+	.reg .f32 	%f<65>;
+	.reg .b32 	%r<35>;
+	.reg .b64 	%rd<46>;
+
+
+	ld.param.u64 	%rd5, [computeDifog_raw_f_param_0];
+	ld.param.u64 	%rd6, [computeDifog_raw_f_param_1];
+	ld.param.u64 	%rd7, [computeDifog_raw_f_param_2];
+	ld.param.u64 	%rd8, [computeDifog_raw_f_param_3];
+	ld.param.u64 	%rd9, [computeDifog_raw_f_param_4];
+	ld.param.u64 	%rd10, [computeDifog_raw_f_param_5];
+	ld.param.u64 	%rd11, [computeDifog_raw_f_param_6];
+	ld.param.u64 	%rd12, [computeDifog_raw_f_param_7];
+	ld.param.u32 	%r2, [computeDifog_raw_f_param_9];
+	ld.param.u32 	%r3, [computeDifog_raw_f_param_11];
+	ld.param.u32 	%r4, [computeDifog_raw_f_param_12];
+	mov.u32 	%r5, %ntid.x;
+	mov.u32 	%r6, %ctaid.x;
+	mov.u32 	%r7, %tid.x;
+	mad.lo.s32 	%r1, %r5, %r6, %r7;
+	setp.ge.u32	%p1, %r1, %r4;
+	@%p1 bra 	BB137_10;
+
+	cvta.to.global.u64 	%rd13, %rd5;
+	cvta.to.global.u64 	%rd14, %rd7;
+	mul.wide.s32 	%rd15, %r1, 4;
+	add.s64 	%rd16, %rd14, %rd15;
+	ld.global.f32 	%f1, [%rd16];
+	div.s32 	%r8, %r1, %r3;
+	shl.b32 	%r9, %r3, 2;
+	rem.s32 	%r10, %r1, %r3;
+	mad.lo.s32 	%r11, %r8, %r9, %r10;
+	cvt.s64.s32	%rd1, %r11;
+	mul.wide.s32 	%rd17, %r11, 4;
+	add.s64 	%rd18, %rd13, %rd17;
+	ld.global.f32 	%f2, [%rd18];
+	mul.wide.s32 	%rd19, %r3, 4;
+	add.s64 	%rd20, %rd18, %rd19;
+	ld.global.f32 	%f3, [%rd20];
+	add.s64 	%rd21, %rd20, %rd19;
+	ld.global.f32 	%f4, [%rd21];
+	add.s64 	%rd22, %rd21, %rd19;
+	ld.global.f32 	%f5, [%rd22];
+	cvta.to.global.u64 	%rd23, %rd6;
+	add.s64 	%rd24, %rd23, %rd15;
+	ld.global.f32 	%f6, [%rd24];
+	setp.eq.s32	%p2, %r2, 0;
+	@%p2 bra 	BB137_3;
+
+	cvta.to.global.u64 	%rd25, %rd8;
+	add.s32 	%r12, %r2, -1;
+	mad.lo.s32 	%r13, %r12, %r4, %r1;
+	mul.wide.u32 	%rd26, %r13, 4;
+	add.s64 	%rd45, %rd25, %rd26;
+	bra.uni 	BB137_4;
+
+BB137_3:
+	cvta.to.global.u64 	%rd27, %rd9;
+	add.s64 	%rd45, %rd27, %rd15;
+
+BB137_4:
+	ld.global.f32 	%f7, [%rd45];
+	abs.f32 	%f8, %f6;
+	setp.ltu.f32	%p3, %f8, 0f3F0CCCCD;
+	@%p3 bra 	BB137_6;
+	bra.uni 	BB137_5;
+
+BB137_6:
+	mul.f32 	%f31, %f6, %f6;
+	mov.f32 	%f32, 0fBD57BE66;
+	mov.f32 	%f33, 0f3C86A81B;
+	fma.rn.f32 	%f34, %f33, %f31, %f32;
+	mov.f32 	%f35, 0f3E08677B;
+	fma.rn.f32 	%f36, %f34, %f31, %f35;
+	mov.f32 	%f37, 0fBEAAAA29;
+	fma.rn.f32 	%f38, %f36, %f31, %f37;
+	mul.f32 	%f39, %f31, %f38;
+	fma.rn.f32 	%f40, %f39, %f6, %f6;
+	add.f32 	%f41, %f6, %f6;
+	setp.eq.f32	%p5, %f6, 0f00000000;
+	selp.f32	%f63, %f41, %f40, %p5;
+	bra.uni 	BB137_7;
+
+BB137_5:
+	add.f32 	%f18, %f8, %f8;
+	mul.f32 	%f19, %f18, 0f3FB8AA3B;
+	cvt.rzi.f32.f32	%f20, %f19;
+	mov.f32 	%f21, 0fBF317200;
+	fma.rn.f32 	%f22, %f20, %f21, %f18;
+	mov.f32 	%f23, 0fB5BFBE8E;
+	fma.rn.f32 	%f24, %f20, %f23, %f22;
+	mul.f32 	%f25, %f24, 0f3FB8AA3B;
+	ex2.approx.ftz.f32 	%f26, %f25;
+	ex2.approx.f32 	%f27, %f20;
+	mov.f32 	%f28, 0f3F800000;
+	fma.rn.f32 	%f17, %f26, %f27, %f28;
+	// inline asm
+	rcp.approx.ftz.f32 %f16,%f17;
+	// inline asm
+	mov.f32 	%f29, 0fC0000000;
+	fma.rn.f32 	%f30, %f16, %f29, %f28;
+	mov.b32 	 %r18, %f30;
+	setp.ltu.f32	%p4, %f8, 0f42B00000;
+	selp.b32	%r19, %r18, 1065353216, %p4;
+	mov.b32 	 %r20, %f6;
+	and.b32  	%r21, %r20, -2147483648;
+	or.b32  	%r22, %r19, %r21;
+	mov.b32 	 %f63, %r22;
+
+BB137_7:
+	cvta.to.global.u64 	%rd29, %rd11;
+	add.s64 	%rd31, %rd29, %rd15;
+	mul.f32 	%f42, %f63, %f63;
+	mov.f32 	%f43, 0f3F800000;
+	sub.f32 	%f44, %f43, %f42;
+	mul.f32 	%f45, %f1, %f44;
+	ld.global.f32 	%f46, [%rd31];
+	fma.rn.f32 	%f12, %f4, %f45, %f46;
+	mul.f32 	%f64, %f3, %f12;
+	mul.f32 	%f14, %f1, %f63;
+	setp.ne.s32	%p6, %r2, 0;
+	@%p6 bra 	BB137_9;
+
+	cvta.to.global.u64 	%rd32, %rd12;
+	add.s64 	%rd34, %rd32, %rd15;
+	st.global.f32 	[%rd34], %f64;
+	mov.f32 	%f64, %f12;
+
+BB137_9:
+	mul.f32 	%f47, %f7, %f12;
+	mul.f32 	%f48, %f5, %f12;
+	mul.f32 	%f49, %f2, %f12;
+	st.global.f32 	[%rd31], %f64;
+	sub.f32 	%f51, %f43, %f2;
+	mul.f32 	%f52, %f2, %f51;
+	mul.f32 	%f53, %f52, %f48;
+	cvta.to.global.u64 	%rd38, %rd10;
+	shl.b64 	%rd39, %rd1, 2;
+	add.s64 	%rd40, %rd38, %rd39;
+	st.global.f32 	[%rd40], %f53;
+	sub.f32 	%f54, %f43, %f3;
+	mul.f32 	%f55, %f3, %f54;
+	mul.f32 	%f56, %f55, %f47;
+	add.s64 	%rd42, %rd40, %rd19;
+	st.global.f32 	[%rd42], %f56;
+	sub.f32 	%f57, %f43, %f4;
+	mul.f32 	%f58, %f4, %f57;
+	mul.f32 	%f59, %f58, %f14;
+	add.s64 	%rd43, %rd42, %rd19;
+	st.global.f32 	[%rd43], %f59;
+	mul.f32 	%f60, %f5, %f5;
+	sub.f32 	%f61, %f43, %f60;
+	mul.f32 	%f62, %f61, %f49;
+	add.s64 	%rd44, %rd43, %rd19;
+	st.global.f32 	[%rd44], %f62;
+
+BB137_10:
+	ret;
+}
+
+	// .globl	postProcessNNLstmBackward_d
+.visible .entry postProcessNNLstmBackward_d(
+	.param .u64 postProcessNNLstmBackward_d_param_0,
+	.param .u64 postProcessNNLstmBackward_d_param_1,
+	.param .u64 postProcessNNLstmBackward_d_param_2,
+	.param .u64 postProcessNNLstmBackward_d_param_3,
+	.param .u64 postProcessNNLstmBackward_d_param_4,
+	.param .u32 postProcessNNLstmBackward_d_param_5,
+	.param .u32 postProcessNNLstmBackward_d_param_6,
+	.param .u32 postProcessNNLstmBackward_d_param_7,
+	.param .u32 postProcessNNLstmBackward_d_param_8,
+	.param .u32 postProcessNNLstmBackward_d_param_9,
+	.param .u32 postProcessNNLstmBackward_d_param_10,
+	.param .u32 postProcessNNLstmBackward_d_param_11,
+	.param .u32 postProcessNNLstmBackward_d_param_12,
+	.param .u32 postProcessNNLstmBackward_d_param_13,
+	.param .u32 postProcessNNLstmBackward_d_param_14,
+	.param .u32 postProcessNNLstmBackward_d_param_15
+)
+{
+	.reg .pred 	%p<5>;
+	.reg .b32 	%r<28>;
+	.reg .f64 	%fd<5>;
+	.reg .b64 	%rd<23>;
+
+
+	ld.param.u64 	%rd7, [postProcessNNLstmBackward_d_param_0];
+	ld.param.u64 	%rd3, [postProcessNNLstmBackward_d_param_1];
+	ld.param.u64 	%rd4, [postProcessNNLstmBackward_d_param_2];
+	ld.param.u64 	%rd5, [postProcessNNLstmBackward_d_param_3];
+	ld.param.u64 	%rd6, [postProcessNNLstmBackward_d_param_4];
+	ld.param.u32 	%r4, [postProcessNNLstmBackward_d_param_5];
+	ld.param.u32 	%r5, [postProcessNNLstmBackward_d_param_6];
+	ld.param.u32 	%r6, [postProcessNNLstmBackward_d_param_8];
+	ld.param.u32 	%r7, [postProcessNNLstmBackward_d_param_9];
+	ld.param.u32 	%r12, [postProcessNNLstmBackward_d_param_10];
+	ld.param.u32 	%r8, [postProcessNNLstmBackward_d_param_11];
+	ld.param.u32 	%r9, [postProcessNNLstmBackward_d_param_12];
+	ld.param.u32 	%r10, [postProcessNNLstmBackward_d_param_13];
+	ld.param.u32 	%r11, [postProcessNNLstmBackward_d_param_14];
+	cvta.to.global.u64 	%rd1, %rd7;
+	mov.u32 	%r13, %ntid.x;
+	mov.u32 	%r14, %ctaid.x;
+	mov.u32 	%r15, %tid.x;
+	mad.lo.s32 	%r1, %r13, %r14, %r15;
+	setp.ge.s32	%p1, %r1, %r12;
+	@%p1 bra 	BB138_2;
+
+	cvta.to.global.u64 	%rd8, %rd6;
+	div.s32 	%r16, %r1, %r6;
+	rem.s32 	%r17, %r1, %r6;
+	mad.lo.s32 	%r18, %r16, %r11, %r17;
+	mul.wide.s32 	%rd9, %r18, 8;
+	add.s64 	%rd10, %rd1, %rd9;
+	ld.global.f64 	%fd2, [%rd10];
+	mul.lo.s32 	%r19, %r6, %r5;
+	mad.lo.s32 	%r20, %r16, %r9, %r19;
+	add.s32 	%r21, %r20, %r17;
+	mul.wide.s32 	%rd11, %r21, 8;
+	add.s64 	%rd12, %rd8, %rd11;
+	st.global.f64 	[%rd12], %fd2;
+
+BB138_2:
+	setp.ge.s32	%p2, %r1, %r8;
+	@%p2 bra 	BB138_8;
+
+	div.s32 	%r2, %r1, %r7;
+	mad.lo.s32 	%r22, %r2, %r11, %r6;
+	rem.s32 	%r3, %r1, %r7;
+	add.s32 	%r23, %r22, %r3;
+	mul.wide.s32 	%rd13, %r23, 8;
+	add.s64 	%rd14, %rd1, %rd13;
+	ld.global.f64 	%fd1, [%rd14];
+	setp.eq.s32	%p3, %r5, 0;
+	@%p3 bra 	BB138_7;
+
+	cvta.to.global.u64 	%rd15, %rd5;
+	setp.eq.s32	%p4, %r4, 0;
+	mul.wide.s32 	%rd16, %r1, 8;
+	add.s64 	%rd2, %rd15, %rd16;
+	@%p4 bra 	BB138_6;
+
+	cvta.to.global.u64 	%rd17, %rd4;
+	add.s32 	%r24, %r5, -1;
+	mul.lo.s32 	%r25, %r24, %r7;
+	mad.lo.s32 	%r26, %r2, %r10, %r25;
+	add.s32 	%r27, %r26, %r3;
+	mul.wide.s32 	%rd18, %r27, 8;
+	add.s64 	%rd19, %rd17, %rd18;
+	ld.global.f64 	%fd3, [%rd19];
+	add.f64 	%fd4, %fd1, %fd3;
+	st.global.f64 	[%rd2], %fd4;
+	bra.uni 	BB138_8;
+
+BB138_7:
+	cvta.to.global.u64 	%rd20, %rd3;
+	mul.wide.s32 	%rd21, %r1, 8;
+	add.s64 	%rd22, %rd20, %rd21;
+	st.global.f64 	[%rd22], %fd1;
+	bra.uni 	BB138_8;
+
+BB138_6:
+	st.global.f64 	[%rd2], %fd1;
+
+BB138_8:
+	ret;
+}
+
+	// .globl	postProcessNNLstmBackward_f
+.visible .entry postProcessNNLstmBackward_f(
+	.param .u64 postProcessNNLstmBackward_f_param_0,
+	.param .u64 postProcessNNLstmBackward_f_param_1,
+	.param .u64 postProcessNNLstmBackward_f_param_2,
+	.param .u64 postProcessNNLstmBackward_f_param_3,
+	.param .u64 postProcessNNLstmBackward_f_param_4,
+	.param .u32 postProcessNNLstmBackward_f_param_5,
+	.param .u32 postProcessNNLstmBackward_f_param_6,
+	.param .u32 postProcessNNLstmBackward_f_param_7,
+	.param .u32 postProcessNNLstmBackward_f_param_8,
+	.param .u32 postProcessNNLstmBackward_f_param_9,
+	.param .u32 postProcessNNLstmBackward_f_param_10,
+	.param .u32 postProcessNNLstmBackward_f_param_11,
+	.param .u32 postProcessNNLstmBackward_f_param_12,
+	.param .u32 postProcessNNLstmBackward_f_param_13,
+	.param .u32 postProcessNNLstmBackward_f_param_14,
+	.param .u32 postProcessNNLstmBackward_f_param_15
+)
+{
+	.reg .pred 	%p<5>;
+	.reg .f32 	%f<5>;
+	.reg .b32 	%r<28>;
+	.reg .b64 	%rd<23>;
+
+
+	ld.param.u64 	%rd7, [postProcessNNLstmBackward_f_param_0];
+	ld.param.u64 	%rd3, [postProcessNNLstmBackward_f_param_1];
+	ld.param.u64 	%rd4, [postProcessNNLstmBackward_f_param_2];
+	ld.param.u64 	%rd5, [postProcessNNLstmBackward_f_param_3];
+	ld.param.u64 	%rd6, [postProcessNNLstmBackward_f_param_4];
+	ld.param.u32 	%r4, [postProcessNNLstmBackward_f_param_5];
+	ld.param.u32 	%r5, [postProcessNNLstmBackward_f_param_6];
+	ld.param.u32 	%r6, [postProcessNNLstmBackward_f_param_8];
+	ld.param.u32 	%r7, [postProcessNNLstmBackward_f_param_9];
+	ld.param.u32 	%r12, [postProcessNNLstmBackward_f_param_10];
+	ld.param.u32 	%r8, [postProcessNNLstmBackward_f_param_11];
+	ld.param.u32 	%r9, [postProcessNNLstmBackward_f_param_12];
+	ld.param.u32 	%r10, [postProcessNNLstmBackward_f_param_13];
+	ld.param.u32 	%r11, [postProcessNNLstmBackward_f_param_14];
+	cvta.to.global.u64 	%rd1, %rd7;
+	mov.u32 	%r13, %ntid.x;
+	mov.u32 	%r14, %ctaid.x;
+	mov.u32 	%r15, %tid.x;
+	mad.lo.s32 	%r1, %r13, %r14, %r15;
+	setp.ge.s32	%p1, %r1, %r12;
+	@%p1 bra 	BB139_2;
+
+	cvta.to.global.u64 	%rd8, %rd6;
+	div.s32 	%r16, %r1, %r6;
+	rem.s32 	%r17, %r1, %r6;
+	mad.lo.s32 	%r18, %r16, %r11, %r17;
+	mul.wide.s32 	%rd9, %r18, 4;
+	add.s64 	%rd10, %rd1, %rd9;
+	ld.global.f32 	%f2, [%rd10];
+	mul.lo.s32 	%r19, %r6, %r5;
+	mad.lo.s32 	%r20, %r16, %r9, %r19;
+	add.s32 	%r21, %r20, %r17;
+	mul.wide.s32 	%rd11, %r21, 4;
+	add.s64 	%rd12, %rd8, %rd11;
+	st.global.f32 	[%rd12], %f2;
+
+BB139_2:
+	setp.ge.s32	%p2, %r1, %r8;
+	@%p2 bra 	BB139_8;
+
+	div.s32 	%r2, %r1, %r7;
+	mad.lo.s32 	%r22, %r2, %r11, %r6;
+	rem.s32 	%r3, %r1, %r7;
+	add.s32 	%r23, %r22, %r3;
+	mul.wide.s32 	%rd13, %r23, 4;
+	add.s64 	%rd14, %rd1, %rd13;
+	ld.global.f32 	%f1, [%rd14];
+	setp.eq.s32	%p3, %r5, 0;
+	@%p3 bra 	BB139_7;
+
+	cvta.to.global.u64 	%rd15, %rd5;
+	setp.eq.s32	%p4, %r4, 0;
+	mul.wide.s32 	%rd16, %r1, 4;
+	add.s64 	%rd2, %rd15, %rd16;
+	@%p4 bra 	BB139_6;
+
+	cvta.to.global.u64 	%rd17, %rd4;
+	add.s32 	%r24, %r5, -1;
+	mul.lo.s32 	%r25, %r24, %r7;
+	mad.lo.s32 	%r26, %r2, %r10, %r25;
+	add.s32 	%r27, %r26, %r3;
+	mul.wide.s32 	%rd18, %r27, 4;
+	add.s64 	%rd19, %rd17, %rd18;
+	ld.global.f32 	%f3, [%rd19];
+	add.f32 	%f4, %f1, %f3;
+	st.global.f32 	[%rd2], %f4;
+	bra.uni 	BB139_8;
+
+BB139_7:
+	cvta.to.global.u64 	%rd20, %rd3;
+	mul.wide.s32 	%rd21, %r1, 4;
+	add.s64 	%rd22, %rd20, %rd21;
+	st.global.f32 	[%rd22], %f1;
+	bra.uni 	BB139_8;
+
+BB139_6:
+	st.global.f32 	[%rd2], %f1;
+
+BB139_8:
+	ret;
+}
+
 .func  (.param .b64 func_retval0) __internal_trig_reduction_slowpathd(
 	.param .b64 __internal_trig_reduction_slowpathd_param_0,
 	.param .b64 __internal_trig_reduction_slowpathd_param_1
 )
 {
-	.local .align 8 .b8 	__local_depot126[40];
+	.local .align 8 .b8 	__local_depot140[40];
 	.reg .b64 	%SP;
 	.reg .b64 	%SPL;
 	.reg .pred 	%p<9>;
@@ -15232,7 +17252,7 @@ BB125_2:
 	.reg .b64 	%rd<102>;
 
 
-	mov.u64 	%rd101, __local_depot126;
+	mov.u64 	%rd101, __local_depot140;
 	cvta.local.u64 	%SP, %rd101;
 	ld.param.f64 	%fd4, [__internal_trig_reduction_slowpathd_param_0];
 	ld.param.u64 	%rd37, [__internal_trig_reduction_slowpathd_param_1];
@@ -15246,7 +17266,7 @@ BB125_2:
 	shr.u32 	%r3, %r1, 20;
 	bfe.u32 	%r4, %r1, 20, 11;
 	setp.eq.s32	%p1, %r4, 2047;
-	@%p1 bra 	BB126_13;
+	@%p1 bra 	BB140_13;
 
 	add.s32 	%r15, %r4, -1024;
 	shr.u32 	%r16, %r15, 6;
@@ -15259,7 +17279,7 @@ BB125_2:
 	mov.u64 	%rd94, 0;
 	setp.ge.s32	%p2, %r5, %r6;
 	mov.u64 	%rd93, %rd1;
-	@%p2 bra 	BB126_4;
+	@%p2 bra 	BB140_4;
 
 	mov.b64 	 %rd41, %fd4;
 	shl.b64 	%rd42, %rd41, 11;
@@ -15276,7 +17296,7 @@ BB125_2:
 	mov.u64 	%rd91, %rd1;
 	mov.u32 	%r39, %r5;
 
-BB126_3:
+BB140_3:
 	.pragma "nounroll";
 	ld.const.u64 	%rd47, [%rd89];
 	// inline asm
@@ -15306,15 +17326,15 @@ BB126_3:
 	add.s64 	%rd93, %rd93, 8;
 	add.s64 	%rd89, %rd89, 8;
 	setp.lt.s32	%p3, %r39, %r6;
-	@%p3 bra 	BB126_3;
+	@%p3 bra 	BB140_3;
 
-BB126_4:
+BB140_4:
 	st.local.u64 	[%rd93], %rd94;
 	ld.local.u64 	%rd95, [%rd1+16];
 	ld.local.u64 	%rd96, [%rd1+24];
 	and.b32  	%r9, %r3, 63;
 	setp.eq.s32	%p4, %r9, 0;
-	@%p4 bra 	BB126_6;
+	@%p4 bra 	BB140_6;
 
 	mov.u32 	%r27, 64;
 	sub.s32 	%r28, %r27, %r9;
@@ -15326,7 +17346,7 @@ BB126_4:
 	shr.u64 	%rd55, %rd54, %r28;
 	or.b64  	%rd95, %rd55, %rd53;
 
-BB126_6:
+BB140_6:
 	cvta.to.local.u64 	%rd56, %rd37;
 	shr.u64 	%rd57, %rd96, 62;
 	cvt.u32.u64	%r29, %rd57;
@@ -15343,7 +17363,7 @@ BB126_6:
 	selp.b32	%r34, %r32, %r33, %p5;
 	st.local.u32 	[%rd56], %r34;
 	setp.eq.s32	%p6, %r31, 0;
-	@%p6 bra 	BB126_8;
+	@%p6 bra 	BB140_8;
 
 	mov.u64 	%rd64, 0;
 	// inline asm
@@ -15363,10 +17383,10 @@ BB126_6:
 	// inline asm
 	xor.b32  	%r40, %r40, -2147483648;
 
-BB126_8:
+BB140_8:
 	clz.b64 	%r41, %rd98;
 	setp.eq.s32	%p7, %r41, 0;
-	@%p7 bra 	BB126_10;
+	@%p7 bra 	BB140_10;
 
 	shl.b64 	%rd67, %rd98, %r41;
 	mov.u32 	%r35, 64;
@@ -15374,7 +17394,7 @@ BB126_8:
 	shr.u64 	%rd68, %rd97, %r36;
 	or.b64  	%rd98, %rd68, %rd67;
 
-BB126_10:
+BB140_10:
 	mov.u64 	%rd72, -3958705157555305931;
 	// inline asm
 	{
@@ -15395,7 +17415,7 @@ BB126_10:
 	}
 	// inline asm
 	setp.lt.s64	%p8, %rd100, 1;
-	@%p8 bra 	BB126_12;
+	@%p8 bra 	BB140_12;
 
 	// inline asm
 	{
@@ -15414,7 +17434,7 @@ BB126_10:
 	// inline asm
 	add.s32 	%r41, %r41, 1;
 
-BB126_12:
+BB140_12:
 	cvt.u64.u32	%rd79, %r40;
 	shl.b64 	%rd80, %rd79, 32;
 	mov.u32 	%r37, 1022;
@@ -15429,7 +17449,7 @@ BB126_12:
 	or.b64  	%rd88, %rd87, %rd80;
 	mov.b64 	 %fd4, %rd88;
 
-BB126_13:
+BB140_13:
 	st.param.f64	[func_retval0+0], %fd4;
 	ret;
 }
@@ -15457,7 +17477,7 @@ BB126_13:
 	}
 	shr.u32 	%r51, %r50, 20;
 	setp.ne.s32	%p1, %r51, 0;
-	@%p1 bra 	BB127_2;
+	@%p1 bra 	BB141_2;
 
 	mul.f64 	%fd14, %fd12, 0d4350000000000000;
 	{
@@ -15471,13 +17491,13 @@ BB126_13:
 	shr.u32 	%r16, %r50, 20;
 	add.s32 	%r51, %r16, -54;
 
-BB127_2:
+BB141_2:
 	add.s32 	%r52, %r51, -1023;
 	and.b32  	%r17, %r50, -2146435073;
 	or.b32  	%r18, %r17, 1072693248;
 	mov.b64 	%fd135, {%r49, %r18};
 	setp.lt.u32	%p2, %r18, 1073127583;
-	@%p2 bra 	BB127_4;
+	@%p2 bra 	BB141_4;
 
 	{
 	.reg .b32 %temp; 
@@ -15491,7 +17511,7 @@ BB127_2:
 	mov.b64 	%fd135, {%r19, %r21};
 	add.s32 	%r52, %r51, -1022;
 
-BB127_4:
+BB141_4:
 	add.f64 	%fd15, %fd135, 0d3FF0000000000000;
 	rcp.approx.ftz.f64 	%fd16, %fd15;
 	neg.f64 	%fd17, %fd15;
@@ -15654,13 +17674,13 @@ BB127_4:
 	mov.b32 	 %f2, %r35;
 	abs.f32 	%f1, %f2;
 	setp.lt.f32	%p4, %f1, 0f4086232B;
-	@%p4 bra 	BB127_7;
+	@%p4 bra 	BB141_7;
 
 	setp.lt.f64	%p5, %fd4, 0d0000000000000000;
 	add.f64 	%fd129, %fd4, 0d7FF0000000000000;
 	selp.f64	%fd136, 0d0000000000000000, %fd129, %p5;
 	setp.geu.f32	%p6, %f1, 0f40874800;
-	@%p6 bra 	BB127_7;
+	@%p6 bra 	BB141_7;
 
 	mov.f64 	%fd134, 0d4338000000000000;
 	mov.f64 	%fd133, 0d3FF71547652B82FE;
@@ -15682,26 +17702,26 @@ BB127_4:
 	mov.b64 	%fd131, {%r44, %r43};
 	mul.f64 	%fd136, %fd130, %fd131;
 
-BB127_7:
+BB141_7:
 	{
 	.reg .b32 %temp; 
 	mov.b64 	{%temp, %r45}, %fd136;
 	}
 	and.b32  	%r46, %r45, 2147483647;
 	setp.ne.s32	%p7, %r46, 2146435072;
-	@%p7 bra 	BB127_9;
+	@%p7 bra 	BB141_9;
 
 	{
 	.reg .b32 %temp; 
 	mov.b64 	{%r47, %temp}, %fd136;
 	}
 	setp.eq.s32	%p8, %r47, 0;
-	@%p8 bra 	BB127_10;
+	@%p8 bra 	BB141_10;
 
-BB127_9:
+BB141_9:
 	fma.rn.f64 	%fd136, %fd136, %fd5, %fd136;
 
-BB127_10:
+BB141_10:
 	st.param.f64	[func_retval0+0], %fd136;
 	ret;
 }