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

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

Repository: systemml
Updated Branches:
  refs/heads/master 8fb74b158 -> 628ffad1b


http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/java/org/apache/sysml/utils/GPUStatistics.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/GPUStatistics.java b/src/main/java/org/apache/sysml/utils/GPUStatistics.java
index c0f1d5e..e6ea4a7 100644
--- a/src/main/java/org/apache/sysml/utils/GPUStatistics.java
+++ b/src/main/java/org/apache/sysml/utils/GPUStatistics.java
@@ -26,7 +26,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.sysml.api.DMLScript;
 
@@ -35,190 +35,186 @@ import org.apache.sysml.api.DMLScript;
  * Printed as part of {@link Statistics}.
  */
 public class GPUStatistics {
-	//TODO fix formatting 
-	//TODO replace AtomicLong with LongAdder
-	
-	
-  // Whether or not extra per-instruction statistics will be recorded and shown for the GPU
-  public static boolean DISPLAY_STATISTICS = false;
-
-  private static int iNoOfExecutedGPUInst = 0;
-
-  public static long cudaInitTime = 0;
-  public static long cudaLibrariesInitTime = 0;
-  public static AtomicLong cudaSparseToDenseTime = new AtomicLong(0);		// time spent in converting sparse matrix block to dense
-  public static AtomicLong cudaDenseToSparseTime = new AtomicLong(0);		// time spent in converting dense matrix block to sparse
-  public static AtomicLong cudaSparseConversionTime = new AtomicLong(0);	// time spent in converting between sparse block types
-  public static AtomicLong cudaSparseToDenseCount = new AtomicLong(0);
-  public static AtomicLong cudaDenseToSparseCount = new AtomicLong(0);
-  public static AtomicLong cudaSparseConversionCount = new AtomicLong(0);
-
-  public static AtomicLong cudaAllocTime = new AtomicLong(0);             // time spent in allocating memory on the GPU
-  public static AtomicLong cudaDeAllocTime = new AtomicLong(0);           // time spent in deallocating memory on the GPU
-  public static AtomicLong cudaMemSet0Time = new AtomicLong(0);           // time spent in setting memory to 0 on the GPU (part of reusing and for new allocates)
-  public static AtomicLong cudaToDevTime = new AtomicLong(0);             // time spent in copying data from host (CPU) to device (GPU) memory
-  public static AtomicLong cudaFromDevTime = new AtomicLong(0);           // time spent in copying data from device to host
-  public static AtomicLong cudaAllocCount = new AtomicLong(0);
-  public static AtomicLong cudaDeAllocCount = new AtomicLong(0);
-  public static AtomicLong cudaMemSet0Count = new AtomicLong(0);
-  public static AtomicLong cudaToDevCount = new AtomicLong(0);
-  public static AtomicLong cudaFromDevCount = new AtomicLong(0);
-  public static AtomicLong cudaEvictionCount = new AtomicLong(0);
-
-  // Per instruction miscellaneous timers.
-  // Used to record events in a CP Heavy Hitter instruction and
-  // provide a breakdown of how time was spent in that instruction
-  private static HashMap<String, HashMap<String, Long>> _cpInstMiscTime = new HashMap<String, HashMap<String, Long>> ();
-  private static HashMap<String, HashMap<String, Long>> _cpInstMiscCount = new HashMap<String, HashMap<String, Long>> ();
-
-  /**
-   * Resets the miscellaneous timers & counters
-   */
-  public static void resetMiscTimers(){
-    _cpInstMiscTime.clear();
-    _cpInstMiscCount.clear();
-  }
-
-  /**
-   * Resets all the cuda counters and timers, including the misc timers & counters
-   */
-  public static void reset(){
-    cudaInitTime = 0;
-    cudaLibrariesInitTime = 0;
-    cudaAllocTime.set(0);
-    cudaDeAllocTime.set(0);
-    cudaMemSet0Time.set(0);
-    cudaMemSet0Count.set(0);
-    cudaToDevTime.set(0);
-    cudaFromDevTime.set(0);
-    cudaAllocCount.set(0);
-    cudaDeAllocCount.set(0);
-    cudaToDevCount.set(0);
-    cudaFromDevCount.set(0);
-    cudaEvictionCount.set(0);
-    resetMiscTimers();
-  }
-
-
-  public static synchronized void setNoOfExecutedGPUInst(int numJobs) {
-    iNoOfExecutedGPUInst = numJobs;
-  }
-
-  public static synchronized void incrementNoOfExecutedGPUInst() {
-    iNoOfExecutedGPUInst ++;
-  }
-
-  public static synchronized int getNoOfExecutedGPUInst() {
-    return iNoOfExecutedGPUInst;
-  }
-
-  /**
-   * "Maintains" or adds time to miscellaneous timers per instruction/op, also increments associated count
-   * @param instructionName	name of the instruction/op
-   * @param miscTimer				name of the miscellaneous timer
-   * @param timeNanos				time in nano seconds
-   * @param incrementCount	how much to increment the count of the miscTimer by
-   */
-  public synchronized static void maintainCPMiscTimes( String instructionName, String miscTimer, long timeNanos, long incrementCount)
-  {
-    if (!(DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS))
-      return;
-
-    HashMap<String, Long> miscTimesMap = _cpInstMiscTime.get(instructionName);
-    if (miscTimesMap == null) {
-      miscTimesMap = new HashMap<String, Long>();
-      _cpInstMiscTime.put(instructionName, miscTimesMap);
-    }
-    Long oldVal = miscTimesMap.get(miscTimer);
-    Long newVal = timeNanos + ((oldVal!=null) ? oldVal : 0);
-    miscTimesMap.put(miscTimer, newVal);
-
-    HashMap<String, Long> miscCountMap = _cpInstMiscCount.get(instructionName);
-    if (miscCountMap == null){
-      miscCountMap = new HashMap<String, Long>();
-      _cpInstMiscCount.put(instructionName, miscCountMap);
-    }
-    Long oldCnt = miscCountMap.get(miscTimer);
-    Long newCnt = incrementCount + ((oldCnt!=null) ? oldCnt : 0);
-    miscCountMap.put(miscTimer, newCnt);
-  }
-
-  /**
-   * "Maintains" or adds time to miscellaneous timers per instruction/op, also increments associated count by 1
-   * @param instructionName	name of the instruction/op
-   * @param miscTimer				name of the miscellaneous timer
-   * @param timeNanos				time in nano seconds
-   */
-  public synchronized static void maintainCPMiscTimes( String instructionName, String miscTimer, long timeNanos){
-    maintainCPMiscTimes(instructionName, miscTimer, timeNanos, 1);
-  }
-
-  /**
-   * Used to print misc timers (and their counts) for a given instruction/op
-   * @param instructionName name of the instruction/op
-   * @return  a formatted string of misc timers for a given instruction/op
-   */
-  public static String getStringForCPMiscTimesPerInstruction(String instructionName) {
-    StringBuffer sb = new StringBuffer();
-    HashMap<String, Long> miscTimerMap = _cpInstMiscTime.get(instructionName);
-    if (miscTimerMap != null) {
-      List<Map.Entry<String, Long>> sortedList = new ArrayList<Map.Entry<String, Long>>(miscTimerMap.entrySet());
-      // Sort the times to display by the most expensive first
-      Collections.sort(sortedList, new Comparator<Map.Entry<String, Long>>() {
-        @Override
-        public int compare(Map.Entry<String, Long> o1, Map.Entry<String, Long> o2) {
-          return (int) (o1.getValue() - o2.getValue());
-        }
-      });
-      Iterator<Map.Entry<String, Long>> miscTimeIter = sortedList.iterator();
-      HashMap<String, Long> miscCountMap = _cpInstMiscCount.get(instructionName);
-      while (miscTimeIter.hasNext()) {
-        Map.Entry<String, Long> e = miscTimeIter.next();
-        String miscTimerName = e.getKey();
-        Long miscTimerTime = e.getValue();
-        Long miscCount = miscCountMap.get(miscTimerName);
-        sb.append(miscTimerName + "[" + String.format("%.3f", (double) miscTimerTime / 1000000000.0) + "s," + miscCount + "]");
-        if (miscTimeIter.hasNext())
-          sb.append(", ");
-      }
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Used to print out cuda timers & counters
-   * @return a formatted string of cuda timers & counters
-   */
-  public static String getStringForCudaTimers() {
-    StringBuffer sb = new StringBuffer();
-    sb.append("CUDA/CuLibraries init time:\t" + String.format("%.3f", cudaInitTime*1e-9) + "/"
-            + String.format("%.3f", cudaLibrariesInitTime*1e-9) + " sec.\n");
-    sb.append("Number of executed GPU inst:\t" + getNoOfExecutedGPUInst() + ".\n");
-    sb.append("GPU mem tx time  (alloc/dealloc/set0/toDev/fromDev):\t"
-            + String.format("%.3f", cudaAllocTime.get()*1e-9) + "/"
-            + String.format("%.3f", cudaDeAllocTime.get()*1e-9) + "/"
-            + String.format("%.3f", cudaMemSet0Time.get()*1e-9) + "/"
-            + String.format("%.3f", cudaToDevTime.get()*1e-9) + "/"
-            + String.format("%.3f", cudaFromDevTime.get()*1e-9)  + " sec.\n");
-    sb.append("GPU mem tx count (alloc/dealloc/set0/toDev/fromDev/evict):\t"
-            + cudaAllocCount.get() + "/"
-            + cudaDeAllocCount.get() + "/"
-            + cudaMemSet0Count.get() + "/"
-            + cudaSparseConversionCount.get() + "/"
-            + cudaToDevCount.get() + "/"
-            + cudaFromDevCount.get() + "/"
-            + cudaEvictionCount.get() + ".\n");
-    sb.append("GPU conversion time  (sparseConv/sp2dense/dense2sp):\t"
-            + String.format("%.3f", cudaSparseConversionTime.get()*1e-9) + "/"
-            + String.format("%.3f", cudaSparseToDenseTime.get()*1e-9) + "/"
-            + String.format("%.3f", cudaDenseToSparseTime.get()*1e-9) + " sec.\n");
-    sb.append("GPU conversion count (sparseConv/sp2dense/dense2sp):\t"
-            + cudaSparseConversionCount.get() + "/"
-            + cudaSparseToDenseCount.get() + "/"
-            + cudaDenseToSparseCount.get() + ".\n");
-
-    return sb.toString();
-  }
+	// Whether or not extra per-instruction statistics will be recorded and shown for the GPU
+	public static boolean DISPLAY_STATISTICS = false;
+
+	private static int iNoOfExecutedGPUInst = 0;
+
+	public static long cudaInitTime = 0;
+	public static long cudaLibrariesInitTime = 0;
+	public static LongAdder cudaSparseToDenseTime = new LongAdder();		// time spent in converting sparse matrix block to dense
+	public static LongAdder cudaDenseToSparseTime = new LongAdder();		// time spent in converting dense matrix block to sparse
+	public static LongAdder cudaSparseConversionTime = new LongAdder();	// time spent in converting between sparse block types
+	public static LongAdder cudaSparseToDenseCount = new LongAdder();
+	public static LongAdder cudaDenseToSparseCount = new LongAdder();
+	public static LongAdder cudaSparseConversionCount = new LongAdder();
+
+	public static LongAdder cudaAllocTime = new LongAdder();             // time spent in allocating memory on the GPU
+	public static LongAdder cudaDeAllocTime = new LongAdder();           // time spent in deallocating memory on the GPU
+	public static LongAdder cudaMemSet0Time = new LongAdder();           // time spent in setting memory to 0 on the GPU (part of reusing and for new allocates)
+	public static LongAdder cudaToDevTime = new LongAdder();             // time spent in copying data from host (CPU) to device (GPU) memory
+	public static LongAdder cudaFromDevTime = new LongAdder();           // time spent in copying data from device to host
+	public static LongAdder cudaAllocCount = new LongAdder();
+	public static LongAdder cudaDeAllocCount = new LongAdder();
+	public static LongAdder cudaMemSet0Count = new LongAdder();
+	public static LongAdder cudaToDevCount = new LongAdder();
+	public static LongAdder cudaFromDevCount = new LongAdder();
+	public static LongAdder cudaEvictionCount = new LongAdder();
+
+	// Per instruction miscellaneous timers.
+	// Used to record events in a CP Heavy Hitter instruction and
+	// provide a breakdown of how time was spent in that instruction
+	private static HashMap<String, HashMap<String, Long>> _cpInstMiscTime = new HashMap<String, HashMap<String, Long>> ();
+	private static HashMap<String, HashMap<String, Long>> _cpInstMiscCount = new HashMap<String, HashMap<String, Long>> ();
+
+	/**
+	 * Resets the miscellaneous timers & counters
+	 */
+	public static void resetMiscTimers(){
+		_cpInstMiscTime.clear();
+		_cpInstMiscCount.clear();
+	}
+
+	/**
+	 * Resets all the cuda counters and timers, including the misc timers & counters
+	 */
+	public static void reset(){
+		cudaInitTime = 0;
+		cudaLibrariesInitTime = 0;
+		cudaAllocTime.reset();
+		cudaDeAllocTime.reset();
+		cudaMemSet0Time.reset();
+		cudaMemSet0Count.reset();
+		cudaToDevTime.reset();
+		cudaFromDevTime.reset();
+		cudaAllocCount.reset();
+		cudaDeAllocCount.reset();
+		cudaToDevCount.reset();
+		cudaFromDevCount.reset();
+		cudaEvictionCount.reset();
+		resetMiscTimers();
+	}
+
+
+	public static synchronized void setNoOfExecutedGPUInst(int numJobs) {
+		iNoOfExecutedGPUInst = numJobs;
+	}
+
+	public static synchronized void incrementNoOfExecutedGPUInst() {
+		iNoOfExecutedGPUInst ++;
+	}
+
+	public static synchronized int getNoOfExecutedGPUInst() {
+		return iNoOfExecutedGPUInst;
+	}
+
+	/**
+	 * "Maintains" or adds time to miscellaneous timers per instruction/op, also increments associated count
+	 * @param instructionName	name of the instruction/op
+	 * @param miscTimer				name of the miscellaneous timer
+	 * @param timeNanos				time in nano seconds
+	 * @param incrementCount	how much to increment the count of the miscTimer by
+	 */
+	public synchronized static void maintainCPMiscTimes( String instructionName, String miscTimer, long timeNanos, long incrementCount)
+	{
+		if (!(DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS))
+			return;
+
+		HashMap<String, Long> miscTimesMap = _cpInstMiscTime.get(instructionName);
+		if (miscTimesMap == null) {
+			miscTimesMap = new HashMap<String, Long>();
+			_cpInstMiscTime.put(instructionName, miscTimesMap);
+		}
+		Long oldVal = miscTimesMap.get(miscTimer);
+		Long newVal = timeNanos + ((oldVal!=null) ? oldVal : 0);
+		miscTimesMap.put(miscTimer, newVal);
+
+		HashMap<String, Long> miscCountMap = _cpInstMiscCount.get(instructionName);
+		if (miscCountMap == null){
+			miscCountMap = new HashMap<String, Long>();
+			_cpInstMiscCount.put(instructionName, miscCountMap);
+		}
+		Long oldCnt = miscCountMap.get(miscTimer);
+		Long newCnt = incrementCount + ((oldCnt!=null) ? oldCnt : 0);
+		miscCountMap.put(miscTimer, newCnt);
+	}
+
+	/**
+	 * "Maintains" or adds time to miscellaneous timers per instruction/op, also increments associated count by 1
+	 * @param instructionName	name of the instruction/op
+	 * @param miscTimer				name of the miscellaneous timer
+	 * @param timeNanos				time in nano seconds
+	 */
+	public synchronized static void maintainCPMiscTimes( String instructionName, String miscTimer, long timeNanos){
+		maintainCPMiscTimes(instructionName, miscTimer, timeNanos, 1);
+	}
+
+	/**
+	 * Used to print misc timers (and their counts) for a given instruction/op
+	 * @param instructionName name of the instruction/op
+	 * @return  a formatted string of misc timers for a given instruction/op
+	 */
+	public static String getStringForCPMiscTimesPerInstruction(String instructionName) {
+		StringBuffer sb = new StringBuffer();
+		HashMap<String, Long> miscTimerMap = _cpInstMiscTime.get(instructionName);
+		if (miscTimerMap != null) {
+			List<Map.Entry<String, Long>> sortedList = new ArrayList<Map.Entry<String, Long>>(miscTimerMap.entrySet());
+			// Sort the times to display by the most expensive first
+			Collections.sort(sortedList, new Comparator<Map.Entry<String, Long>>() {
+				@Override
+				public int compare(Map.Entry<String, Long> o1, Map.Entry<String, Long> o2) {
+					return (int) (o1.getValue() - o2.getValue());
+				}
+			});
+			Iterator<Map.Entry<String, Long>> miscTimeIter = sortedList.iterator();
+			HashMap<String, Long> miscCountMap = _cpInstMiscCount.get(instructionName);
+			while (miscTimeIter.hasNext()) {
+				Map.Entry<String, Long> e = miscTimeIter.next();
+				String miscTimerName = e.getKey();
+				Long miscTimerTime = e.getValue();
+				Long miscCount = miscCountMap.get(miscTimerName);
+				sb.append(miscTimerName + "[" + String.format("%.3f", (double) miscTimerTime / 1000000000.0) + "s," + miscCount + "]");
+				if (miscTimeIter.hasNext())
+					sb.append(", ");
+			}
+		}
+		return sb.toString();
+	}
+
+	/**
+	 * Used to print out cuda timers & counters
+	 * @return a formatted string of cuda timers & counters
+	 */
+	public static String getStringForCudaTimers() {
+		StringBuffer sb = new StringBuffer();
+		sb.append("CUDA/CuLibraries init time:\t" + String.format("%.3f", cudaInitTime*1e-9) + "/"
+				+ String.format("%.3f", cudaLibrariesInitTime*1e-9) + " sec.\n");
+		sb.append("Number of executed GPU inst:\t" + getNoOfExecutedGPUInst() + ".\n");
+		sb.append("GPU mem tx time  (alloc/dealloc/set0/toDev/fromDev):\t"
+				+ String.format("%.3f", cudaAllocTime.longValue()*1e-9) + "/"
+				+ String.format("%.3f", cudaDeAllocTime.longValue()*1e-9) + "/"
+				+ String.format("%.3f", cudaMemSet0Time.longValue()*1e-9) + "/"
+				+ String.format("%.3f", cudaToDevTime.longValue()*1e-9) + "/"
+				+ String.format("%.3f", cudaFromDevTime.longValue()*1e-9)  + " sec.\n");
+		sb.append("GPU mem tx count (alloc/dealloc/set0/toDev/fromDev/evict):\t"
+				+ cudaAllocCount.longValue() + "/"
+				+ cudaDeAllocCount.longValue() + "/"
+				+ cudaMemSet0Count.longValue() + "/"
+				+ cudaSparseConversionCount.longValue() + "/"
+				+ cudaToDevCount.longValue() + "/"
+				+ cudaFromDevCount.longValue() + "/"
+				+ cudaEvictionCount.longValue() + ".\n");
+		sb.append("GPU conversion time  (sparseConv/sp2dense/dense2sp):\t"
+				+ String.format("%.3f", cudaSparseConversionTime.longValue()*1e-9) + "/"
+				+ String.format("%.3f", cudaSparseToDenseTime.longValue()*1e-9) + "/"
+				+ String.format("%.3f", cudaDenseToSparseTime.longValue()*1e-9) + " sec.\n");
+		sb.append("GPU conversion count (sparseConv/sp2dense/dense2sp):\t"
+				+ cudaSparseConversionCount.longValue() + "/"
+				+ cudaSparseToDenseCount.longValue() + "/"
+				+ cudaDenseToSparseCount.longValue() + ".\n");
+
+		return sb.toString();
+	}
 
 
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/test/java/org/apache/sysml/test/gpu/RightIndexingTests.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/gpu/RightIndexingTests.java b/src/test/java/org/apache/sysml/test/gpu/RightIndexingTests.java
new file mode 100644
index 0000000..e891b7f
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/gpu/RightIndexingTests.java
@@ -0,0 +1,74 @@
+/*
+ * 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.api.mlcontext.Matrix;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * This test uses the script: O = X[rl:ru,cl:cu]
+ */
+public class RightIndexingTests extends GPUTests {
+	
+	private final static String TEST_NAME = "RightIndexingTests";
+	private final int [] indexes1 = new int[] {1, 5, 10, 100};
+	private final int [] indexes2 = new int[] {1, 5, 10, 100};
+	private final double[] sparsities = new double[] { 0.0, 0.03, 0.3, 0.9 };
+	private final int seed = 42;
+
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(TEST_DIR, TEST_NAME);
+		getAndLoadTestConfiguration(TEST_NAME);
+	}
+
+	@Test
+	public void runRightIndexingTest() {
+		int dim1 = Arrays.stream(indexes1).max().getAsInt();
+		int dim2 = Arrays.stream(indexes2).max().getAsInt();
+		for(int i1 = 0; i1 < indexes1.length; i1++) {
+			for(int i2 = i1; i2 < indexes1.length; i2++) {
+				for(int j1 = 0; j1 < indexes2.length; j1++) {
+					for(int j2 = j1; j2 < indexes2.length; j2++) {
+						int rl = indexes1[i1]; int ru = indexes1[i2];
+						int cl = indexes2[j1]; int cu = indexes2[j2];
+						for (int k = 0; k < sparsities.length; k++) {
+							double sparsity = sparsities[k];
+							Matrix X = generateInputMatrix(spark, dim1, dim2, sparsity, seed);
+							Matrix Y = generateInputMatrix(spark, dim1, dim2, sparsity, seed);
+							HashMap<String, Object> inputs = new HashMap<>();
+							inputs.put("X", X);
+							String scriptStr = "O = X[" + rl + ":" + ru + "," +  cl + ":" + cu + "];";
+							System.out.println("Executing the script: " + scriptStr);
+							List<Object> cpuOut = runOnCPU(spark, scriptStr, inputs, Arrays.asList("O"));
+							List<Object> gpuOut = runOnGPU(spark, scriptStr, inputs, Arrays.asList("O"));
+							assertEqualObjects(cpuOut.get(0), gpuOut.get(0));
+						}
+					}
+				}
+			}
+		}
+	}
+}


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

Posted by ni...@apache.org.
[SYSTEMML-1793] Support matrix range indexing on GPU

- This commit supports matrix range indexing (i.e. right indexing) without requiring sparse to dense conversion of inputs. Note: this PR only supports dense output.
- Also, added RightIndexingTests in gpu package.

Closes #637.


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

Branch: refs/heads/master
Commit: 628ffad1b26a056edd3782787f89b8bf7711f0e5
Parents: 8fb74b1
Author: Niketan Pansare <np...@us.ibm.com>
Authored: Thu Aug 24 14:38:52 2017 -0700
Committer: Niketan Pansare <np...@us.ibm.com>
Committed: Thu Aug 24 14:40:34 2017 -0700

----------------------------------------------------------------------
 src/main/cpp/kernels/SystemML.cu                |   30 +
 src/main/cpp/kernels/SystemML.ptx               | 2817 +++++++++---------
 .../java/org/apache/sysml/hops/IndexingOp.java  |   11 +-
 .../instructions/GPUInstructionParser.java      |    8 +
 .../instructions/gpu/GPUInstruction.java        |    6 +-
 .../gpu/MatrixIndexingGPUInstruction.java       |  148 +
 .../instructions/gpu/context/CSRPointer.java    |   20 +-
 .../instructions/gpu/context/GPUContext.java    |   66 +-
 .../instructions/gpu/context/GPUObject.java     |   48 +-
 .../runtime/matrix/data/LibMatrixCUDA.java      |  930 +++---
 .../org/apache/sysml/utils/GPUStatistics.java   |  366 ++-
 .../sysml/test/gpu/RightIndexingTests.java      |   74 +
 12 files changed, 2472 insertions(+), 2052 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/cpp/kernels/SystemML.cu
----------------------------------------------------------------------
diff --git a/src/main/cpp/kernels/SystemML.cu b/src/main/cpp/kernels/SystemML.cu
index dcd64b2..d64d8aa 100644
--- a/src/main/cpp/kernels/SystemML.cu
+++ b/src/main/cpp/kernels/SystemML.cu
@@ -28,6 +28,36 @@ nvcc -ptx -arch=sm_30 SystemML.cu
 
 
 /**
+ * Performs a slice operation where the input matrix is sparse and the output matrix is dense.
+ * This function avoids unnecessary sparse to dense conversion of the input matrix.
+ * 
+ * @params inVal input val pointer
+ * @params inRowPtr input row pointer
+ * @params colInd input col index pointer
+ * @params ret dense output pointer
+ * @param rl row lower
+ * @param ru row upper
+ * @param cl column lower
+ * @param cu column upper
+ */
+extern "C"
+__global__ void slice_sparse_dense(double* inVal, int* inRowPtr, int* colInd, double* ret, int rl, int ru, int cl, int cu) {
+	int index = blockIdx.x * blockDim.x + threadIdx.x;
+	int rowIndex = index + rl;
+    if (rowIndex <= ru){
+    	int retClen = cu - cl + 1;
+    	// Iterate over elements of the row 'rowIndex'.
+    	for(int i = inRowPtr[rowIndex]; i < inRowPtr[rowIndex+1]; i++) {
+    		// Only slice if the index falls into the given range
+    		if(cl <= colInd[i] && colInd[i] <= cu) {
+    			ret[ index*retClen + (colInd[i] - cl) ] = inVal[i];
+    		}
+    	}
+    }
+}
+
+
+/**
  * Does a copy of upper to lower triangle of the given matrix
  * @param ret the input and output array allocated on the GPU
  * @param dim the number of rows of the square matrix ret


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

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

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

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

http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/java/org/apache/sysml/runtime/instructions/gpu/MatrixIndexingGPUInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/MatrixIndexingGPUInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/MatrixIndexingGPUInstruction.java
new file mode 100644
index 0000000..5e2c8fc
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/MatrixIndexingGPUInstruction.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sysml.runtime.instructions.gpu;
+
+import org.apache.sysml.parser.Expression.DataType;
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
+import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
+import org.apache.sysml.runtime.instructions.InstructionUtils;
+import org.apache.sysml.runtime.instructions.cp.CPOperand;
+import org.apache.sysml.runtime.matrix.data.LibMatrixCUDA;
+import org.apache.sysml.runtime.matrix.operators.Operator;
+import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
+import org.apache.sysml.runtime.util.IndexRange;
+import org.apache.sysml.utils.GPUStatistics;
+
+public class MatrixIndexingGPUInstruction extends GPUInstruction {
+	CPOperand rowLower, rowUpper, colLower, colUpper;
+	CPOperand input1; CPOperand input2; CPOperand output;
+	
+	public MatrixIndexingGPUInstruction(Operator op, CPOperand in, 
+			CPOperand rl, CPOperand ru, CPOperand cl, CPOperand cu, CPOperand out, String opcode, String istr){
+		super(op, opcode, istr);
+		_gputype = GPUINSTRUCTION_TYPE.MatrixIndexing;
+		rowLower = rl;
+		rowUpper = ru;
+		colLower = cl;
+		colUpper = cu;
+		input1 = in;
+		output = out;
+	}
+	
+	public MatrixIndexingGPUInstruction(Operator op, CPOperand lhsInput, CPOperand rhsInput, 
+			CPOperand rl, CPOperand ru, CPOperand cl, CPOperand cu, CPOperand out, String opcode, String istr){
+		super(op, opcode, istr);
+		_gputype = GPUINSTRUCTION_TYPE.MatrixIndexing;
+		rowLower = rl;
+		rowUpper = ru;
+		colLower = cl;
+		colUpper = cu;
+		input1 = lhsInput;
+		input2 = rhsInput;
+		output = out;
+	}
+	
+	public static MatrixIndexingGPUInstruction parseInstruction ( String str ) throws DMLRuntimeException {
+		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);
+		String opcode = parts[0];
+		
+		if ( opcode.equalsIgnoreCase("rangeReIndex") ) {
+			if ( parts.length == 7 ) {
+				// Example: rangeReIndex:mVar1:Var2:Var3:Var4:Var5:mVar6
+				CPOperand in, rl, ru, cl, cu, out;
+				in = new CPOperand();
+				rl = new CPOperand();
+				ru = new CPOperand();
+				cl = new CPOperand();
+				cu = new CPOperand();
+				out = new CPOperand();
+				in.split(parts[1]);
+				rl.split(parts[2]);
+				ru.split(parts[3]);
+				cl.split(parts[4]);
+				cu.split(parts[5]);
+				out.split(parts[6]);
+				if( in.getDataType()==DataType.MATRIX )
+					return new MatrixIndexingGPUInstruction(new SimpleOperator(null), in, rl, ru, cl, cu, out, opcode, str);
+				else 
+					throw new DMLRuntimeException("Can index only on Matrices in GPU");
+			}
+			else {
+				throw new DMLRuntimeException("Invalid number of operands in instruction: " + str);
+			}
+		} 
+		else if ( opcode.equalsIgnoreCase("leftIndex")) {
+			if ( parts.length == 8 ) {
+				// Example: leftIndex:mVar1:mvar2:Var3:Var4:Var5:Var6:mVar7
+				CPOperand lhsInput, rhsInput, rl, ru, cl, cu, out;
+				lhsInput = new CPOperand();
+				rhsInput = new CPOperand();
+				rl = new CPOperand();
+				ru = new CPOperand();
+				cl = new CPOperand();
+				cu = new CPOperand();
+				out = new CPOperand();
+				lhsInput.split(parts[1]);
+				rhsInput.split(parts[2]);
+				rl.split(parts[3]);
+				ru.split(parts[4]);
+				cl.split(parts[5]);
+				cu.split(parts[6]);
+				out.split(parts[7]);
+				if( lhsInput.getDataType()==DataType.MATRIX )
+					return new MatrixIndexingGPUInstruction(new SimpleOperator(null), lhsInput, rhsInput, rl, ru, cl, cu, out, opcode, str);
+				else 
+					throw new DMLRuntimeException("Can index only on Matrices in GPU");
+			}
+			else {
+				throw new DMLRuntimeException("Invalid number of operands in instruction: " + str);
+			}
+		}
+		else {
+			throw new DMLRuntimeException("Unknown opcode while parsing a MatrixIndexingGPUInstruction: " + str);
+		}
+	}
+
+	@Override
+	public void processInstruction(ExecutionContext ec)
+			throws DMLRuntimeException {
+		GPUStatistics.incrementNoOfExecutedGPUInst();
+		String opcode = getOpcode();
+		
+		IndexRange ixrange = getIndexRange(ec);
+		if ( opcode.equalsIgnoreCase("rangeReIndex") ) {
+			MatrixObject mat1 = getMatrixInputForGPUInstruction(ec, input1.getName());
+			LibMatrixCUDA.sliceOperations(ec, ec.getGPUContext(0), getExtendedOpcode(), mat1, ixrange, output.getName());
+			ec.releaseMatrixInputForGPUInstruction(input1.getName());
+			ec.releaseMatrixOutputForGPUInstruction(output.getName());
+		}
+		else {
+			throw new DMLRuntimeException("Unsupported GPU operator:" + opcode);
+		}
+	}
+	
+	IndexRange getIndexRange(ExecutionContext ec) throws DMLRuntimeException {
+		return new IndexRange( //rl, ru, cl, ru
+			(int)(ec.getScalarInput(rowLower.getName(), rowLower.getValueType(), rowLower.isLiteral()).getLongValue()-1),
+			(int)(ec.getScalarInput(rowUpper.getName(), rowUpper.getValueType(), rowUpper.isLiteral()).getLongValue()-1),
+			(int)(ec.getScalarInput(colLower.getName(), colLower.getValueType(), colLower.isLiteral()).getLongValue()-1),
+			(int)(ec.getScalarInput(colUpper.getName(), colUpper.getValueType(), colUpper.isLiteral()).getLongValue()-1));		
+	}
+}
\ No newline at end of file

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

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

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


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

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/src/main/cpp/kernels/SystemML.ptx
----------------------------------------------------------------------
diff --git a/src/main/cpp/kernels/SystemML.ptx b/src/main/cpp/kernels/SystemML.ptx
index 7778317..f6ba15a 100644
--- a/src/main/cpp/kernels/SystemML.ptx
+++ b/src/main/cpp/kernels/SystemML.ptx
@@ -1,8 +1,8 @@
 //
 // Generated by NVIDIA NVVM Compiler
 //
-// Compiler Build ID: CL-21554848
-// Cuda compilation tools, release 8.0, V8.0.61
+// Compiler Build ID: CL-21124049
+// Cuda compilation tools, release 8.0, V8.0.44
 // Based on LLVM 3.4svn
 //
 
@@ -10,7 +10,7 @@
 .target sm_30
 .address_size 64
 
-	// .globl	copy_u2l_dense
+	// .globl	slice_sparse_dense
 .func  (.param .b64 func_retval0) __internal_trig_reduction_slowpathd
 (
 	.param .b64 __internal_trig_reduction_slowpathd_param_0,
@@ -27,6 +27,86 @@
 .const .align 8 .b8 __cudart_i2opi_d[144] = {8, 93, 141, 31, 177, 95, 251, 107, 234, 146, 82, 138, 247, 57, 7, 61, 123, 241, 229, 235, 199, 186, 39, 117, 45, 234, 95, 158, 102, 63, 70, 79, 183, 9, 203, 39, 207, 126, 54, 109, 31, 109, 10, 90, 139, 17, 47, 239, 15, 152, 5, 222, 255, 151, 248, 31, 59, 40, 249, 189, 139, 95, 132, 156, 244, 57, 83, 131, 57, 214, 145, 57, 65, 126, 95, 180, 38, 112, 156, 233, 132, 68, 187, 46, 245, 53, 130, 232, 62, 167, 41, 177, 28, 235, 29, 254, 28, 146, 209, 9, 234, 46, 73, 6, 224, 210, 77, 66, 58, 110, 36, 183, 97, 197, 187, 222, 171, 99, 81, 254, 65, 144, 67, 60, 153, 149, 98, 219, 192, 221, 52, 245, 209, 87, 39, 252, 41, 21, 68, 78, 110, 131, 249, 162};
 .const .align 8 .b8 __cudart_sin_cos_coeffs[128] = {186, 94, 120, 249, 101, 219, 229, 61, 70, 210, 176, 44, 241, 229, 90, 190, 146, 227, 172, 105, 227, 29, 199, 62, 161, 98, 219, 25, 160, 1, 42, 191, 24, 8, 17, 17, 17, 17, 129, 63, 84, 85, 85, 85, 85, 85, 197, 191, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 100, 129, 253, 32, 131, 255, 168, 189, 40, 133, 239, 193, 167, 238, 33, 62, 217, 230, 6, 142, 79, 126, 146, 190, 233, 188, 221, 25, 160, 1, 250, 62, 71, 93, 193, 22, 108, 193, 86, 191, 81, 85, 85, 85, 85, 85, 165, 63, 0, 0, 0, 0, 0, 0, 224, 191, 0, 0, 0, 0, 0, 0, 240, 63};
 
+.visible .entry slice_sparse_dense(
+	.param .u64 slice_sparse_dense_param_0,
+	.param .u64 slice_sparse_dense_param_1,
+	.param .u64 slice_sparse_dense_param_2,
+	.param .u64 slice_sparse_dense_param_3,
+	.param .u32 slice_sparse_dense_param_4,
+	.param .u32 slice_sparse_dense_param_5,
+	.param .u32 slice_sparse_dense_param_6,
+	.param .u32 slice_sparse_dense_param_7
+)
+{
+	.reg .pred 	%p<7>;
+	.reg .b32 	%r<26>;
+	.reg .f64 	%fd<2>;
+	.reg .b64 	%rd<23>;
+
+
+	ld.param.u64 	%rd9, [slice_sparse_dense_param_0];
+	ld.param.u64 	%rd10, [slice_sparse_dense_param_1];
+	ld.param.u64 	%rd11, [slice_sparse_dense_param_2];
+	ld.param.u64 	%rd12, [slice_sparse_dense_param_3];
+	ld.param.u32 	%r14, [slice_sparse_dense_param_4];
+	ld.param.u32 	%r15, [slice_sparse_dense_param_5];
+	ld.param.u32 	%r12, [slice_sparse_dense_param_6];
+	ld.param.u32 	%r13, [slice_sparse_dense_param_7];
+	mov.u32 	%r16, %ntid.x;
+	mov.u32 	%r17, %ctaid.x;
+	mov.u32 	%r18, %tid.x;
+	mad.lo.s32 	%r1, %r16, %r17, %r18;
+	add.s32 	%r2, %r1, %r14;
+	setp.gt.s32	%p1, %r2, %r15;
+	@%p1 bra 	BB0_6;
+
+	cvta.to.global.u64 	%rd13, %rd10;
+	mul.wide.s32 	%rd14, %r2, 4;
+	add.s64 	%rd1, %rd13, %rd14;
+	ld.global.u32 	%r25, [%rd1];
+	ld.global.u32 	%r24, [%rd1+4];
+	setp.ge.s32	%p2, %r25, %r24;
+	@%p2 bra 	BB0_6;
+
+	cvta.to.global.u64 	%rd2, %rd12;
+	cvta.to.global.u64 	%rd15, %rd11;
+	mov.u32 	%r19, 1;
+	sub.s32 	%r20, %r19, %r12;
+	add.s32 	%r21, %r20, %r13;
+	mul.lo.s32 	%r22, %r1, %r21;
+	sub.s32 	%r5, %r22, %r12;
+	cvta.to.global.u64 	%rd16, %rd9;
+	mul.wide.s32 	%rd17, %r25, 8;
+	add.s64 	%rd22, %rd16, %rd17;
+	mul.wide.s32 	%rd18, %r25, 4;
+	add.s64 	%rd21, %rd15, %rd18;
+
+BB0_3:
+	ld.global.u32 	%r8, [%rd21];
+	setp.lt.s32	%p3, %r8, %r12;
+	setp.gt.s32	%p4, %r8, %r13;
+	or.pred  	%p5, %p3, %p4;
+	@%p5 bra 	BB0_5;
+
+	ld.global.f64 	%fd1, [%rd22];
+	add.s32 	%r23, %r5, %r8;
+	mul.wide.s32 	%rd19, %r23, 8;
+	add.s64 	%rd20, %rd2, %rd19;
+	st.global.f64 	[%rd20], %fd1;
+	ld.global.u32 	%r24, [%rd1+4];
+
+BB0_5:
+	add.s64 	%rd22, %rd22, 8;
+	add.s64 	%rd21, %rd21, 4;
+	add.s32 	%r25, %r25, 1;
+	setp.lt.s32	%p6, %r25, %r24;
+	@%p6 bra 	BB0_3;
+
+BB0_6:
+	ret;
+}
+
+	// .globl	copy_u2l_dense
 .visible .entry copy_u2l_dense(
 	.param .u64 copy_u2l_dense_param_0,
 	.param .u32 copy_u2l_dense_param_1,
@@ -52,10 +132,10 @@
 	setp.gt.s32	%p1, %r9, %r8;
 	setp.lt.s32	%p2, %r2, %r4;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB0_2;
-	bra.uni 	BB0_1;
+	@!%p3 bra 	BB1_2;
+	bra.uni 	BB1_1;
 
-BB0_1:
+BB1_1:
 	cvta.to.global.u64 	%rd2, %rd1;
 	mul.wide.s32 	%rd3, %r1, 8;
 	add.s64 	%rd4, %rd2, %rd3;
@@ -64,7 +144,7 @@ BB0_1:
 	add.s64 	%rd6, %rd2, %rd5;
 	st.global.f64 	[%rd6], %fd1;
 
-BB0_2:
+BB1_2:
 	ret;
 }
 
@@ -94,10 +174,10 @@ BB0_2:
 	setp.lt.s32	%p1, %r2, %r4;
 	setp.gt.s32	%p2, %r3, -1;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB1_2;
-	bra.uni 	BB1_1;
+	@!%p3 bra 	BB2_2;
+	bra.uni 	BB2_1;
 
-BB1_1:
+BB2_1:
 	rem.s32 	%r8, %r1, %r3;
 	cvta.to.global.u64 	%rd3, %rd1;
 	mad.lo.s32 	%r9, %r2, %r3, %r8;
@@ -110,7 +190,7 @@ BB1_1:
 	add.s64 	%rd7, %rd6, %rd4;
 	st.global.f64 	[%rd7], %fd3;
 
-BB1_2:
+BB2_2:
 	ret;
 }
 
@@ -142,10 +222,10 @@ BB1_2:
 	setp.lt.s32	%p1, %r2, %r4;
 	setp.gt.s32	%p2, %r3, -1;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB2_4;
-	bra.uni 	BB2_1;
+	@!%p3 bra 	BB3_4;
+	bra.uni 	BB3_1;
 
-BB2_1:
+BB3_1:
 	rem.s32 	%r8, %r1, %r3;
 	cvta.to.global.u64 	%rd5, %rd2;
 	mad.lo.s32 	%r9, %r2, %r3, %r8;
@@ -155,20 +235,20 @@ BB2_1:
 	ld.global.f64 	%fd4, [%rd7];
 	mov.f64 	%fd5, 0d0000000000000000;
 	setp.leu.f64	%p4, %fd4, 0d0000000000000000;
-	@%p4 bra 	BB2_3;
+	@%p4 bra 	BB3_3;
 
 	cvta.to.global.u64 	%rd8, %rd3;
 	shl.b64 	%rd9, %rd1, 3;
 	add.s64 	%rd10, %rd8, %rd9;
 	ld.global.f64 	%fd5, [%rd10];
 
-BB2_3:
+BB3_3:
 	cvta.to.global.u64 	%rd11, %rd4;
 	shl.b64 	%rd12, %rd1, 3;
 	add.s64 	%rd13, %rd11, %rd12;
 	st.global.f64 	[%rd13], %fd5;
 
-BB2_4:
+BB3_4:
 	ret;
 }
 
@@ -202,10 +282,10 @@ BB2_4:
 	setp.lt.s32	%p1, %r2, %r5;
 	setp.gt.s32	%p2, %r3, -1;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB3_2;
-	bra.uni 	BB3_1;
+	@!%p3 bra 	BB4_2;
+	bra.uni 	BB4_1;
 
-BB3_1:
+BB4_1:
 	rem.s32 	%r9, %r1, %r3;
 	cvta.to.global.u64 	%rd4, %rd1;
 	mad.lo.s32 	%r10, %r2, %r3, %r9;
@@ -222,7 +302,7 @@ BB3_1:
 	add.s64 	%rd11, %rd10, %rd5;
 	st.global.f64 	[%rd11], %fd3;
 
-BB3_2:
+BB4_2:
 	ret;
 }
 
@@ -261,10 +341,10 @@ BB3_2:
 	setp.lt.s32	%p1, %r1, %r5;
 	setp.gt.s32	%p2, %r3, -1;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB4_4;
-	bra.uni 	BB4_1;
+	@!%p3 bra 	BB5_4;
+	bra.uni 	BB5_1;
 
-BB4_1:
+BB5_1:
 	cvta.to.global.u64 	%rd6, %rd4;
 	mad.lo.s32 	%r10, %r1, %r3, %r2;
 	cvta.to.global.u64 	%rd7, %rd3;
@@ -273,25 +353,25 @@ BB4_1:
 	ld.global.f64 	%fd1, [%rd9];
 	add.s64 	%rd2, %rd6, %rd8;
 	setp.eq.s32	%p4, %r4, 1;
-	@%p4 bra 	BB4_3;
-	bra.uni 	BB4_2;
+	@%p4 bra 	BB5_3;
+	bra.uni 	BB5_2;
 
-BB4_3:
+BB5_3:
 	mul.wide.s32 	%rd12, %r2, 8;
 	add.s64 	%rd13, %rd1, %rd12;
 	ld.global.f64 	%fd5, [%rd13];
 	fma.rn.f64 	%fd6, %fd5, %fd2, %fd1;
 	st.global.f64 	[%rd2], %fd6;
-	bra.uni 	BB4_4;
+	bra.uni 	BB5_4;
 
-BB4_2:
+BB5_2:
 	mul.wide.s32 	%rd10, %r1, 8;
 	add.s64 	%rd11, %rd1, %rd10;
 	ld.global.f64 	%fd3, [%rd11];
 	fma.rn.f64 	%fd4, %fd3, %fd2, %fd1;
 	st.global.f64 	[%rd2], %fd4;
 
-BB4_4:
+BB5_4:
 	ret;
 }
 
@@ -325,10 +405,10 @@ BB4_4:
 	setp.lt.s32	%p1, %r2, %r5;
 	setp.gt.s32	%p2, %r3, -1;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB5_2;
-	bra.uni 	BB5_1;
+	@!%p3 bra 	BB6_2;
+	bra.uni 	BB6_1;
 
-BB5_1:
+BB6_1:
 	rem.s32 	%r9, %r1, %r3;
 	cvta.to.global.u64 	%rd4, %rd1;
 	mad.lo.s32 	%r10, %r2, %r3, %r9;
@@ -345,7 +425,7 @@ BB5_1:
 	add.s64 	%rd11, %rd10, %rd5;
 	st.global.f64 	[%rd11], %fd3;
 
-BB5_2:
+BB6_2:
 	ret;
 }
 
@@ -387,10 +467,10 @@ BB5_2:
 	setp.lt.s32	%p1, %r8, %r2;
 	setp.gt.s32	%p2, %r3, -1;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB6_6;
-	bra.uni 	BB6_1;
+	@!%p3 bra 	BB7_6;
+	bra.uni 	BB7_1;
 
-BB6_1:
+BB7_1:
 	cvta.to.global.u64 	%rd4, %rd2;
 	mul.wide.s32 	%rd5, %r1, 8;
 	add.s64 	%rd6, %rd4, %rd5;
@@ -400,26 +480,26 @@ BB6_1:
 	setp.lt.f64	%p4, %fd8, %fd3;
 	cvta.to.global.u64 	%rd7, %rd3;
 	add.s64 	%rd1, %rd7, %rd5;
-	@%p4 bra 	BB6_5;
-	bra.uni 	BB6_2;
+	@%p4 bra 	BB7_5;
+	bra.uni 	BB7_2;
 
-BB6_5:
+BB7_5:
 	st.global.f64 	[%rd1], %fd4;
-	bra.uni 	BB6_6;
+	bra.uni 	BB7_6;
 
-BB6_2:
+BB7_2:
 	setp.lt.f64	%p5, %fd1, %fd2;
-	@%p5 bra 	BB6_4;
-	bra.uni 	BB6_3;
+	@%p5 bra 	BB7_4;
+	bra.uni 	BB7_3;
 
-BB6_4:
+BB7_4:
 	st.global.f64 	[%rd1], %fd5;
-	bra.uni 	BB6_6;
+	bra.uni 	BB7_6;
 
-BB6_3:
+BB7_3:
 	st.global.f64 	[%rd1], %fd6;
 
-BB6_6:
+BB7_6:
 	ret;
 }
 
@@ -435,9 +515,9 @@ BB6_6:
 	.param .u32 matrix_matrix_cellwise_op_param_7
 )
 {
-	.reg .pred 	%p<73>;
-	.reg .b32 	%r<66>;
-	.reg .f64 	%fd<56>;
+	.reg .pred 	%p<77>;
+	.reg .b32 	%r<65>;
+	.reg .f64 	%fd<55>;
 	.reg .b64 	%rd<19>;
 
 
@@ -458,93 +538,93 @@ BB6_6:
 	setp.lt.s32	%p2, %r1, %r14;
 	setp.gt.s32	%p3, %r10, -1;
 	and.pred  	%p4, %p2, %p3;
-	@!%p4 bra 	BB7_77;
-	bra.uni 	BB7_1;
+	@!%p4 bra 	BB8_73;
+	bra.uni 	BB8_1;
 
-BB7_1:
+BB8_1:
 	mad.lo.s32 	%r3, %r1, %r10, %r2;
 	setp.eq.s32	%p5, %r11, 1;
-	mov.u32 	%r64, %r1;
-	@%p5 bra 	BB7_5;
+	mov.u32 	%r63, %r1;
+	@%p5 bra 	BB8_5;
 
 	setp.ne.s32	%p6, %r11, 2;
-	mov.u32 	%r65, %r3;
-	@%p6 bra 	BB7_4;
+	mov.u32 	%r64, %r3;
+	@%p6 bra 	BB8_4;
 
-	mov.u32 	%r65, %r2;
+	mov.u32 	%r64, %r2;
 
-BB7_4:
-	mov.u32 	%r59, %r65;
-	mov.u32 	%r4, %r59;
-	mov.u32 	%r64, %r4;
+BB8_4:
+	mov.u32 	%r58, %r64;
+	mov.u32 	%r4, %r58;
+	mov.u32 	%r63, %r4;
 
-BB7_5:
-	mov.u32 	%r5, %r64;
+BB8_5:
+	mov.u32 	%r5, %r63;
 	setp.eq.s32	%p7, %r12, 1;
-	mov.u32 	%r62, %r1;
-	@%p7 bra 	BB7_9;
+	mov.u32 	%r61, %r1;
+	@%p7 bra 	BB8_9;
 
 	setp.ne.s32	%p8, %r12, 2;
-	mov.u32 	%r63, %r3;
-	@%p8 bra 	BB7_8;
+	mov.u32 	%r62, %r3;
+	@%p8 bra 	BB8_8;
 
-	mov.u32 	%r63, %r2;
+	mov.u32 	%r62, %r2;
 
-BB7_8:
-	mov.u32 	%r62, %r63;
+BB8_8:
+	mov.u32 	%r61, %r62;
 
-BB7_9:
+BB8_9:
 	cvta.to.global.u64 	%rd5, %rd3;
 	cvta.to.global.u64 	%rd6, %rd2;
 	mul.wide.s32 	%rd7, %r5, 8;
 	add.s64 	%rd8, %rd6, %rd7;
 	ld.global.f64 	%fd1, [%rd8];
-	mul.wide.s32 	%rd9, %r62, 8;
+	mul.wide.s32 	%rd9, %r61, 8;
 	add.s64 	%rd10, %rd5, %rd9;
 	ld.global.f64 	%fd2, [%rd10];
-	mov.f64 	%fd55, 0d7FEFFFFFFFFFFFFF;
+	mov.f64 	%fd54, 0d7FEFFFFFFFFFFFFF;
 	setp.gt.s32	%p9, %r13, 8;
-	@%p9 bra 	BB7_26;
+	@%p9 bra 	BB8_26;
 
 	setp.gt.s32	%p23, %r13, 3;
-	@%p23 bra 	BB7_18;
+	@%p23 bra 	BB8_18;
 
 	setp.gt.s32	%p30, %r13, 1;
-	@%p30 bra 	BB7_15;
+	@%p30 bra 	BB8_15;
 
 	setp.eq.s32	%p33, %r13, 0;
-	@%p33 bra 	BB7_75;
-	bra.uni 	BB7_13;
+	@%p33 bra 	BB8_71;
+	bra.uni 	BB8_13;
 
-BB7_75:
-	add.f64 	%fd55, %fd1, %fd2;
-	bra.uni 	BB7_76;
+BB8_71:
+	add.f64 	%fd54, %fd1, %fd2;
+	bra.uni 	BB8_72;
 
-BB7_26:
+BB8_26:
 	setp.gt.s32	%p10, %r13, 13;
-	@%p10 bra 	BB7_35;
+	@%p10 bra 	BB8_35;
 
 	setp.gt.s32	%p17, %r13, 10;
-	@%p17 bra 	BB7_31;
+	@%p17 bra 	BB8_31;
 
 	setp.eq.s32	%p21, %r13, 9;
-	@%p21 bra 	BB7_55;
-	bra.uni 	BB7_29;
+	@%p21 bra 	BB8_53;
+	bra.uni 	BB8_29;
 
-BB7_55:
-	setp.eq.f64	%p48, %fd1, %fd2;
-	selp.f64	%fd55, 0d3FF0000000000000, 0d0000000000000000, %p48;
-	bra.uni 	BB7_76;
+BB8_53:
+	setp.eq.f64	%p50, %fd1, %fd2;
+	selp.f64	%fd54, 0d3FF0000000000000, 0d0000000000000000, %p50;
+	bra.uni 	BB8_72;
 
-BB7_18:
+BB8_18:
 	setp.gt.s32	%p24, %r13, 5;
-	@%p24 bra 	BB7_22;
+	@%p24 bra 	BB8_22;
 
 	setp.eq.s32	%p28, %r13, 4;
-	@%p28 bra 	BB7_58;
-	bra.uni 	BB7_20;
+	@%p28 bra 	BB8_56;
+	bra.uni 	BB8_20;
 
-BB7_58:
+BB8_56:
 	{
 	.reg .b32 %temp; 
 	mov.b64 	{%temp, %r8}, %fd1;
@@ -557,7 +637,7 @@ BB7_58:
 	add.s32 	%r32, %r31, -1012;
 	mov.b64 	 %rd15, %fd2;
 	shl.b64 	%rd1, %rd15, %r32;
-	setp.eq.s64	%p53, %rd1, -9223372036854775808;
+	setp.eq.s64	%p55, %rd1, -9223372036854775808;
 	abs.f64 	%fd19, %fd1;
 	// Callseq Start 0
 	{
@@ -574,342 +654,340 @@ BB7_58:
 	param0, 
 	param1
 	);
-	ld.param.f64	%fd54, [retval0+0];
+	ld.param.f64	%fd53, [retval0+0];
 	
 	//{
 	}// Callseq End 0
-	setp.lt.s32	%p54, %r8, 0;
-	and.pred  	%p1, %p54, %p53;
-	@!%p1 bra 	BB7_60;
-	bra.uni 	BB7_59;
+	setp.lt.s32	%p56, %r8, 0;
+	and.pred  	%p1, %p56, %p55;
+	@!%p1 bra 	BB8_58;
+	bra.uni 	BB8_57;
 
-BB7_59:
+BB8_57:
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r33}, %fd54;
+	mov.b64 	{%temp, %r33}, %fd53;
 	}
 	xor.b32  	%r34, %r33, -2147483648;
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r35, %temp}, %fd54;
+	mov.b64 	{%r35, %temp}, %fd53;
 	}
-	mov.b64 	%fd54, {%r35, %r34};
+	mov.b64 	%fd53, {%r35, %r34};
 
-BB7_60:
-	mov.f64 	%fd53, %fd54;
-	setp.eq.f64	%p55, %fd1, 0d0000000000000000;
-	@%p55 bra 	BB7_63;
-	bra.uni 	BB7_61;
+BB8_58:
+	mov.f64 	%fd52, %fd53;
+	setp.eq.f64	%p57, %fd1, 0d0000000000000000;
+	@%p57 bra 	BB8_61;
+	bra.uni 	BB8_59;
 
-BB7_63:
-	selp.b32	%r36, %r8, 0, %p53;
+BB8_61:
+	selp.b32	%r36, %r8, 0, %p55;
 	or.b32  	%r37, %r36, 2146435072;
-	setp.lt.s32	%p59, %r9, 0;
-	selp.b32	%r38, %r37, %r36, %p59;
+	setp.lt.s32	%p61, %r9, 0;
+	selp.b32	%r38, %r37, %r36, %p61;
 	mov.u32 	%r39, 0;
-	mov.b64 	%fd53, {%r39, %r38};
-	bra.uni 	BB7_64;
+	mov.b64 	%fd52, {%r39, %r38};
+	bra.uni 	BB8_62;
 
-BB7_35:
+BB8_35:
 	setp.gt.s32	%p11, %r13, 15;
-	@%p11 bra 	BB7_39;
+	@%p11 bra 	BB8_39;
 
 	setp.eq.s32	%p15, %r13, 14;
-	@%p15 bra 	BB7_52;
-	bra.uni 	BB7_37;
+	@%p15 bra 	BB8_50;
+	bra.uni 	BB8_37;
 
-BB7_52:
+BB8_50:
 	cvt.rni.s64.f64	%rd11, %fd1;
 	cvt.rni.s64.f64	%rd12, %fd2;
 	cvt.u32.u64	%r25, %rd11;
 	cvt.u32.u64	%r26, %rd12;
 	or.b32  	%r27, %r26, %r25;
-	setp.eq.s32	%p45, %r27, 0;
-	selp.f64	%fd55, 0d0000000000000000, 0d3FF0000000000000, %p45;
-	bra.uni 	BB7_76;
+	setp.eq.s32	%p47, %r27, 0;
+	selp.f64	%fd54, 0d0000000000000000, 0d3FF0000000000000, %p47;
+	bra.uni 	BB8_72;
 
-BB7_15:
+BB8_15:
 	setp.eq.s32	%p31, %r13, 2;
-	@%p31 bra 	BB7_74;
-	bra.uni 	BB7_16;
+	@%p31 bra 	BB8_70;
+	bra.uni 	BB8_16;
 
-BB7_74:
-	mul.f64 	%fd55, %fd1, %fd2;
-	bra.uni 	BB7_76;
+BB8_70:
+	mul.f64 	%fd54, %fd1, %fd2;
+	bra.uni 	BB8_72;
 
-BB7_31:
+BB8_31:
 	setp.eq.s32	%p18, %r13, 11;
-	@%p18 bra 	BB7_54;
+	@%p18 bra 	BB8_52;
 
 	setp.eq.s32	%p19, %r13, 12;
-	@%p19 bra 	BB7_53;
-	bra.uni 	BB7_33;
+	@%p19 bra 	BB8_51;
+	bra.uni 	BB8_33;
 
-BB7_53:
-	max.f64 	%fd55, %fd1, %fd2;
-	bra.uni 	BB7_76;
+BB8_51:
+	max.f64 	%fd54, %fd1, %fd2;
+	bra.uni 	BB8_72;
 
-BB7_22:
+BB8_22:
 	setp.eq.s32	%p25, %r13, 6;
-	@%p25 bra 	BB7_57;
+	@%p25 bra 	BB8_55;
 
 	setp.eq.s32	%p26, %r13, 7;
-	@%p26 bra 	BB7_56;
-	bra.uni 	BB7_24;
+	@%p26 bra 	BB8_54;
+	bra.uni 	BB8_24;
 
-BB7_56:
-	setp.gt.f64	%p50, %fd1, %fd2;
-	selp.f64	%fd55, 0d3FF0000000000000, 0d0000000000000000, %p50;
-	bra.uni 	BB7_76;
+BB8_54:
+	setp.gt.f64	%p52, %fd1, %fd2;
+	selp.f64	%fd54, 0d3FF0000000000000, 0d0000000000000000, %p52;
+	bra.uni 	BB8_72;
 
-BB7_39:
+BB8_39:
 	setp.eq.s32	%p12, %r13, 16;
-	@%p12 bra 	BB7_51;
+	@%p12 bra 	BB8_49;
 
 	setp.eq.s32	%p13, %r13, 17;
-	@%p13 bra 	BB7_46;
-	bra.uni 	BB7_41;
+	@%p13 bra 	BB8_45;
+	bra.uni 	BB8_41;
 
-BB7_46:
-	setp.eq.f64	%p38, %fd2, 0d0000000000000000;
-	setp.eq.f64	%p39, %fd2, 0d8000000000000000;
-	or.pred  	%p40, %p38, %p39;
-	mov.f64 	%fd55, 0d7FF8000000000000;
-	@%p40 bra 	BB7_76;
+BB8_45:
+	setp.eq.f64	%p39, %fd2, 0d0000000000000000;
+	setp.eq.f64	%p40, %fd2, 0d8000000000000000;
+	or.pred  	%p41, %p39, %p40;
+	mov.f64 	%fd54, 0d7FF8000000000000;
+	@%p41 bra 	BB8_72;
 
-	div.rn.f64 	%fd55, %fd1, %fd2;
-	abs.f64 	%fd39, %fd55;
-	setp.gtu.f64	%p41, %fd39, 0d7FF0000000000000;
-	@%p41 bra 	BB7_76;
+	div.rn.f64 	%fd54, %fd1, %fd2;
+	abs.f64 	%fd39, %fd54;
+	setp.gtu.f64	%p42, %fd39, 0d7FF0000000000000;
+	@%p42 bra 	BB8_72;
 
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r22}, %fd55;
+	mov.b64 	{%r22, %temp}, %fd54;
 	}
-	and.b32  	%r23, %r22, 2147483647;
-	setp.ne.s32	%p42, %r23, 2146435072;
-	@%p42 bra 	BB7_50;
-
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r24, %temp}, %fd55;
+	mov.b64 	{%temp, %r23}, %fd54;
 	}
-	setp.eq.s32	%p43, %r24, 0;
-	@%p43 bra 	BB7_76;
+	and.b32  	%r24, %r23, 2147483647;
+	setp.ne.s32	%p43, %r24, 2146435072;
+	setp.ne.s32	%p44, %r22, 0;
+	or.pred  	%p45, %p43, %p44;
+	@!%p45 bra 	BB8_72;
+	bra.uni 	BB8_48;
 
-BB7_50:
-	cvt.rmi.f64.f64	%fd40, %fd55;
+BB8_48:
+	cvt.rmi.f64.f64	%fd40, %fd54;
 	mul.f64 	%fd41, %fd2, %fd40;
-	sub.f64 	%fd55, %fd1, %fd41;
-	bra.uni 	BB7_76;
+	sub.f64 	%fd54, %fd1, %fd41;
+	bra.uni 	BB8_72;
 
-BB7_13:
+BB8_13:
 	setp.eq.s32	%p34, %r13, 1;
-	@%p34 bra 	BB7_14;
-	bra.uni 	BB7_76;
+	@%p34 bra 	BB8_14;
+	bra.uni 	BB8_72;
 
-BB7_14:
-	sub.f64 	%fd55, %fd1, %fd2;
-	bra.uni 	BB7_76;
+BB8_14:
+	sub.f64 	%fd54, %fd1, %fd2;
+	bra.uni 	BB8_72;
 
-BB7_29:
+BB8_29:
 	setp.eq.s32	%p22, %r13, 10;
-	@%p22 bra 	BB7_30;
-	bra.uni 	BB7_76;
+	@%p22 bra 	BB8_30;
+	bra.uni 	BB8_72;
 
-BB7_30:
-	setp.neu.f64	%p47, %fd1, %fd2;
-	selp.f64	%fd55, 0d3FF0000000000000, 0d0000000000000000, %p47;
-	bra.uni 	BB7_76;
+BB8_30:
+	setp.neu.f64	%p49, %fd1, %fd2;
+	selp.f64	%fd54, 0d3FF0000000000000, 0d0000000000000000, %p49;
+	bra.uni 	BB8_72;
 
-BB7_20:
+BB8_20:
 	setp.eq.s32	%p29, %r13, 5;
-	@%p29 bra 	BB7_21;
-	bra.uni 	BB7_76;
+	@%p29 bra 	BB8_21;
+	bra.uni 	BB8_72;
 
-BB7_21:
-	setp.lt.f64	%p52, %fd1, %fd2;
-	selp.f64	%fd55, 0d3FF0000000000000, 0d0000000000000000, %p52;
-	bra.uni 	BB7_76;
+BB8_21:
+	setp.lt.f64	%p54, %fd1, %fd2;
+	selp.f64	%fd54, 0d3FF0000000000000, 0d0000000000000000, %p54;
+	bra.uni 	BB8_72;
 
-BB7_37:
+BB8_37:
 	setp.eq.s32	%p16, %r13, 15;
-	@%p16 bra 	BB7_38;
-	bra.uni 	BB7_76;
+	@%p16 bra 	BB8_38;
+	bra.uni 	BB8_72;
 
-BB7_38:
+BB8_38:
 	mul.f64 	%fd43, %fd1, %fd2;
 	mov.f64 	%fd44, 0d3FF0000000000000;
-	sub.f64 	%fd55, %fd44, %fd43;
-	bra.uni 	BB7_76;
+	sub.f64 	%fd54, %fd44, %fd43;
+	bra.uni 	BB8_72;
 
-BB7_16:
+BB8_16:
 	setp.eq.s32	%p32, %r13, 3;
-	@%p32 bra 	BB7_17;
-	bra.uni 	BB7_76;
+	@%p32 bra 	BB8_17;
+	bra.uni 	BB8_72;
 
-BB7_17:
-	div.rn.f64 	%fd55, %fd1, %fd2;
-	bra.uni 	BB7_76;
+BB8_17:
+	div.rn.f64 	%fd54, %fd1, %fd2;
+	bra.uni 	BB8_72;
 
-BB7_54:
-	min.f64 	%fd55, %fd1, %fd2;
-	bra.uni 	BB7_76;
+BB8_52:
+	min.f64 	%fd54, %fd1, %fd2;
+	bra.uni 	BB8_72;
 
-BB7_33:
+BB8_33:
 	setp.eq.s32	%p20, %r13, 13;
-	@%p20 bra 	BB7_34;
-	bra.uni 	BB7_76;
+	@%p20 bra 	BB8_34;
+	bra.uni 	BB8_72;
 
-BB7_34:
+BB8_34:
 	cvt.rni.s64.f64	%rd13, %fd1;
 	cvt.rni.s64.f64	%rd14, %fd2;
 	cvt.u32.u64	%r28, %rd13;
 	cvt.u32.u64	%r29, %rd14;
 	and.b32  	%r30, %r29, %r28;
-	setp.eq.s32	%p46, %r30, 0;
-	selp.f64	%fd55, 0d0000000000000000, 0d3FF0000000000000, %p46;
-	bra.uni 	BB7_76;
+	setp.eq.s32	%p48, %r30, 0;
+	selp.f64	%fd54, 0d0000000000000000, 0d3FF0000000000000, %p48;
+	bra.uni 	BB8_72;
 
-BB7_57:
-	setp.le.f64	%p51, %fd1, %fd2;
-	selp.f64	%fd55, 0d3FF0000000000000, 0d0000000000000000, %p51;
-	bra.uni 	BB7_76;
+BB8_55:
+	setp.le.f64	%p53, %fd1, %fd2;
+	selp.f64	%fd54, 0d3FF0000000000000, 0d0000000000000000, %p53;
+	bra.uni 	BB8_72;
 
-BB7_24:
+BB8_24:
 	setp.eq.s32	%p27, %r13, 8;
-	@%p27 bra 	BB7_25;
-	bra.uni 	BB7_76;
+	@%p27 bra 	BB8_25;
+	bra.uni 	BB8_72;
 
-BB7_25:
-	setp.ge.f64	%p49, %fd1, %fd2;
-	selp.f64	%fd55, 0d3FF0000000000000, 0d0000000000000000, %p49;
-	bra.uni 	BB7_76;
+BB8_25:
+	setp.ge.f64	%p51, %fd1, %fd2;
+	selp.f64	%fd54, 0d3FF0000000000000, 0d0000000000000000, %p51;
+	bra.uni 	BB8_72;
 
-BB7_51:
-	setp.neu.f64	%p44, %fd1, 0d0000000000000000;
+BB8_49:
+	setp.neu.f64	%p46, %fd1, 0d0000000000000000;
 	sub.f64 	%fd42, %fd1, %fd2;
-	selp.f64	%fd55, %fd42, 0d0000000000000000, %p44;
-	bra.uni 	BB7_76;
+	selp.f64	%fd54, %fd42, 0d0000000000000000, %p46;
+	bra.uni 	BB8_72;
 
-BB7_41:
+BB8_41:
 	setp.ne.s32	%p14, %r13, 18;
-	@%p14 bra 	BB7_76;
+	@%p14 bra 	BB8_72;
 
-	div.rn.f64 	%fd55, %fd1, %fd2;
-	abs.f64 	%fd37, %fd55;
+	div.rn.f64 	%fd54, %fd1, %fd2;
+	abs.f64 	%fd37, %fd54;
 	setp.gtu.f64	%p35, %fd37, 0d7FF0000000000000;
-	@%p35 bra 	BB7_76;
+	@%p35 bra 	BB8_72;
 
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r19}, %fd55;
+	mov.b64 	{%r19, %temp}, %fd54;
 	}
-	and.b32  	%r20, %r19, 2147483647;
-	setp.ne.s32	%p36, %r20, 2146435072;
-	@%p36 bra 	BB7_45;
-
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r21, %temp}, %fd55;
+	mov.b64 	{%temp, %r20}, %fd54;
 	}
-	setp.eq.s32	%p37, %r21, 0;
-	@%p37 bra 	BB7_76;
+	and.b32  	%r21, %r20, 2147483647;
+	setp.ne.s32	%p36, %r21, 2146435072;
+	setp.ne.s32	%p37, %r19, 0;
+	or.pred  	%p38, %p36, %p37;
+	@!%p38 bra 	BB8_72;
+	bra.uni 	BB8_44;
 
-BB7_45:
-	cvt.rmi.f64.f64	%fd55, %fd55;
-	bra.uni 	BB7_76;
+BB8_44:
+	cvt.rmi.f64.f64	%fd54, %fd54;
+	bra.uni 	BB8_72;
 
-BB7_61:
-	setp.gt.s32	%p56, %r8, -1;
-	@%p56 bra 	BB7_64;
+BB8_59:
+	setp.gt.s32	%p58, %r8, -1;
+	@%p58 bra 	BB8_62;
 
 	cvt.rzi.f64.f64	%fd45, %fd2;
-	setp.neu.f64	%p57, %fd45, %fd2;
-	selp.f64	%fd53, 0dFFF8000000000000, %fd53, %p57;
+	setp.neu.f64	%p59, %fd45, %fd2;
+	selp.f64	%fd52, 0dFFF8000000000000, %fd52, %p59;
 
-BB7_64:
-	mov.f64 	%fd25, %fd53;
+BB8_62:
+	mov.f64 	%fd25, %fd52;
 	add.f64 	%fd26, %fd1, %fd2;
 	{
 	.reg .b32 %temp; 
 	mov.b64 	{%temp, %r40}, %fd26;
 	}
 	and.b32  	%r41, %r40, 2146435072;
-	setp.ne.s32	%p60, %r41, 2146435072;
-	mov.f64 	%fd52, %fd25;
-	@%p60 bra 	BB7_73;
-
-	setp.gtu.f64	%p61, %fd19, 0d7FF0000000000000;
-	mov.f64 	%fd52, %fd26;
-	@%p61 bra 	BB7_73;
+	setp.ne.s32	%p62, %r41, 2146435072;
+	mov.f64 	%fd51, %fd25;
+	@%p62 bra 	BB8_69;
 
-	abs.f64 	%fd46, %fd2;
-	setp.gtu.f64	%p62, %fd46, 0d7FF0000000000000;
+	setp.gtu.f64	%p63, %fd19, 0d7FF0000000000000;
 	mov.f64 	%fd51, %fd26;
-	mov.f64 	%fd52, %fd51;
-	@%p62 bra 	BB7_73;
+	@%p63 bra 	BB8_69;
 
-	and.b32  	%r42, %r9, 2147483647;
-	setp.ne.s32	%p63, %r42, 2146435072;
-	@%p63 bra 	BB7_69;
+	abs.f64 	%fd46, %fd2;
+	setp.gtu.f64	%p64, %fd46, 0d7FF0000000000000;
+	mov.f64 	%fd50, %fd26;
+	mov.f64 	%fd51, %fd50;
+	@%p64 bra 	BB8_69;
 
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r43, %temp}, %fd2;
+	mov.b64 	{%r42, %temp}, %fd2;
 	}
-	setp.eq.s32	%p64, %r43, 0;
-	@%p64 bra 	BB7_72;
+	and.b32  	%r43, %r9, 2147483647;
+	setp.eq.s32	%p65, %r43, 2146435072;
+	setp.eq.s32	%p66, %r42, 0;
+	and.pred  	%p67, %p65, %p66;
+	@%p67 bra 	BB8_68;
+	bra.uni 	BB8_66;
 
-BB7_69:
-	and.b32  	%r44, %r8, 2147483647;
-	setp.ne.s32	%p65, %r44, 2146435072;
-	mov.f64 	%fd49, %fd25;
-	mov.f64 	%fd52, %fd49;
-	@%p65 bra 	BB7_73;
+BB8_68:
+	setp.gt.f64	%p71, %fd19, 0d3FF0000000000000;
+	selp.b32	%r51, 2146435072, 0, %p71;
+	xor.b32  	%r52, %r51, 2146435072;
+	setp.lt.s32	%p72, %r9, 0;
+	selp.b32	%r53, %r52, %r51, %p72;
+	setp.eq.f64	%p73, %fd1, 0dBFF0000000000000;
+	selp.b32	%r54, 1072693248, %r53, %p73;
+	mov.u32 	%r55, 0;
+	mov.b64 	%fd51, {%r55, %r54};
+	bra.uni 	BB8_69;
 
+BB8_66:
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r45, %temp}, %fd1;
+	mov.b64 	{%r44, %temp}, %fd1;
 	}
-	setp.ne.s32	%p66, %r45, 0;
-	mov.f64 	%fd52, %fd25;
-	@%p66 bra 	BB7_73;
+	and.b32  	%r45, %r8, 2147483647;
+	setp.eq.s32	%p68, %r45, 2146435072;
+	setp.eq.s32	%p69, %r44, 0;
+	and.pred  	%p70, %p68, %p69;
+	mov.f64 	%fd51, %fd25;
+	@!%p70 bra 	BB8_69;
+	bra.uni 	BB8_67;
 
+BB8_67:
 	shr.s32 	%r46, %r9, 31;
 	and.b32  	%r47, %r46, -2146435072;
-	add.s32 	%r48, %r47, 2146435072;
-	or.b32  	%r49, %r48, -2147483648;
-	selp.b32	%r50, %r49, %r48, %p1;
-	mov.u32 	%r51, 0;
-	mov.b64 	%fd52, {%r51, %r50};
-	bra.uni 	BB7_73;
-
-BB7_72:
-	setp.gt.f64	%p67, %fd19, 0d3FF0000000000000;
-	selp.b32	%r52, 2146435072, 0, %p67;
-	xor.b32  	%r53, %r52, 2146435072;
-	setp.lt.s32	%p68, %r9, 0;
-	selp.b32	%r54, %r53, %r52, %p68;
-	setp.eq.f64	%p69, %fd1, 0dBFF0000000000000;
-	selp.b32	%r55, 1072693248, %r54, %p69;
-	mov.u32 	%r56, 0;
-	mov.b64 	%fd52, {%r56, %r55};
-
-BB7_73:
-	setp.eq.f64	%p70, %fd2, 0d0000000000000000;
-	setp.eq.f64	%p71, %fd1, 0d3FF0000000000000;
-	or.pred  	%p72, %p71, %p70;
-	selp.f64	%fd55, 0d3FF0000000000000, %fd52, %p72;
+	selp.b32	%r48, -1048576, 2146435072, %p1;
+	add.s32 	%r49, %r48, %r47;
+	mov.u32 	%r50, 0;
+	mov.b64 	%fd51, {%r50, %r49};
 
-BB7_76:
+BB8_69:
+	setp.eq.f64	%p74, %fd2, 0d0000000000000000;
+	setp.eq.f64	%p75, %fd1, 0d3FF0000000000000;
+	or.pred  	%p76, %p75, %p74;
+	selp.f64	%fd54, 0d3FF0000000000000, %fd51, %p76;
+
+BB8_72:
 	cvta.to.global.u64 	%rd16, %rd4;
 	mul.wide.s32 	%rd17, %r3, 8;
 	add.s64 	%rd18, %rd16, %rd17;
-	st.global.f64 	[%rd18], %fd55;
+	st.global.f64 	[%rd18], %fd54;
 	bar.sync 	0;
 
-BB7_77:
+BB8_73:
 	ret;
 }
 
@@ -923,9 +1001,9 @@ BB7_77:
 	.param .u32 matrix_scalar_op_param_5
 )
 {
-	.reg .pred 	%p<133>;
-	.reg .b32 	%r<88>;
-	.reg .f64 	%fd<109>;
+	.reg .pred 	%p<141>;
+	.reg .b32 	%r<86>;
+	.reg .f64 	%fd<107>;
 	.reg .b64 	%rd<20>;
 
 
@@ -940,7 +1018,7 @@ BB7_77:
 	mov.u32 	%r11, %tid.x;
 	mad.lo.s32 	%r1, %r9, %r10, %r11;
 	setp.ge.s32	%p3, %r1, %r8;
-	@%p3 bra 	BB8_138;
+	@%p3 bra 	BB9_130;
 
 	cvta.to.global.u64 	%rd6, %rd5;
 	cvta.to.global.u64 	%rd7, %rd4;
@@ -949,86 +1027,86 @@ BB7_77:
 	ld.global.f64 	%fd1, [%rd9];
 	add.s64 	%rd1, %rd6, %rd8;
 	setp.eq.s32	%p4, %r7, 0;
-	@%p4 bra 	BB8_70;
+	@%p4 bra 	BB9_66;
 
-	mov.f64 	%fd99, 0d7FEFFFFFFFFFFFFF;
+	mov.f64 	%fd98, 0d7FEFFFFFFFFFFFFF;
 	setp.gt.s32	%p5, %r6, 8;
-	@%p5 bra 	BB8_19;
+	@%p5 bra 	BB9_19;
 
 	setp.gt.s32	%p19, %r6, 3;
-	@%p19 bra 	BB8_11;
+	@%p19 bra 	BB9_11;
 
 	setp.gt.s32	%p26, %r6, 1;
-	@%p26 bra 	BB8_8;
+	@%p26 bra 	BB9_8;
 
 	setp.eq.s32	%p29, %r6, 0;
-	@%p29 bra 	BB8_68;
-	bra.uni 	BB8_6;
+	@%p29 bra 	BB9_64;
+	bra.uni 	BB9_6;
 
-BB8_68:
-	add.f64 	%fd99, %fd1, %fd68;
-	bra.uni 	BB8_69;
+BB9_64:
+	add.f64 	%fd98, %fd1, %fd68;
+	bra.uni 	BB9_65;
 
-BB8_70:
-	mov.f64 	%fd108, 0d7FEFFFFFFFFFFFFF;
-	setp.gt.s32	%p69, %r6, 8;
-	@%p69 bra 	BB8_87;
+BB9_66:
+	mov.f64 	%fd106, 0d7FEFFFFFFFFFFFFF;
+	setp.gt.s32	%p73, %r6, 8;
+	@%p73 bra 	BB9_83;
 
-	setp.gt.s32	%p83, %r6, 3;
-	@%p83 bra 	BB8_79;
+	setp.gt.s32	%p87, %r6, 3;
+	@%p87 bra 	BB9_75;
 
-	setp.gt.s32	%p90, %r6, 1;
-	@%p90 bra 	BB8_76;
+	setp.gt.s32	%p94, %r6, 1;
+	@%p94 bra 	BB9_72;
 
-	setp.eq.s32	%p93, %r6, 0;
-	@%p93 bra 	BB8_136;
-	bra.uni 	BB8_74;
+	setp.eq.s32	%p97, %r6, 0;
+	@%p97 bra 	BB9_128;
+	bra.uni 	BB9_70;
 
-BB8_136:
-	add.f64 	%fd108, %fd1, %fd68;
-	bra.uni 	BB8_137;
+BB9_128:
+	add.f64 	%fd106, %fd1, %fd68;
+	bra.uni 	BB9_129;
 
-BB8_19:
+BB9_19:
 	setp.gt.s32	%p6, %r6, 13;
-	@%p6 bra 	BB8_28;
+	@%p6 bra 	BB9_28;
 
 	setp.gt.s32	%p13, %r6, 10;
-	@%p13 bra 	BB8_24;
+	@%p13 bra 	BB9_24;
 
 	setp.eq.s32	%p17, %r6, 9;
-	@%p17 bra 	BB8_48;
-	bra.uni 	BB8_22;
+	@%p17 bra 	BB9_46;
+	bra.uni 	BB9_22;
 
-BB8_48:
-	setp.eq.f64	%p44, %fd1, %fd68;
-	selp.f64	%fd99, 0d3FF0000000000000, 0d0000000000000000, %p44;
-	bra.uni 	BB8_69;
+BB9_46:
+	setp.eq.f64	%p46, %fd1, %fd68;
+	selp.f64	%fd98, 0d3FF0000000000000, 0d0000000000000000, %p46;
+	bra.uni 	BB9_65;
 
-BB8_87:
-	setp.gt.s32	%p70, %r6, 13;
-	@%p70 bra 	BB8_96;
+BB9_83:
+	setp.gt.s32	%p74, %r6, 13;
+	@%p74 bra 	BB9_92;
 
-	setp.gt.s32	%p77, %r6, 10;
-	@%p77 bra 	BB8_92;
+	setp.gt.s32	%p81, %r6, 10;
+	@%p81 bra 	BB9_88;
 
-	setp.eq.s32	%p81, %r6, 9;
-	@%p81 bra 	BB8_116;
-	bra.uni 	BB8_90;
+	setp.eq.s32	%p85, %r6, 9;
+	@%p85 bra 	BB9_110;
+	bra.uni 	BB9_86;
 
-BB8_116:
-	setp.eq.f64	%p108, %fd1, %fd68;
-	selp.f64	%fd108, 0d3FF0000000000000, 0d0000000000000000, %p108;
-	bra.uni 	BB8_137;
+BB9_110:
+	setp.eq.f64	%p114, %fd1, %fd68;
+	selp.f64	%fd106, 0d3FF0000000000000, 0d0000000000000000, %p114;
+	bra.uni 	BB9_129;
 
-BB8_11:
+BB9_11:
 	setp.gt.s32	%p20, %r6, 5;
-	@%p20 bra 	BB8_15;
+	@%p20 bra 	BB9_15;
 
 	setp.eq.s32	%p24, %r6, 4;
-	@%p24 bra 	BB8_51;
-	bra.uni 	BB8_13;
+	@%p24 bra 	BB9_49;
+	bra.uni 	BB9_13;
 
-BB8_51:
+BB9_49:
 	{
 	.reg .b32 %temp; 
 	mov.b64 	{%temp, %r2}, %fd68;
@@ -1041,7 +1119,7 @@ BB8_51:
 	add.s32 	%r25, %r24, -1012;
 	mov.b64 	 %rd14, %fd1;
 	shl.b64 	%rd2, %rd14, %r25;
-	setp.eq.s64	%p49, %rd2, -9223372036854775808;
+	setp.eq.s64	%p51, %rd2, -9223372036854775808;
 	abs.f64 	%fd18, %fd68;
 	// Callseq Start 1
 	{
@@ -1058,69 +1136,69 @@ BB8_51:
 	param0, 
 	param1
 	);
-	ld.param.f64	%fd98, [retval0+0];
+	ld.param.f64	%fd97, [retval0+0];
 	
 	//{
 	}// Callseq End 1
-	setp.lt.s32	%p50, %r2, 0;
-	and.pred  	%p1, %p50, %p49;
-	@!%p1 bra 	BB8_53;
-	bra.uni 	BB8_52;
+	setp.lt.s32	%p52, %r2, 0;
+	and.pred  	%p1, %p52, %p51;
+	@!%p1 bra 	BB9_51;
+	bra.uni 	BB9_50;
 
-BB8_52:
+BB9_50:
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r26}, %fd98;
+	mov.b64 	{%temp, %r26}, %fd97;
 	}
 	xor.b32  	%r27, %r26, -2147483648;
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r28, %temp}, %fd98;
+	mov.b64 	{%r28, %temp}, %fd97;
 	}
-	mov.b64 	%fd98, {%r28, %r27};
+	mov.b64 	%fd97, {%r28, %r27};
 
-BB8_53:
-	mov.f64 	%fd97, %fd98;
-	setp.eq.f64	%p51, %fd68, 0d0000000000000000;
-	@%p51 bra 	BB8_56;
-	bra.uni 	BB8_54;
+BB9_51:
+	mov.f64 	%fd96, %fd97;
+	setp.eq.f64	%p53, %fd68, 0d0000000000000000;
+	@%p53 bra 	BB9_54;
+	bra.uni 	BB9_52;
 
-BB8_56:
-	selp.b32	%r29, %r2, 0, %p49;
+BB9_54:
+	selp.b32	%r29, %r2, 0, %p51;
 	or.b32  	%r30, %r29, 2146435072;
-	setp.lt.s32	%p55, %r3, 0;
-	selp.b32	%r31, %r30, %r29, %p55;
+	setp.lt.s32	%p57, %r3, 0;
+	selp.b32	%r31, %r30, %r29, %p57;
 	mov.u32 	%r32, 0;
-	mov.b64 	%fd97, {%r32, %r31};
-	bra.uni 	BB8_57;
+	mov.b64 	%fd96, {%r32, %r31};
+	bra.uni 	BB9_55;
 
-BB8_28:
+BB9_28:
 	setp.gt.s32	%p7, %r6, 15;
-	@%p7 bra 	BB8_32;
+	@%p7 bra 	BB9_32;
 
 	setp.eq.s32	%p11, %r6, 14;
-	@%p11 bra 	BB8_45;
-	bra.uni 	BB8_30;
+	@%p11 bra 	BB9_43;
+	bra.uni 	BB9_30;
 
-BB8_45:
+BB9_43:
 	cvt.rni.s64.f64	%rd10, %fd68;
 	cvt.rni.s64.f64	%rd11, %fd1;
 	cvt.u32.u64	%r18, %rd10;
 	cvt.u32.u64	%r19, %rd11;
 	or.b32  	%r20, %r19, %r18;
-	setp.eq.s32	%p41, %r20, 0;
-	selp.f64	%fd99, 0d0000000000000000, 0d3FF0000000000000, %p41;
-	bra.uni 	BB8_69;
+	setp.eq.s32	%p43, %r20, 0;
+	selp.f64	%fd98, 0d0000000000000000, 0d3FF0000000000000, %p43;
+	bra.uni 	BB9_65;
 
-BB8_79:
-	setp.gt.s32	%p84, %r6, 5;
-	@%p84 bra 	BB8_83;
+BB9_75:
+	setp.gt.s32	%p88, %r6, 5;
+	@%p88 bra 	BB9_79;
 
-	setp.eq.s32	%p88, %r6, 4;
-	@%p88 bra 	BB8_119;
-	bra.uni 	BB8_81;
+	setp.eq.s32	%p92, %r6, 4;
+	@%p92 bra 	BB9_113;
+	bra.uni 	BB9_77;
 
-BB8_119:
+BB9_113:
 	{
 	.reg .b32 %temp; 
 	mov.b64 	{%temp, %r4}, %fd1;
@@ -1129,11 +1207,11 @@ BB8_119:
 	.reg .b32 %temp; 
 	mov.b64 	{%temp, %r5}, %fd68;
 	}
-	bfe.u32 	%r62, %r5, 20, 11;
-	add.s32 	%r63, %r62, -1012;
+	bfe.u32 	%r61, %r5, 20, 11;
+	add.s32 	%r62, %r61, -1012;
 	mov.b64 	 %rd19, %fd68;
-	shl.b64 	%rd3, %rd19, %r63;
-	setp.eq.s64	%p113, %rd3, -9223372036854775808;
+	shl.b64 	%rd3, %rd19, %r62;
+	setp.eq.s64	%p119, %rd3, -9223372036854775808;
 	abs.f64 	%fd51, %fd1;
 	// Callseq Start 2
 	{
@@ -1150,616 +1228,612 @@ BB8_119:
 	param0, 
 	param1
 	);
-	ld.param.f64	%fd107, [retval0+0];
+	ld.param.f64	%fd105, [retval0+0];
 	
 	//{
 	}// Callseq End 2
-	setp.lt.s32	%p114, %r4, 0;
-	and.pred  	%p2, %p114, %p113;
-	@!%p2 bra 	BB8_121;
-	bra.uni 	BB8_120;
+	setp.lt.s32	%p120, %r4, 0;
+	and.pred  	%p2, %p120, %p119;
+	@!%p2 bra 	BB9_115;
+	bra.uni 	BB9_114;
 
-BB8_120:
+BB9_114:
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r64}, %fd107;
+	mov.b64 	{%temp, %r63}, %fd105;
 	}
-	xor.b32  	%r65, %r64, -2147483648;
+	xor.b32  	%r64, %r63, -2147483648;
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r66, %temp}, %fd107;
+	mov.b64 	{%r65, %temp}, %fd105;
 	}
-	mov.b64 	%fd107, {%r66, %r65};
-
-BB8_121:
-	mov.f64 	%fd106, %fd107;
-	setp.eq.f64	%p115, %fd1, 0d0000000000000000;
-	@%p115 bra 	BB8_124;
-	bra.uni 	BB8_122;
-
-BB8_124:
-	selp.b32	%r67, %r4, 0, %p113;
-	or.b32  	%r68, %r67, 2146435072;
-	setp.lt.s32	%p119, %r5, 0;
-	selp.b32	%r69, %r68, %r67, %p119;
-	mov.u32 	%r70, 0;
-	mov.b64 	%fd106, {%r70, %r69};
-	bra.uni 	BB8_125;
-
-BB8_96:
-	setp.gt.s32	%p71, %r6, 15;
-	@%p71 bra 	BB8_100;
-
-	setp.eq.s32	%p75, %r6, 14;
-	@%p75 bra 	BB8_113;
-	bra.uni 	BB8_98;
-
-BB8_113:
+	mov.b64 	%fd105, {%r65, %r64};
+
+BB9_115:
+	mov.f64 	%fd104, %fd105;
+	setp.eq.f64	%p121, %fd1, 0d0000000000000000;
+	@%p121 bra 	BB9_118;
+	bra.uni 	BB9_116;
+
+BB9_118:
+	selp.b32	%r66, %r4, 0, %p119;
+	or.b32  	%r67, %r66, 2146435072;
+	setp.lt.s32	%p125, %r5, 0;
+	selp.b32	%r68, %r67, %r66, %p125;
+	mov.u32 	%r69, 0;
+	mov.b64 	%fd104, {%r69, %r68};
+	bra.uni 	BB9_119;
+
+BB9_92:
+	setp.gt.s32	%p75, %r6, 15;
+	@%p75 bra 	BB9_96;
+
+	setp.eq.s32	%p79, %r6, 14;
+	@%p79 bra 	BB9_107;
+	bra.uni 	BB9_94;
+
+BB9_107:
 	cvt.rni.s64.f64	%rd15, %fd1;
 	cvt.rni.s64.f64	%rd16, %fd68;
-	cvt.u32.u64	%r56, %rd15;
-	cvt.u32.u64	%r57, %rd16;
-	or.b32  	%r58, %r57, %r56;
-	setp.eq.s32	%p105, %r58, 0;
-	selp.f64	%fd108, 0d0000000000000000, 0d3FF0000000000000, %p105;
-	bra.uni 	BB8_137;
-
-BB8_8:
+	cvt.u32.u64	%r55, %rd15;
+	cvt.u32.u64	%r56, %rd16;
+	or.b32  	%r57, %r56, %r55;
+	setp.eq.s32	%p111, %r57, 0;
+	selp.f64	%fd106, 0d0000000000000000, 0d3FF0000000000000, %p111;
+	bra.uni 	BB9_129;
+
+BB9_8:
 	setp.eq.s32	%p27, %r6, 2;
-	@%p27 bra 	BB8_67;
-	bra.uni 	BB8_9;
+	@%p27 bra 	BB9_63;
+	bra.uni 	BB9_9;
 
-BB8_67:
-	mul.f64 	%fd99, %fd1, %fd68;
-	bra.uni 	BB8_69;
+BB9_63:
+	mul.f64 	%fd98, %fd1, %fd68;
+	bra.uni 	BB9_65;
 
-BB8_24:
+BB9_24:
 	setp.eq.s32	%p14, %r6, 11;
-	@%p14 bra 	BB8_47;
+	@%p14 bra 	BB9_45;
 
 	setp.eq.s32	%p15, %r6, 12;
-	@%p15 bra 	BB8_46;
-	bra.uni 	BB8_26;
+	@%p15 bra 	BB9_44;
+	bra.uni 	BB9_26;
 
-BB8_46:
-	max.f64 	%fd99, %fd68, %fd1;
-	bra.uni 	BB8_69;
+BB9_44:
+	max.f64 	%fd98, %fd68, %fd1;
+	bra.uni 	BB9_65;
 
-BB8_15:
+BB9_15:
 	setp.eq.s32	%p21, %r6, 6;
-	@%p21 bra 	BB8_50;
+	@%p21 bra 	BB9_48;
 
 	setp.eq.s32	%p22, %r6, 7;
-	@%p22 bra 	BB8_49;
-	bra.uni 	BB8_17;
+	@%p22 bra 	BB9_47;
+	bra.uni 	BB9_17;
 
-BB8_49:
-	setp.lt.f64	%p46, %fd1, %fd68;
-	selp.f64	%fd99, 0d3FF0000000000000, 0d0000000000000000, %p46;
-	bra.uni 	BB8_69;
+BB9_47:
+	setp.lt.f64	%p48, %fd1, %fd68;
+	selp.f64	%fd98, 0d3FF0000000000000, 0d0000000000000000, %p48;
+	bra.uni 	BB9_65;
 
-BB8_32:
+BB9_32:
 	setp.eq.s32	%p8, %r6, 16;
-	@%p8 bra 	BB8_44;
+	@%p8 bra 	BB9_42;
 
 	setp.eq.s32	%p9, %r6, 17;
-	@%p9 bra 	BB8_39;
-	bra.uni 	BB8_34;
+	@%p9 bra 	BB9_38;
+	bra.uni 	BB9_34;
 
-BB8_39:
-	setp.eq.f64	%p34, %fd1, 0d0000000000000000;
-	setp.eq.f64	%p35, %fd1, 0d8000000000000000;
-	or.pred  	%p36, %p34, %p35;
-	mov.f64 	%fd99, 0d7FF8000000000000;
-	@%p36 bra 	BB8_69;
+BB9_38:
+	setp.eq.f64	%p35, %fd1, 0d0000000000000000;
+	setp.eq.f64	%p36, %fd1, 0d8000000000000000;
+	or.pred  	%p37, %p35, %p36;
+	mov.f64 	%fd98, 0d7FF8000000000000;
+	@%p37 bra 	BB9_65;
 
-	div.rn.f64 	%fd99, %fd68, %fd1;
-	abs.f64 	%fd72, %fd99;
-	setp.gtu.f64	%p37, %fd72, 0d7FF0000000000000;
-	@%p37 bra 	BB8_69;
+	div.rn.f64 	%fd98, %fd68, %fd1;
+	abs.f64 	%fd72, %fd98;
+	setp.gtu.f64	%p38, %fd72, 0d7FF0000000000000;
+	@%p38 bra 	BB9_65;
 
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r15}, %fd99;
+	mov.b64 	{%r15, %temp}, %fd98;
 	}
-	and.b32  	%r16, %r15, 2147483647;
-	setp.ne.s32	%p38, %r16, 2146435072;
-	@%p38 bra 	BB8_43;
-
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r17, %temp}, %fd99;
+	mov.b64 	{%temp, %r16}, %fd98;
 	}
-	setp.eq.s32	%p39, %r17, 0;
-	@%p39 bra 	BB8_69;
-
-BB8_43:
-	cvt.rmi.f64.f64	%fd73, %fd99;
+	and.b32  	%r17, %r16, 2147483647;
+	setp.ne.s32	%p39, %r17, 2146435072;
+	setp.ne.s32	%p40, %r15, 0;
+	or.pred  	%p41, %p39, %p40;
+	@!%p41 bra 	BB9_65;
+	bra.uni 	BB9_41;
+
+BB9_41:
+	cvt.rmi.f64.f64	%fd73, %fd98;
 	mul.f64 	%fd74, %fd1, %fd73;
-	sub.f64 	%fd99, %fd68, %fd74;
-	bra.uni 	BB8_69;
-
-BB8_76:
-	setp.eq.s32	%p91, %r6, 2;
-	@%p91 bra 	BB8_135;
-	bra.uni 	BB8_77;
-
-BB8_135:
-	mul.f64 	%fd108, %fd1, %fd68;
-	bra.uni 	BB8_137;
-
-BB8_92:
-	setp.eq.s32	%p78, %r6, 11;
-	@%p78 bra 	BB8_115;
-
-	setp.eq.s32	%p79, %r6, 12;
-	@%p79 bra 	BB8_114;
-	bra.uni 	BB8_94;
-
-BB8_114:
-	max.f64 	%fd108, %fd1, %fd68;
-	bra.uni 	BB8_137;
-
-BB8_83:
-	setp.eq.s32	%p85, %r6, 6;
-	@%p85 bra 	BB8_118;
-
-	setp.eq.s32	%p86, %r6, 7;
-	@%p86 bra 	BB8_117;
-	bra.uni 	BB8_85;
-
-BB8_117:
-	setp.gt.f64	%p110, %fd1, %fd68;
-	selp.f64	%fd108, 0d3FF0000000000000, 0d0000000000000000, %p110;
-	bra.uni 	BB8_137;
-
-BB8_100:
-	setp.eq.s32	%p72, %r6, 16;
-	@%p72 bra 	BB8_112;
-
-	setp.eq.s32	%p73, %r6, 17;
-	@%p73 bra 	BB8_107;
-	bra.uni 	BB8_102;
-
-BB8_107:
-	setp.eq.f64	%p98, %fd68, 0d0000000000000000;
-	setp.eq.f64	%p99, %fd68, 0d8000000000000000;
-	or.pred  	%p100, %p98, %p99;
-	mov.f64 	%fd108, 0d7FF8000000000000;
-	@%p100 bra 	BB8_137;
-
-	div.rn.f64 	%fd108, %fd1, %fd68;
-	abs.f64 	%fd83, %fd108;
-	setp.gtu.f64	%p101, %fd83, 0d7FF0000000000000;
-	@%p101 bra 	BB8_137;
+	sub.f64 	%fd98, %fd68, %fd74;
+	bra.uni 	BB9_65;
+
+BB9_72:
+	setp.eq.s32	%p95, %r6, 2;
+	@%p95 bra 	BB9_127;
+	bra.uni 	BB9_73;
+
+BB9_127:
+	mul.f64 	%fd106, %fd1, %fd68;
+	bra.uni 	BB9_129;
+
+BB9_88:
+	setp.eq.s32	%p82, %r6, 11;
+	@%p82 bra 	BB9_109;
+
+	setp.eq.s32	%p83, %r6, 12;
+	@%p83 bra 	BB9_108;
+	bra.uni 	BB9_90;
+
+BB9_108:
+	max.f64 	%fd106, %fd1, %fd68;
+	bra.uni 	BB9_129;
+
+BB9_79:
+	setp.eq.s32	%p89, %r6, 6;
+	@%p89 bra 	BB9_112;
+
+	setp.eq.s32	%p90, %r6, 7;
+	@%p90 bra 	BB9_111;
+	bra.uni 	BB9_81;
+
+BB9_111:
+	setp.gt.f64	%p116, %fd1, %fd68;
+	selp.f64	%fd106, 0d3FF0000000000000, 0d0000000000000000, %p116;
+	bra.uni 	BB9_129;
+
+BB9_96:
+	setp.eq.s32	%p76, %r6, 16;
+	@%p76 bra 	BB9_106;
+
+	setp.eq.s32	%p77, %r6, 17;
+	@%p77 bra 	BB9_102;
+	bra.uni 	BB9_98;
+
+BB9_102:
+	setp.eq.f64	%p103, %fd68, 0d0000000000000000;
+	setp.eq.f64	%p104, %fd68, 0d8000000000000000;
+	or.pred  	%p105, %p103, %p104;
+	mov.f64 	%fd106, 0d7FF8000000000000;
+	@%p105 bra 	BB9_129;
+
+	div.rn.f64 	%fd106, %fd1, %fd68;
+	abs.f64 	%fd83, %fd106;
+	setp.gtu.f64	%p106, %fd83, 0d7FF0000000000000;
+	@%p106 bra 	BB9_129;
 
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r53}, %fd108;
+	mov.b64 	{%r52, %temp}, %fd106;
 	}
-	and.b32  	%r54, %r53, 2147483647;
-	setp.ne.s32	%p102, %r54, 2146435072;
-	@%p102 bra 	BB8_111;
-
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r55, %temp}, %fd108;
+	mov.b64 	{%temp, %r53}, %fd106;
 	}
-	setp.eq.s32	%p103, %r55, 0;
-	@%p103 bra 	BB8_137;
-
-BB8_111:
-	cvt.rmi.f64.f64	%fd84, %fd108;
+	and.b32  	%r54, %r53, 2147483647;
+	setp.ne.s32	%p107, %r54, 2146435072;
+	setp.ne.s32	%p108, %r52, 0;
+	or.pred  	%p109, %p107, %p108;
+	@!%p109 bra 	BB9_129;
+	bra.uni 	BB9_105;
+
+BB9_105:
+	cvt.rmi.f64.f64	%fd84, %fd106;
 	mul.f64 	%fd85, %fd84, %fd68;
-	sub.f64 	%fd108, %fd1, %fd85;
-	bra.uni 	BB8_137;
+	sub.f64 	%fd106, %fd1, %fd85;
+	bra.uni 	BB9_129;
 
-BB8_6:
+BB9_6:
 	setp.eq.s32	%p30, %r6, 1;
-	@%p30 bra 	BB8_7;
-	bra.uni 	BB8_69;
+	@%p30 bra 	BB9_7;
+	bra.uni 	BB9_65;
 
-BB8_7:
-	sub.f64 	%fd99, %fd68, %fd1;
-	bra.uni 	BB8_69;
+BB9_7:
+	sub.f64 	%fd98, %fd68, %fd1;
+	bra.uni 	BB9_65;
 
-BB8_22:
+BB9_22:
 	setp.eq.s32	%p18, %r6, 10;
-	@%p18 bra 	BB8_23;
-	bra.uni 	BB8_69;
+	@%p18 bra 	BB9_23;
+	bra.uni 	BB9_65;
 
-BB8_23:
-	setp.neu.f64	%p43, %fd1, %fd68;
-	selp.f64	%fd99, 0d3FF0000000000000, 0d0000000000000000, %p43;
-	bra.uni 	BB8_69;
+BB9_23:
+	setp.neu.f64	%p45, %fd1, %fd68;
+	selp.f64	%fd98, 0d3FF0000000000000, 0d0000000000000000, %p45;
+	bra.uni 	BB9_65;
 
-BB8_13:
+BB9_13:
 	setp.eq.s32	%p25, %r6, 5;
-	@%p25 bra 	BB8_14;
-	bra.uni 	BB8_69;
+	@%p25 bra 	BB9_14;
+	bra.uni 	BB9_65;
 
-BB8_14:
-	setp.gt.f64	%p48, %fd1, %fd68;
-	selp.f64	%fd99, 0d3FF0000000000000, 0d0000000000000000, %p48;
-	bra.uni 	BB8_69;
+BB9_14:
+	setp.gt.f64	%p50, %fd1, %fd68;
+	selp.f64	%fd98, 0d3FF0000000000000, 0d0000000000000000, %p50;
+	bra.uni 	BB9_65;
 
-BB8_30:
+BB9_30:
 	setp.eq.s32	%p12, %r6, 15;
-	@%p12 bra 	BB8_31;
-	bra.uni 	BB8_69;
+	@%p12 bra 	BB9_31;
+	bra.uni 	BB9_65;
 
-BB8_31:
+BB9_31:
 	mul.f64 	%fd76, %fd1, %fd68;
 	mov.f64 	%fd77, 0d3FF0000000000000;
-	sub.f64 	%fd99, %fd77, %fd76;
-	bra.uni 	BB8_69;
+	sub.f64 	%fd98, %fd77, %fd76;
+	bra.uni 	BB9_65;
 
-BB8_9:
+BB9_9:
 	setp.eq.s32	%p28, %r6, 3;
-	@%p28 bra 	BB8_10;
-	bra.uni 	BB8_69;
+	@%p28 bra 	BB9_10;
+	bra.uni 	BB9_65;
 
-BB8_10:
-	div.rn.f64 	%fd99, %fd68, %fd1;
-	bra.uni 	BB8_69;
+BB9_10:
+	div.rn.f64 	%fd98, %fd68, %fd1;
+	bra.uni 	BB9_65;
 
-BB8_47:
-	min.f64 	%fd99, %fd68, %fd1;
-	bra.uni 	BB8_69;
+BB9_45:
+	min.f64 	%fd98, %fd68, %fd1;
+	bra.uni 	BB9_65;
 
-BB8_26:
+BB9_26:
 	setp.eq.s32	%p16, %r6, 13;
-	@%p16 bra 	BB8_27;
-	bra.uni 	BB8_69;
+	@%p16 bra 	BB9_27;
+	bra.uni 	BB9_65;
 
-BB8_27:
+BB9_27:
 	cvt.rni.s64.f64	%rd12, %fd68;
 	cvt.rni.s64.f64	%rd13, %fd1;
 	cvt.u32.u64	%r21, %rd12;
 	cvt.u32.u64	%r22, %rd13;
 	and.b32  	%r23, %r22, %r21;
-	setp.eq.s32	%p42, %r23, 0;
-	selp.f64	%fd99, 0d0000000000000000, 0d3FF0000000000000, %p42;
-	bra.uni 	BB8_69;
+	setp.eq.s32	%p44, %r23, 0;
+	selp.f64	%fd98, 0d0000000000000000, 0d3FF0000000000000, %p44;
+	bra.uni 	BB9_65;
 
-BB8_50:
-	setp.ge.f64	%p47, %fd1, %fd68;
-	selp.f64	%fd99, 0d3FF0000000000000, 0d0000000000000000, %p47;
-	bra.uni 	BB8_69;
+BB9_48:
+	setp.ge.f64	%p49, %fd1, %fd68;
+	selp.f64	%fd98, 0d3FF0000000000000, 0d0000000000000000, %p49;
+	bra.uni 	BB9_65;
 
-BB8_17:
+BB9_17:
 	setp.eq.s32	%p23, %r6, 8;
-	@%p23 bra 	BB8_18;
-	bra.uni 	BB8_69;
+	@%p23 bra 	BB9_18;
+	bra.uni 	BB9_65;
 
-BB8_18:
-	setp.le.f64	%p45, %fd1, %fd68;
-	selp.f64	%fd99, 0d3FF0000000000000, 0d0000000000000000, %p45;
-	bra.uni 	BB8_69;
+BB9_18:
+	setp.le.f64	%p47, %fd1, %fd68;
+	selp.f64	%fd98, 0d3FF0000000000000, 0d0000000000000000, %p47;
+	bra.uni 	BB9_65;
 
-BB8_44:
-	setp.neu.f64	%p40, %fd68, 0d0000000000000000;
+BB9_42:
+	setp.neu.f64	%p42, %fd68, 0d0000000000000000;
 	sub.f64 	%fd75, %fd68, %fd1;
-	selp.f64	%fd99, %fd75, 0d0000000000000000, %p40;
-	bra.uni 	BB8_69;
+	selp.f64	%fd98, %fd75, 0d0000000000000000, %p42;
+	bra.uni 	BB9_65;
 
-BB8_34:
+BB9_34:
 	setp.ne.s32	%p10, %r6, 18;
-	@%p10 bra 	BB8_69;
+	@%p10 bra 	BB9_65;
 
-	div.rn.f64 	%fd99, %fd68, %fd1;
-	abs.f64 	%fd70, %fd99;
+	div.rn.f64 	%fd98, %fd68, %fd1;
+	abs.f64 	%fd70, %fd98;
 	setp.gtu.f64	%p31, %fd70, 0d7FF0000000000000;
-	@%p31 bra 	BB8_69;
+	@%p31 bra 	BB9_65;
 
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r12}, %fd99;
+	mov.b64 	{%r12, %temp}, %fd98;
 	}
-	and.b32  	%r13, %r12, 2147483647;
-	setp.ne.s32	%p32, %r13, 2146435072;
-	@%p32 bra 	BB8_38;
-
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r14, %temp}, %fd99;
+	mov.b64 	{%temp, %r13}, %fd98;
 	}
-	setp.eq.s32	%p33, %r14, 0;
-	@%p33 bra 	BB8_69;
-
-BB8_38:
-	cvt.rmi.f64.f64	%fd99, %fd99;
-	bra.uni 	BB8_69;
-
-BB8_74:
-	setp.eq.s32	%p94, %r6, 1;
-	@%p94 bra 	BB8_75;
-	bra.uni 	BB8_137;
-
-BB8_75:
-	sub.f64 	%fd108, %fd1, %fd68;
-	bra.uni 	BB8_137;
-
-BB8_90:
-	setp.eq.s32	%p82, %r6, 10;
-	@%p82 bra 	BB8_91;
-	bra.uni 	BB8_137;
-
-BB8_91:
-	setp.neu.f64	%p107, %fd1, %fd68;
-	selp.f64	%fd108, 0d3FF0000000000000, 0d0000000000000000, %p107;
-	bra.uni 	BB8_137;
-
-BB8_81:
-	setp.eq.s32	%p89, %r6, 5;
-	@%p89 bra 	BB8_82;
-	bra.uni 	BB8_137;
-
-BB8_82:
-	setp.lt.f64	%p112, %fd1, %fd68;
-	selp.f64	%fd108, 0d3FF0000000000000, 0d0000000000000000, %p112;
-	bra.uni 	BB8_137;
-
-BB8_98:
-	setp.eq.s32	%p76, %r6, 15;
-	@%p76 bra 	BB8_99;
-	bra.uni 	BB8_137;
-
-BB8_99:
+	and.b32  	%r14, %r13, 2147483647;
+	setp.ne.s32	%p32, %r14, 2146435072;
+	setp.ne.s32	%p33, %r12, 0;
+	or.pred  	%p34, %p32, %p33;
+	@!%p34 bra 	BB9_65;
+	bra.uni 	BB9_37;
+
+BB9_37:
+	cvt.rmi.f64.f64	%fd98, %fd98;
+	bra.uni 	BB9_65;
+
+BB9_70:
+	setp.eq.s32	%p98, %r6, 1;
+	@%p98 bra 	BB9_71;
+	bra.uni 	BB9_129;
+
+BB9_71:
+	sub.f64 	%fd106, %fd1, %fd68;
+	bra.uni 	BB9_129;
+
+BB9_86:
+	setp.eq.s32	%p86, %r6, 10;
+	@%p86 bra 	BB9_87;
+	bra.uni 	BB9_129;
+
+BB9_87:
+	setp.neu.f64	%p113, %fd1, %fd68;
+	selp.f64	%fd106, 0d3FF0000000000000, 0d0000000000000000, %p113;
+	bra.uni 	BB9_129;
+
+BB9_77:
+	setp.eq.s32	%p93, %r6, 5;
+	@%p93 bra 	BB9_78;
+	bra.uni 	BB9_129;
+
+BB9_78:
+	setp.lt.f64	%p118, %fd1, %fd68;
+	selp.f64	%fd106, 0d3FF0000000000000, 0d0000000000000000, %p118;
+	bra.uni 	BB9_129;
+
+BB9_94:
+	setp.eq.s32	%p80, %r6, 15;
+	@%p80 bra 	BB9_95;
+	bra.uni 	BB9_129;
+
+BB9_95:
 	mul.f64 	%fd87, %fd1, %fd68;
 	mov.f64 	%fd88, 0d3FF0000000000000;
-	sub.f64 	%fd108, %fd88, %fd87;
-	bra.uni 	BB8_137;
+	sub.f64 	%fd106, %fd88, %fd87;
+	bra.uni 	BB9_129;
 
-BB8_77:
-	setp.eq.s32	%p92, %r6, 3;
-	@%p92 bra 	BB8_78;
-	bra.uni 	BB8_137;
+BB9_73:
+	setp.eq.s32	%p96, %r6, 3;
+	@%p96 bra 	BB9_74;
+	bra.uni 	BB9_129;
 
-BB8_78:
-	div.rn.f64 	%fd108, %fd1, %fd68;
-	bra.uni 	BB8_137;
+BB9_74:
+	div.rn.f64 	%fd106, %fd1, %fd68;
+	bra.uni 	BB9_129;
 
-BB8_115:
-	min.f64 	%fd108, %fd1, %fd68;
-	bra.uni 	BB8_137;
+BB9_109:
+	min.f64 	%fd106, %fd1, %fd68;
+	bra.uni 	BB9_129;
 
-BB8_94:
-	setp.eq.s32	%p80, %r6, 13;
-	@%p80 bra 	BB8_95;
-	bra.uni 	BB8_137;
+BB9_90:
+	setp.eq.s32	%p84, %r6, 13;
+	@%p84 bra 	BB9_91;
+	bra.uni 	BB9_129;
 
-BB8_95:
+BB9_91:
 	cvt.rni.s64.f64	%rd17, %fd1;
 	cvt.rni.s64.f64	%rd18, %fd68;
-	cvt.u32.u64	%r59, %rd17;
-	cvt.u32.u64	%r60, %rd18;
-	and.b32  	%r61, %r60, %r59;
-	setp.eq.s32	%p106, %r61, 0;
-	selp.f64	%fd108, 0d0000000000000000, 0d3FF0000000000000, %p106;
-	bra.uni 	BB8_137;
-
-BB8_118:
-	setp.le.f64	%p111, %fd1, %fd68;
-	selp.f64	%fd108, 0d3FF0000000000000, 0d0000000000000000, %p111;
-	bra.uni 	BB8_137;
-
-BB8_85:
-	setp.eq.s32	%p87, %r6, 8;
-	@%p87 bra 	BB8_86;
-	bra.uni 	BB8_137;
-
-BB8_86:
-	setp.ge.f64	%p109, %fd1, %fd68;
-	selp.f64	%fd108, 0d3FF0000000000000, 0d0000000000000000, %p109;
-	bra.uni 	BB8_137;
-
-BB8_112:
-	setp.neu.f64	%p104, %fd1, 0d0000000000000000;
+	cvt.u32.u64	%r58, %rd17;
+	cvt.u32.u64	%r59, %rd18;
+	and.b32  	%r60, %r59, %r58;
+	setp.eq.s32	%p112, %r60, 0;
+	selp.f64	%fd106, 0d0000000000000000, 0d3FF0000000000000, %p112;
+	bra.uni 	BB9_129;
+
+BB9_112:
+	setp.le.f64	%p117, %fd1, %fd68;
+	selp.f64	%fd106, 0d3FF0000000000000, 0d0000000000000000, %p117;
+	bra.uni 	BB9_129;
+
+BB9_81:
+	setp.eq.s32	%p91, %r6, 8;
+	@%p91 bra 	BB9_82;
+	bra.uni 	BB9_129;
+
+BB9_82:
+	setp.ge.f64	%p115, %fd1, %fd68;
+	selp.f64	%fd106, 0d3FF0000000000000, 0d0000000000000000, %p115;
+	bra.uni 	BB9_129;
+
+BB9_106:
+	setp.neu.f64	%p110, %fd1, 0d0000000000000000;
 	sub.f64 	%fd86, %fd1, %fd68;
-	selp.f64	%fd108, %fd86, 0d0000000000000000, %p104;
-	bra.uni 	BB8_137;
+	selp.f64	%fd106, %fd86, 0d0000000000000000, %p110;
+	bra.uni 	BB9_129;
 
-BB8_102:
-	setp.ne.s32	%p74, %r6, 18;
-	@%p74 bra 	BB8_137;
+BB9_98:
+	setp.ne.s32	%p78, %r6, 18;
+	@%p78 bra 	BB9_129;
 
-	div.rn.f64 	%fd108, %fd1, %fd68;
-	abs.f64 	%fd81, %fd108;
-	setp.gtu.f64	%p95, %fd81, 0d7FF0000000000000;
-	@%p95 bra 	BB8_137;
+	div.rn.f64 	%fd106, %fd1, %fd68;
+	abs.f64 	%fd81, %fd106;
+	setp.gtu.f64	%p99, %fd81, 0d7FF0000000000000;
+	@%p99 bra 	BB9_129;
 
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r50}, %fd108;
+	mov.b64 	{%r49, %temp}, %fd106;
 	}
-	and.b32  	%r51, %r50, 2147483647;
-	setp.ne.s32	%p96, %r51, 2146435072;
-	@%p96 bra 	BB8_106;
-
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r52, %temp}, %fd108;
+	mov.b64 	{%temp, %r50}, %fd106;
 	}
-	setp.eq.s32	%p97, %r52, 0;
-	@%p97 bra 	BB8_137;
+	and.b32  	%r51, %r50, 2147483647;
+	setp.ne.s32	%p100, %r51, 2146435072;
+	setp.ne.s32	%p101, %r49, 0;
+	or.pred  	%p102, %p100, %p101;
+	@!%p102 bra 	BB9_129;
+	bra.uni 	BB9_101;
 
-BB8_106:
-	cvt.rmi.f64.f64	%fd108, %fd108;
-	bra.uni 	BB8_137;
+BB9_101:
+	cvt.rmi.f64.f64	%fd106, %fd106;
+	bra.uni 	BB9_129;
 
-BB8_54:
-	setp.gt.s32	%p52, %r2, -1;
-	@%p52 bra 	BB8_57;
+BB9_52:
+	setp.gt.s32	%p54, %r2, -1;
+	@%p54 bra 	BB9_55;
 
 	cvt.rzi.f64.f64	%fd78, %fd1;
-	setp.neu.f64	%p53, %fd78, %fd1;
-	selp.f64	%fd97, 0dFFF8000000000000, %fd97, %p53;
+	setp.neu.f64	%p55, %fd78, %fd1;
+	selp.f64	%fd96, 0dFFF8000000000000, %fd96, %p55;
 
-BB8_57:
-	mov.f64 	%fd24, %fd97;
+BB9_55:
+	mov.f64 	%fd24, %fd96;
 	add.f64 	%fd25, %fd1, %fd68;
 	{
 	.reg .b32 %temp; 
 	mov.b64 	{%temp, %r33}, %fd25;
 	}
 	and.b32  	%r34, %r33, 2146435072;
-	setp.ne.s32	%p56, %r34, 2146435072;
-	mov.f64 	%fd96, %fd24;
-	@%p56 bra 	BB8_66;
-
-	setp.gtu.f64	%p57, %fd18, 0d7FF0000000000000;
-	mov.f64 	%fd96, %fd25;
-	@%p57 bra 	BB8_66;
+	setp.ne.s32	%p58, %r34, 2146435072;
+	mov.f64 	%fd95, %fd24;
+	@%p58 bra 	BB9_62;
 
-	abs.f64 	%fd79, %fd1;
-	setp.gtu.f64	%p58, %fd79, 0d7FF0000000000000;
+	setp.gtu.f64	%p59, %fd18, 0d7FF0000000000000;
 	mov.f64 	%fd95, %fd25;
-	mov.f64 	%fd96, %fd95;
-	@%p58 bra 	BB8_66;
-
-	and.b32  	%r35, %r3, 2147483647;
-	setp.ne.s32	%p59, %r35, 2146435072;
-	@%p59 bra 	BB8_62;
+	@%p59 bra 	BB9_62;
 
-	{
-	.reg .b32 %temp; 
-	mov.b64 	{%r36, %temp}, %fd1;
-	}
-	setp.eq.s32	%p60, %r36, 0;
-	@%p60 bra 	BB8_65;
-
-BB8_62:
-	and.b32  	%r37, %r2, 2147483647;
-	setp.ne.s32	%p61, %r37, 2146435072;
-	mov.f64 	%fd93, %fd24;
-	mov.f64 	%fd96, %fd93;
-	@%p61 bra 	BB8_66;
+	abs.f64 	%fd79, %fd1;
+	setp.gtu.f64	%p60, %fd79, 0d7FF0000000000000;
+	mov.f64 	%fd94, %fd25;
+	mov.f64 	%fd95, %fd94;
+	@%p60 bra 	BB9_62;
 
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r38, %temp}, %fd68;
+	mov.b64 	{%r35, %temp}, %fd1;
 	}
-	setp.ne.s32	%p62, %r38, 0;
-	mov.f64 	%fd96, %fd24;
-	@%p62 bra 	BB8_66;
-
-	shr.s32 	%r39, %r3, 31;
-	and.b32  	%r40, %r39, -2146435072;
-	add.s32 	%r41, %r40, 2146435072;
-	or.b32  	%r42, %r41, -2147483648;
-	selp.b32	%r43, %r42, %r41, %p1;
-	mov.u32 	%r44, 0;
-	mov.b64 	%fd96, {%r44, %r43};
-	bra.uni 	BB8_66;
-
-BB8_122:
-	setp.gt.s32	%p116, %r4, -1;
-	@%p116 bra 	BB8_125;
+	and.b32  	%r36, %r3, 2147483647;
+	setp.eq.s32	%p61, %r36, 2146435072;
+	setp.eq.s32	%p62, %r35, 0;
+	and.pred  	%p63, %p61, %p62;
+	@%p63 bra 	BB9_61;
+	bra.uni 	BB9_59;
+
+BB9_61:
+	setp.gt.f64	%p67, %fd18, 0d3FF0000000000000;
+	selp.b32	%r44, 2146435072, 0, %p67;
+	xor.b32  	%r45, %r44, 2146435072;
+	setp.lt.s32	%p68, %r3, 0;
+	selp.b32	%r46, %r45, %r44, %p68;
+	setp.eq.f64	%p69, %fd68, 0dBFF0000000000000;
+	selp.b32	%r47, 1072693248, %r46, %p69;
+	mov.u32 	%r48, 0;
+	mov.b64 	%fd95, {%r48, %r47};
+	bra.uni 	BB9_62;
+
+BB9_116:
+	setp.gt.s32	%p122, %r4, -1;
+	@%p122 bra 	BB9_119;
 
 	cvt.rzi.f64.f64	%fd89, %fd68;
-	setp.neu.f64	%p117, %fd89, %fd68;
-	selp.f64	%fd106, 0dFFF8000000000000, %fd106, %p117;
+	setp.neu.f64	%p123, %fd89, %fd68;
+	selp.f64	%fd104, 0dFFF8000000000000, %fd104, %p123;
 
-BB8_125:
-	mov.f64 	%fd57, %fd106;
+BB9_119:
+	mov.f64 	%fd57, %fd104;
 	add.f64 	%fd58, %fd1, %fd68;
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%temp, %r71}, %fd58;
+	mov.b64 	{%temp, %r70}, %fd58;
 	}
-	and.b32  	%r72, %r71, 2146435072;
-	setp.ne.s32	%p120, %r72, 2146435072;
-	mov.f64 	%fd105, %fd57;
-	@%p120 bra 	BB8_134;
+	and.b32  	%r71, %r70, 2146435072;
+	setp.ne.s32	%p126, %r71, 2146435072;
+	mov.f64 	%fd103, %fd57;
+	@%p126 bra 	BB9_126;
 
-	setp.gtu.f64	%p121, %fd51, 0d7FF0000000000000;
-	mov.f64 	%fd105, %fd58;
-	@%p121 bra 	BB8_134;
+	setp.gtu.f64	%p127, %fd51, 0d7FF0000000000000;
+	mov.f64 	%fd103, %fd58;
+	@%p127 bra 	BB9_126;
 
 	abs.f64 	%fd90, %fd68;
-	setp.gtu.f64	%p122, %fd90, 0d7FF0000000000000;
-	mov.f64 	%fd104, %fd58;
-	mov.f64 	%fd105, %fd104;
-	@%p122 bra 	BB8_134;
+	setp.gtu.f64	%p128, %fd90, 0d7FF0000000000000;
+	mov.f64 	%fd102, %fd58;
+	mov.f64 	%fd103, %fd102;
+	@%p128 bra 	BB9_126;
 
+	{
+	.reg .b32 %temp; 
+	mov.b64 	{%r72, %temp}, %fd68;
+	}
 	and.b32  	%r73, %r5, 2147483647;
-	setp.ne.s32	%p123, %r73, 2146435072;
-	@%p123 bra 	BB8_130;
-
+	setp.eq.s32	%p129, %r73, 2146435072;
+	setp.eq.s32	%p130, %r72, 0;
+	and.pred  	%p131, %p129, %p130;
+	@%p131 bra 	BB9_125;
+	bra.uni 	BB9_123;
+
+BB9_125:
+	setp.gt.f64	%p135, %fd51, 0d3FF0000000000000;
+	selp.b32	%r81, 2146435072, 0, %p135;
+	xor.b32  	%r82, %r81, 2146435072;
+	setp.lt.s32	%p136, %r5, 0;
+	selp.b32	%r83, %r82, %r81, %p136;
+	setp.eq.f64	%p137, %fd1, 0dBFF0000000000000;
+	selp.b32	%r84, 1072693248, %r83, %p137;
+	mov.u32 	%r85, 0;
+	mov.b64 	%fd103, {%r85, %r84};
+	bra.uni 	BB9_126;
+
+BB9_59:
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r74, %temp}, %fd68;
+	mov.b64 	{%r37, %temp}, %fd68;
 	}
-	setp.eq.s32	%p124, %r74, 0;
-	@%p124 bra 	BB8_133;
+	and.b32  	%r38, %r2, 2147483647;
+	setp.eq.s32	%p64, %r38, 2146435072;
+	setp.eq.s32	%p65, %r37, 0;
+	and.pred  	%p66, %p64, %p65;
+	mov.f64 	%fd95, %fd24;
+	@!%p66 bra 	BB9_62;
+	bra.uni 	BB9_60;
+
+BB9_60:
+	shr.s32 	%r39, %r3, 31;
+	and.b32  	%r40, %r39, -2146435072;
+	selp.b32	%r41, -1048576, 2146435072, %p1;
+	add.s32 	%r42, %r41, %r40;
+	mov.u32 	%r43, 0;
+	mov.b64 	%fd95, {%r43, %r42};
+
+BB9_62:
+	setp.eq.f64	%p70, %fd1, 0d0000000000000000;
+	setp.eq.f64	%p71, %fd68, 0d3FF0000000000000;
+	or.pred  	%p72, %p71, %p70;
+	selp.f64	%fd98, 0d3FF0000000000000, %fd95, %p72;
 
-BB8_130:
-	and.b32  	%r75, %r4, 2147483647;
-	setp.ne.s32	%p125, %r75, 2146435072;
-	mov.f64 	%fd102, %fd57;
-	mov.f64 	%fd105, %fd102;
-	@%p125 bra 	BB8_134;
+BB9_65:
+	st.global.f64 	[%rd1], %fd98;
+	bra.uni 	BB9_130;
 
+BB9_123:
 	{
 	.reg .b32 %temp; 
-	mov.b64 	{%r76, %temp}, %fd1;
+	mov.b64 	{%r74, %temp}, %fd1;
 	}
-	setp.ne.s32	%p126, %r76, 0;
-	mov.f64 	%fd105, %fd57;
-	@%p126 bra 	BB8_134;
-
-	shr.s32 	%r77, %r5, 31;
-	and.b32  	%r78, %r77, -2146435072;
-	add.s32 	%r79, %r78, 2146435072;
-	or.b32  	%r80, %r79, -2147483648;
-	selp.b32	%r81, %r80, %r79, %p2;
-	mov.u32 	%r82, 0;
-	mov.b64 	%fd105, {%r82, %r81};
-	bra.uni 	BB8_134;
-
-BB8_65:
-	setp.gt.f64	%p63, %fd18, 0d3FF0000000000000;
-	selp.b32	%r45, 2146435072, 0, %p63;
-	xor.b32  	%r46, %r45, 2146435072;
-	setp.lt.s32	%p64, %r3, 0;
-	selp.b32	%r47, %r46, %r45, %p64;
-	setp.eq.f64	%p65, %fd68, 0dBFF0000000000000;
-	selp.b32	%r48, 1072693248, %r47, %p65;
-	mov.u32 	%r49, 0;
-	mov.b64 	%fd96, {%r49, %r48};
-
-BB8_66:
-	setp.eq.f64	%p66, %fd1, 0d0000000000000000;
-	setp.eq.f64	%p67, %fd68, 0d3FF0000000000000;
-	or.pred  	%p68, %p67, %p66;
-	selp.f64	%fd99, 0d3FF0000000000000, %fd96, %p68;
-
-BB8_69:
-	st.global.f64 	[%rd1], %fd99;
-	bra.uni 	BB8_138;
-
-BB8_133:
-	setp.gt.f64	%p127, %fd51, 0d3FF0000000000000;
-	selp.b32	%r83, 2146435072, 0, %p127;
-	xor.b32  	%r84, %r83, 2146435072;
-	setp.lt.s32	%p128, %r5, 0;
-	selp.b32	%r85, %r84, %r83, %p128;
-	setp.eq.f64	%p129, %fd1, 0dBFF0000000000000;
-	selp.b32	%r86, 1072693248, %r85, %p129;
-	mov.u32 	%r87, 0;
-	mov.b64 	%fd105, {%r87, %r86};
-
-BB8_134:
-	setp.eq.f64	%p130, %fd68, 0d0000000000000000;
-	setp.eq.f64	%p131, %fd1, 0d3FF0000000000000;
-	or.pred  	%p132, %p131, %p130;
-	selp.f64	%fd108, 0d3FF0000000000000, %fd105, %p132;
-
-BB8_137:
-	st.global.f64 	[%rd1], %fd108;
-
-BB8_138:
+	and.b32  	%r75, %r4, 2147483647;
+	setp.eq.s32	%p132, %r75, 2146435072;
+	setp.eq.s32	%p133, %r74, 0;
+	and.pred  	%p134, %p132, %p133;
+	mov.f64 	%fd103, %fd57;
+	@!%p134 bra 	BB9_126;
+	bra.uni 	BB9_124;
+
+BB9_124:
+	shr.s32 	%r76, %r5, 31;
+	and.b32  	%r77, %r76, -2146435072;
+	selp.b32	%r78, -1048576, 2146435072, %p2;
+	add.s32 	%r79, %r78, %r77;
+	mov.u32 	%r80, 0;
+	mov.b64 	%fd103, {%r80, %r79};
+
+BB9_126:
+	setp.eq.f64	%p138, %fd68, 0d0000000000000000;
+	setp.eq.f64	%p139, %fd1, 0d3FF0000000000000;
+	or.pred  	%p140, %p139, %p138;
+	selp.f64	%fd106, 0d3FF0000000000000, %fd103, %p140;
+
+BB9_129:
+	st.global.f64 	[%rd1], %fd106;
+
+BB9_130:
 	bar.sync 	0;
 	ret;
 }
@@ -1785,14 +1859,14 @@ BB8_138:
 	mov.u32 	%r5, %tid.x;
 	mad.lo.s32 	%r1, %r4, %r3, %r5;
 	setp.ge.s32	%p1, %r1, %r2;
-	@%p1 bra 	BB9_2;
+	@%p1 bra 	BB10_2;
 
 	cvta.to.global.u64 	%rd2, %rd1;
 	mul.wide.s32 	%rd3, %r1, 8;
 	add.s64 	%rd4, %rd2, %rd3;
 	st.global.f64 	[%rd4], %fd1;
 
-BB9_2:
+BB10_2:
 	ret;
 }
 
@@ -1832,10 +1906,10 @@ BB9_2:
 	setp.lt.s32	%p1, %r1, %r7;
 	setp.lt.s32	%p2, %r2, %r4;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB10_2;
-	bra.uni 	BB10_1;
+	@!%p3 bra 	BB11_2;
+	bra.uni 	BB11_1;
 
-BB10_1:
+BB11_1:
 	cvta.to.global.u64 	%rd5, %rd2;
 	mad.lo.s32 	%r13, %r1, %r4, %r2;
 	mul.wide.s32 	%rd6, %r13, 8;
@@ -1846,14 +1920,14 @@ BB10_1:
 	add.s64 	%rd9, %rd1, %rd8;
 	st.global.f64 	[%rd9], %fd1;
 
-BB10_2:
+BB11_2:
 	setp.lt.s32	%p4, %r1, %r5;
 	setp.lt.s32	%p5, %r2, %r6;
 	and.pred  	%p6, %p4, %p5;
-	@!%p6 bra 	BB10_4;
-	bra.uni 	BB10_3;
+	@!%p6 bra 	BB11_4;
+	bra.uni 	BB11_3;
 
-BB10_3:
+BB11_3:
 	cvta.to.global.u64 	%rd10, %rd3;
 	mad.lo.s32 	%r15, %r1, %r6, %r2;
 	mul.wide.s32 	%rd11, %r15, 8;
@@ -1865,7 +1939,7 @@ BB10_3:
 	add.s64 	%rd14, %rd1, %rd13;
 	st.global.f64 	[%rd14], %fd2;
 
-BB10_4:
+BB11_4:
 	ret;
 }
 
@@ -1904,10 +1978,10 @@ BB10_4:
 	setp.lt.s32	%p1, %r1, %r3;
 	setp.lt.s32	%p2, %r2, %r4;
 	and.pred  	%p3, %p1, %p2;
-	@!%p3 bra 	BB11_2;
-	bra.uni 	BB11_1;
+	@!%p3 bra 	BB12_2;
+	bra.uni 	BB12_1;
 
-BB11_1:
+BB12_1:
 	cvta.to.global.u64 	%rd5, %rd2;
 	mad.lo.s32 	%r12, %r1, %r4, %r2;
 	mul.wide.s32 	%rd6, %r12, 8;
@@ -1916,14 +1990,14 @@ BB11_1:
 	add.s64 	%rd8, %rd1, %rd6;
 	st.global.f64 	[%rd8], %fd1;
 
-BB11_2:
+BB12_2:
 	setp.lt.s32	%p4, %r1, %r5;
 	setp.lt.s32	%p5, %r2, %r6;
 	and.pred  	%p6, %p4, %p5;
-	@!%p6 bra 	BB11_4;
-	bra.uni 	BB11_3;
+	@!%p6 bra 	BB12_4;
+	bra.uni 	BB12_3;
 
-BB11_3:
+BB12_3:
 	cvta.to.global.u64 	%rd9, %rd3;
 	mad.lo.s32 	%r13, %r1, %r6, %r2;
 	mul.wide.s32 	%rd10, %r13, 8;
@@ -1935,7 +2009,7 @@ BB11_3:
 	add.s64 	%rd13, %rd1, %rd12;
 	st.global.f64 	[%rd13], %fd2;
 
-BB11_4:
+BB12_4:
 	ret;
 }
 
@@ -1963,9 +2037,9 @@ BB11_4:
 	mov.f64 	%fd76, 0d0000000000000000;
 	mov.f64 	%fd77, %fd76;
 	setp.ge.u32	%p1, %r32, %r5;
-	@%p1 bra 	BB12_4;
+	@%p1 bra 	BB13_4;
 
-BB12_1:
+BB13_1:
 	mov.f64 	%fd1, %fd77;
 	cvta.to.global.u64 	%rd4, %rd2;
 	mul.wide.u32 	%rd5, %r32, 8;
@@ -1974,23 +2048,23 @@ BB12_1:
 	add.f64 	%fd78, %fd1, %fd30;
 	add.s32 	%r3, %r32, %r9;
 	setp.ge.u32	%p2, %r3, %r5;
-	@%p2 bra 	BB12_3;
+	@%p2 bra 	BB13_3;
 
 	mul.wide.u32 	%rd8, %r3, 8;
 	add.s64 	%rd9, %rd4, %rd8;
 	ld.global.f64 	%fd31, [%rd9];
 	add.f64 	%fd78, %fd78, %fd31;
 
-BB12_3:
+BB13_3:
 	mov.f64 	%fd77, %fd78;
 	shl.b32 	%r12, %r9, 1;
 	mov.u32 	%r13, %nctaid.x;
 	mad.lo.s32 	%r32, %r12, %r13, %r32;
 	setp.lt.u32	%p3, %r32, %r5;
 	mov.f64 	%fd76, %fd77;
-	@%p3 bra 	BB12_1;
+	@%p3 bra 	BB13_1;
 
-BB12_4:
+BB13_4:
 	mov.f64 	%fd74, %fd76;
 	mul.wide.u32 	%rd10, %r6, 8;
 	mov.u64 	%rd11, sdata;
@@ -1998,130 +2072,130 @@ BB12_4:
 	st.shared.f64 	[%rd1], %fd74;
 	bar.sync 	0;
 	setp.lt.u32	%p4, %r9, 1024;
-	@%p4 bra 	BB12_8;
+	@%p4 bra 	BB13_8;
 
 	setp.gt.u32	%p5, %r6, 511;
 	mov.f64 	%fd75, %fd74;
-	@%p5 bra 	BB12_7;
+	@%p5 bra 	BB13_7;
 
 	ld.shared.f64 	%fd32, [%rd1+4096];
 	add.f64 	%fd75, %fd74, %fd32;
 	st.shared.f64 	[%rd1], %fd75;
 
-BB12_7:
+BB13_7:
 	mov.f64 	%fd74, %fd75;
 	bar.sync 	0;
 
-BB12_8:
+BB13_8:
 	mov.f64 	%fd72, %fd74;
 	setp.lt.u32	%p6, %r9, 512;
-	@%p6 bra 	BB12_12;
+	@%p6 bra 	BB13_12;
 
 	setp.gt.u32	%p7, %r6, 255;
 	mov.f64 	%fd73, %fd72;
-	@%p7 bra 	BB12_11;
+	@%p7 bra 	BB13_11;
 
 	ld.shared.f64 	%fd33, [%rd1+2048];
 	add.f64 	%fd73, %fd72, %fd33;
 	st.shared.f64 	[%rd1], %fd73;
 
-BB12_11:
+BB13_11:
 	mov.f64 	%fd72, %fd73;
 	bar.sync 	0;
 
-BB12_12:
+BB13_12:
 	mov.f64 	%fd70, %fd72;
 	setp.lt.u32	%p8, %r9, 256;
-	@%p8 bra 	BB12_16;
+	@%p8 bra 	BB13_16;
 
 	setp.gt.u32	%p9, %r6, 127;
 	mov.f64 	%fd71, %fd70;
-	@%p9 bra 	BB12_15;
+	@%p9 bra 	BB13_15;
 
 	ld.shared.f64 	%fd34, [%rd1+1024];
 	add.f64 	%fd71, %fd70, %fd34;
 	st.shared.f64 	[%rd1], %fd71;
 
-BB12_15:
+BB13_15:
 	mov.f64 	%fd70, %fd71;
 	bar.sync 	0;
 
-BB12_16:
+BB13_16:
 	mov.f64 	%fd68, %fd70;
 	setp.lt.u32	%p10, %r9, 128;
-	@%p10 bra 	BB12_20;
+	@%p10 bra 	BB13_20;
 
 	setp.gt.u32	%p11, %r6, 63;
 	mov.f64 	%fd69, %fd68;
-	@%p11 bra 	BB12_19;
+	@%p11 bra 	BB13_19;
 
 	ld.shared.f64 	%fd35, [%rd1+512];
 	add.f64 	%fd69, %fd68, %fd35;
 	st.shared.f64 	[%rd1], %fd69;
 
-BB12_19:
+BB13_19:
 	mov.f64 	%fd68, %fd69;
 	bar.sync 	0;
 
-BB12_20:
+BB13_20:
 	mov.f64 	%fd67, %fd68;
 	setp.gt.u32	%p12, %r6, 31;
-	@%p12 bra 	BB12_33;
+	@%p12 bra 	BB13_33;
 
 	setp.lt.u32	%p13, %r9, 64;
-	@%p13 bra 	BB12_23;
+	@%p13 bra 	BB13_23;
 
 	ld.volatile.shared.f64 	%fd36, [%rd1+256];
 	add.f64 	%fd67, %fd67, %fd36;
 	st.volatile.shared.f64 	[%rd1], %fd67;
 
-BB12_23:
+BB13_23:
 	mov.f64 	%fd66, %fd67;
 	setp.lt.u32	%p14, %r9, 32;
-	@%p14 bra 	BB12_25;
+	@%p14 bra 	BB13_25;
 
 	ld.volatile.shared.f64 	%fd37, [%rd1+128];
 	add.f64 	%fd66, %fd66, %fd37;
 	st.volatile.shared.f64 	[%rd1], %fd66;
 
-BB12_25:
+BB13_25:
 	mov.f64 	%fd65, %fd66;
 	setp.lt.u32	%p15, %r9, 16;
-	@%p15 bra 	BB12_27;
+	@%p15 bra 	BB13_27;
 
 	ld.volatile.shared.f64 	%fd38, [%rd1+64];
 	add.f64 	%fd65, %fd65, %fd38;
 	st.volatile.shared.f64 	[%rd1], %fd65;
 
-BB12_27:
+BB13_27:
 	mov.f64 	%fd64, %fd65;
 	setp.lt.u32	%p16, %r9, 8;
-	@%p16 bra 	BB12_29;
+	@%p16 bra 	BB13_29;
 
 	ld.volatile.shared.f64 	%fd39, [%rd1+32];
 	add.f64 	%fd64, %fd64, %fd39;
 	st.volatile.shared.f64 	[%rd1], %fd64;
 
-BB12_29:
+BB13_29:
 	mov.f64 	%fd63, %fd64;
 	setp.lt.u32	%p17, %r9, 4;
-	@%p17 bra 	BB12_31;
+	@%p17 bra 	BB13_31;
 
 	ld.volatile.shared.f64 	%fd40, [%rd1+16];
 	add.f64 	%fd63, %fd63, %fd40;
 	st.volatile.shared.f64 	[%rd1], %fd63;
 
-BB12_31:
+BB13_31:
 	setp.lt.u32	%p18, %r9, 2;
-	@%p18 bra 	BB12_33;
+	@%p18 bra 	BB13_33;
 
 	ld.volatile.shared.f64 	%fd41, [%rd1+8];
 	add.f64 	%fd42, %fd63, %fd41;
 	st.volatile.shared.f64 	[%rd1], %fd42;
 
-BB12_33:
+BB13_33:
 	setp.ne.s32	%p19, %r6, 0;
-	@%p19 bra 	BB12_35;
+	@%p19 bra 	BB13_35;
 
 	ld.shared.f64 	%fd43, [sdata];
 	cvta.to.global.u64 	%rd12, %rd3;
@@ -2129,7 +2203,7 @@ BB12_33:
 	add.s64 	%rd14, %rd12, %rd13;
 	st.global.f64 	[%rd14], %fd43;
 
-BB12_35:
+BB13_35:
 	ret;
 }
 
@@ -2153,17 +2227,17 @@ BB12_35:
 	ld.param.u32 	%r4, [reduce_row_sum_param_3];
 	mov.u32 	%r6, %ctaid.x;
 	setp.ge.u32	%p1, %r6, %r5;
-	@%p1 bra 	BB13_35;
+	@%p1 bra 	BB14_35;
 
 	mov.u32 	%r38, %tid.x;
 	mov.f64 	%fd72, 0d0000000000000000;
 	mov.f64 	%fd73, %fd72;
 	setp.ge.u32	%p2, %r38, %r4;
-	@%p2 bra 	BB13_4;
+	@%p2 bra 	BB14_4;
 
 	cvta.to.global.u64 	%rd3, %rd1;
 
-BB13_3:
+BB14_3:
 	mad.lo.s32 	%r8, %r6, %r4, %r38;
 	mul.wide.u32 	%rd4, %r8, 8;
 	add.s64 	%rd5, %rd3, %rd4;
@@ -2173,9 +2247,9 @@ BB13_3:
 	add.s32 	%r38, %r9, %r38;
 	setp.lt.u32	%p3, %r38, %r4;
 	mov.f64 	%fd72, %fd73;
-	@%p3 bra 	BB13_3;
+	@%p3 bra 	BB14_3;
 
-BB13_4:
+BB14_4:
 	mov.f64 	%fd70, %fd72;
 	mov.u32 	%r10, %tid.x;
 	mul.wide.u32 	%rd6, %r10, 8;
@@ -2185,130 +2259,130 @@ BB13_4:
 	bar.sync 	0;
 	mov.u32 	%r11, %ntid.x;
 	setp.lt.u32	%p4, %r11, 1024;
-	@%p4 bra 	BB13_8;
+	@%p4 bra 	BB14_8;
 
 	setp.gt.u32	%p5, %r10, 511;
 	mov.f64 	%fd71, %fd70;
-	@%p5 bra 	BB13_7;
+	@%p5 bra 	BB14_7;
 
 	ld.shared.f64 	%fd29, [%rd8+4096];
 	add.f64 	%fd71, %fd70, %fd29;
 	st.shared.f64 	[%rd8], %fd71;
 
-BB13_7:
+BB14_7:
 	mov.f64 	%fd70, %fd71;
 	bar.sync 	0;
 
-BB13_8:
+BB14_8:
 	mov.f64 	%fd68, %fd70;
 	setp.lt.u32	%p6, %r11, 512;
-	@%p6 bra 	BB13_12;
+	@%p6 bra 	BB14_12;
 
 	setp.gt.u32	%p7, %r10, 255;
 	mov.f64 	%fd69, %fd68;
-	@%p7 bra 	BB13_11;
+	@%p7 bra 	BB14_11;
 
 	ld.shared.f64 	%fd30, [%rd8+2048];
 	add.f64 	%fd69, %fd68, %fd30;
 	st.shared.f64 	[%rd8], %fd69;
 
-BB13_11:
+BB14_11:
 	mov.f64 	%fd68, %fd69;
 	bar.sync 	0;
 
-BB13_12:
+BB14_12:
 	mov.f64 	%fd66, %fd68;
 	setp.lt.u32	%p8, %r11, 256;
-	@%p8 bra 	BB13_16;
+	@%p8 bra 	BB14_16;
 
 	setp.gt.u32	%p9, %r10, 127;
 	mov.f64 	%fd67, %fd66;
-	@%p9 bra 	BB13_15;
+	@%p9 bra 	BB14_15;
 
 	ld.shared.f64 	%fd31, [%rd8+1024];
 	add.f64 	%fd67, %fd66, %fd31;
 	st.shared.f64 	[%rd8], %fd67;
 
-BB13_15:
+BB14_15:
 	mov.f64 	%fd66, %fd67;
 	bar.sync 	0;
 
-BB13_16:
+BB14_16:
 	mov.f64 	%fd64, %fd66;
 	setp.lt.u32	%p10, %r11, 128;
-	@%p10 bra 	BB13_20;
+	@%p10 bra 	BB14_20;
 
 	setp.gt.u32	%p11, %r10, 63;
 	mov.f64 	%fd65, %fd64;
-	@%p11 bra 	BB13_19;
+	@%p11 bra 	BB14_19;
 
 	ld.shared.f64 	%fd32, [%rd8+512];
 	add.f64 	%fd65, %fd64, %fd32;
 	st.shared.f64 	[%rd8], %fd65;
 
-BB13_19:
+BB14_19:
 	mov.f64 	%fd64, %fd65;
 	bar.sync 	0;
 
-BB13_20:
+BB14_20:
 	mov.f64 	%fd63, %fd64;
 	setp.gt.u32	%p12, %r10, 31;
-	@%p12 bra 	BB13_33;
+	@%p12 bra 	BB14_33;
 
 	setp.lt.u32	%p13, %r11, 64;
-	@%p13 bra 	BB13_23;
+	@%p13 bra 	BB14_23;
 
 	ld.volatile.shared.f64 	%fd33, [%rd8+256];
 	add.f64 	%fd63, %fd63, %fd33;
 	st.volatile.shared.f64 	[%rd8], %fd63;
 
-BB13_23:
+BB14_23:
 	mov.f64 	%fd62, %fd63;
 	setp.lt.u32	%p14, %r11, 32;
-	@%p14 bra 	BB13_25;
+	@%p14 bra 	BB14_25;
 
 	ld.volatile.shared.f64 	%fd34, [%rd8+128];
 	add.f64 	%fd62, %fd62, %fd34;
 	st.volatile.shared.f64 	[%rd8], %fd62;
 
-BB13_25:
+BB14_25:
 	mov.f64 	%fd61, %fd62;
 	setp.lt.u32	%p15, %r11, 16;
-	@%p15 bra 	BB13_27;
+	@%p15 bra 	BB14_27;
 
 	ld.volatile.shared.f64 	%fd35, [%rd8+64];
 	add.f64 	%fd61, %fd61, %fd35;
 	st.volatile.shared.f64 	[%rd8], %fd61;
 
-BB13_27:
+BB14_27:
 	mov.f64 	%fd60, %fd61;
 	setp.lt.u32	%p16, %r11, 8;
-	@%p16 bra 	BB13_29;
+	@%p16 bra 	BB14_29;
 
 	ld.volatile.shared.f64 	%fd36, [%rd8+32];
 	add.f64 	%fd60, %fd60, %fd36;
 	st.volatile.shared.f64 	[%rd8], %fd60;
 
-BB13_29:
+BB14_29:
 	mov.f64 	%fd59, %fd60;
 	setp.lt.u32	%p17, %r11, 4;
-	@%p17 bra 	BB13_31;
+	@%p17 bra 	BB14_31;
 
 	ld.volatile.shared.f64 	%fd37, [%rd8+16];
 	add.f64 	%fd59, %fd59, %fd37;
 	st.volatile.shared.f64 	[%rd8], %fd59;
 
-BB13_31:
+BB14_31:
 	setp.lt.u32	%p18, %r11, 2;
-	@%p18 bra 	BB13_33;
+	@%p18 bra 	BB14_33;
 
 	ld.volatile.shared.f64 	%fd38, [%rd8+8];
 	add.f64 	%fd39, %fd59, %fd38;
 	st.volatile.shared.f64 	[%rd8], %fd39;
 
-BB13_33:
+BB14_33:
 	setp.ne.s32	%p19, %r10, 0;
-	@%p19 bra 	BB13_35;
+	@%p19 bra 	BB14_35;
 
 	ld.shared.f64 	%fd40, [sdata];
 	cvta.to.global.u64 	%rd39, %rd2;
@@ -2316,7 +2390,7 @@ BB13_33:
 	add.s64 	%rd41, %rd39, %rd40;
 	st.global.f64 	[%rd41], %fd40;
 
-BB13_35:
+BB14_35:
 	ret;
 }
 
@@ -2343,18 +2417,18 @@ BB13_35:
 	mov.u32 	%r9, %tid.x;
 	mad.lo.s32 	%r1, %r7, %r8, %r9;
 	setp.ge.u32	%p1, %r1, %r6;
-	@%p1 bra 	BB14_5;
+	@%p1 bra 	BB15_5;
 
 	cvta.to.global.u64 	%rd1, %rd2;
 	mul.lo.s32 	%r2, %r6, %r5;
 	mov.f64 	%fd8, 0d0000000000000000;
 	mov.f64 	%fd9, %fd8;
 	setp.ge.u32	%p2, %r1, %r2;
-	@%p2 bra 	BB14_4;
+	@%p2 bra 	BB15_4;
 
 	mov.u32 	%r10, %r1;
 
-BB14_3:
+BB15_3:
 	mov.u32 	%r3, %r10;
 	mul.wide.u32 	%rd4, %r3, 8;
 	add.s64 	%rd5, %rd1, %rd4;
@@ -2364,15 +2438,15 @@ BB14_3:
 	setp.lt.u32	%p3, %r4, %r2;
 	mov.u32 	%r10, %r4;
 	mov.f64 	%fd8, %fd9;
-	@%p3 bra 	BB14_3;
+	@%p3 bra 	BB15_3;
 
-BB14_4:
+BB15_4:
 	cvta.to.global.u64 	%rd6, %rd3;
 	mul.wide.u32 	%rd7, %r1, 8;
 	add.s64 	%rd8, %rd6, %rd7;
 	st.global.f64 	[%rd8], %fd8;
 
-BB14_5:
+BB15_5:
 	ret;
 }
 
@@ -2400,9 +2474,9 @@ BB14_5:
 	mov.f64 	%fd76, 0dFFEFFFFFFFFFFFFF;
 	mov.f64 	%fd77, %fd76;
 	setp.ge.u32	%p1, %r32, %r5;
-	@%p1 bra 	BB15_4;
+	@%p1 bra 	BB16_4;
 
-BB15_1:
+BB16_1:
 	mov.f64 	%fd1, %fd77;
 	cvta.to.global.u64 	%rd4, %rd2;
 	mul.wide.u32 	%rd5, %r32, 8;
@@ -2411,23 +2485,23 @@ BB15_1:
 	max.f64 	%fd78, %fd1, %fd30;
 	add.s32 	%r3, %r32, %r9;
 	setp.ge.u32	%p2, %r3, %r5;
-	@%p2 bra 	BB15_3;
+	@%p2 bra 	BB16_3;
 
 	mul.wide.u32 	%rd8, %r3, 8;
 	add.s64 	%rd9, %rd4, %rd8;
 	ld.global.f64 	%fd31, [%rd9];
 	max.f64 	%fd78, %fd78, %fd31;
 
-BB15_3:
+BB16_3:
 	mov.f64 	%fd77, %fd78;
 	shl.b32 	%r12, %r9, 1;
 	mov.u32 	%r13, %nctaid.x;
 	mad.lo.s32 	%r32, %r12, %r13, %r32;
 	setp.lt.u32	%p3, %r32, %r5;
 	mov.f64 	%fd76, %fd77;
-	@%p3 bra 	BB15_1;
+	@%p3 bra 	BB16_1;
 
-BB15_4:
+BB16_4:
 	mov.f64 	%fd74, %fd76;
 	mul.wide.u32 	%rd10, %r6, 8;
 	mov.u64 	%rd11, sdata;
@@ -2435,130 +2509,130 @@ BB15_4:
 	st.shared.f64 	[%rd1], %fd74;
 	bar.sync 	0;
 	setp.lt.u32	%p4, %r9, 1024;
-	@%p4 bra 	BB15_8;
+	@%p4 bra 	BB16_8;
 
 	setp.gt.u32	%p5, %r6, 511;
 	mov.f64 	%fd75, %fd74;
-	@%p5 bra 	BB15_7;
+	@%p5 bra 	BB16_7;
 
 	ld.shared.f64 	%fd32, [%rd1+4096];
 	max.f64 	%fd75, %fd74, %fd32;
 	st.shared.f64 	[%rd1], %fd75;
 
-BB15_7:
+BB16_7:
 	mov.f64 	%fd74, %fd75;
 	bar.sync 	0;
 
-BB15_8:
+BB16_8:
 	mov.f64 	%fd72, %fd74;
 	setp.lt.u32	%p6, %r9, 512;
-	@%p6 bra 	BB15_12;
+	@%p6 bra 	BB16_12;
 
 	setp.gt.u32	%p7, %r6, 255;
 	mov.f64 	%fd73, %fd72;
-	@%p7 bra 	BB15_11;
+	@%p7 bra 	BB16_11;
 
 	ld.shared.f64 	%fd33, [%rd1+2048];
 	max.f64 	%fd73, %fd72, %fd33;
 	st.shared.f64 	[%rd1], %fd73;
 
-BB15_11:
+BB16_11:
 	mov.f64 	%fd72, %fd73;
 	bar.sync 	0;
 
-BB15_12:
+BB16_12:
 	mov.f64 	%fd70, %fd72;
 	setp.lt.u32	%p8, %r9, 256;
-	@%p8 bra 	BB15_16;
+	@%p8 bra 	BB16_16;
 
 	setp.gt.u32	%p9, %r6, 127;
 	mov.f64 	%fd71, %fd70;
-	@%p9 bra 	BB15_15;
+	@%p9 bra 	BB16_15;
 
 	ld.shared.f64 	%fd34, [%rd1+1024];
 	max.f64 	%fd71, %fd70, %fd34;
 	st.shared.f64 	[%rd1], %fd71;
 
-BB15_15:
+BB16_15:
 	mov.f64 	%fd70, %fd71;
 	bar.sync 	0;
 
-BB15_16:
+BB16_16:
 	mov.f64 	%fd68, %fd70;
 	setp.lt.u32	%p10, %r9, 128;
-	@%p10 bra 	BB15_20;
+	@%p10 bra 	BB16_20;
 
 	setp.gt.u32	%p11, %r6, 63;
 	mov.f64 	%fd69, %fd68;
-	@%p11 bra 	BB15_19;
+	@%p11 bra 	BB16_19;
 
 	ld.shared.f64 	%fd35, [%rd1+512];
 	max.f64 	%fd69, %fd68, %fd35;
 	st.shared.f64 	[%rd1], %fd69;
 
-BB15_19:
+BB16_19:
 	mov.f64 	%fd68, %fd69;
 	bar.sync 	0;
 
-BB15_20:
+BB16_20:
 	mov.f64 	%fd67, %fd68;
 	setp.gt.u32	%p12, %r6, 31;
-	@%p12 bra 	BB15_33;
+	@%p12 bra 	BB16_33;
 
 	setp.lt.u32	%p13, %r9, 64;
-	@%p13 bra 	BB15_23;
+	@%p13 bra 	BB16_23;
 
 	ld.volatile.shared.f64 	%fd36, [%rd1+256];
 	max.f64 	%fd67, %fd67, %fd36;
 	st.volatile.shared.f64 	[%rd1], %fd67;
 
-BB15_23:
+BB16_23:
 	mov.f64 	%fd66, %fd67;
 	setp.lt.u32	%p14, %r9, 32;
-	@%p14 bra 	BB15_25;
+	@%p14 bra 	BB16_25;
 
 	ld.volatile.shared.f64 	%fd37, [%rd1+128];
 	max.f64 	%fd66, %fd66, %fd37;
 	st.volatile.shared.f64 	[%rd1], %fd66;
 
-BB15_25:
+BB16_25:
 	mov.f64 	%fd65, %fd66;
 	setp.lt.u32	%p15, %r9, 16;
-	@%p15 bra 	BB15_27;
+	@%p15 bra 	BB16_27;
 
 	ld.volatile.shared.f64 	%fd38, [%rd1+64];
 	max.f64 	%fd65, %fd65, %fd38;
 	st.volatile.shared.f64 	[%rd1], %fd65;
 
-BB15_27:
+BB16_27:
 	mov.f64 	%fd64, %fd65;
 	setp.lt.u32	%p16, %r9, 8;
-	@%p16 bra 	BB15_29;
+	@%p16 bra 	BB16_29;
 
 	ld.volatile.shared.f64 	%fd39, [%rd1+32];
 	max.f64 	%fd64, %fd64, %fd39;
 	st.volatile.shared.f64 	[%rd1], %fd64;
 
-BB15_29:
+BB16_29:
 	mov.f64 	%fd63, %fd64;
 	setp.lt.u32	%p17, %r9, 4;
-	@%p17 bra 	BB15_31;
+	@%p17 bra 	BB16_31;
 
 	ld.volatile.shared.f64 	%fd40, [%rd1+16];
 	max.f64 	%fd63, %fd63, %fd40;
 	st.volatile.shared.f64 	[%rd1], %fd63;
 
-BB15_31:
+BB16_31:
 	setp.lt.u32	%p18, %r9, 2;
-	@%p18 bra 	BB15_33;
+	@%p18 bra 	BB16_33;
 
 	ld.volatile.shared.f64 	%fd41, [%rd1+8];
 	max.f64 	%fd42, %fd63, %fd41;
 	st.volatile.shared.f64 	[%rd1], %fd42;
 
-BB15_33:
+BB16_33:
 	setp.ne.s32	%p19, %r6, 0;
-	@%p19 bra 	BB15_35;
+	@%p19 bra 	BB16_35;
 
 	ld.shared.f64 	%fd43, [sdata];
 	cvta.to.global.u64 	%rd12, %rd3;
@@ -2566,7 +2640,7 @@ BB15_33:
 	add.s64 	%rd14, %rd12, %rd13;
 	st.global.f64 	[%rd14], %fd43;
 
-BB15_35:
+BB16_35:
 	ret;
 }
 
@@ -2590,17 +2664,17 @@ BB15_35:
 	ld.param.u32 	%r4, [reduce_row_max_param_3];
 	mov.u32 	%r6, %ctaid.x;
 	setp.ge.u32	%p1, %r6, %r5;
-	@%p1 bra 	BB16_35;
+	@%p1 bra 	BB17_35;
 
 	mov.u32 	%r38, %tid.x;
 	mov.f64 	%fd72, 0dFFEFFFFFFFFFFFFF;
 	mov.f64 	%fd73, %fd72;
 	setp.ge.u32	%p2, %r38, %r4;
-	@%p2 bra 	BB16_4;
+	@%p2 bra 	BB17_4;
 
 	cvta.to.global.u64 	%rd3, %rd1;
 
-BB16_3:
+BB17_3:
 	mad.lo.s32 	%r8, %r6, %r4, %r38;
 	mul.wide.u32 	%rd4, %r8, 8;
 	add.s64 	%rd5, %rd3, %rd4;
@@ -2610,9 +2684,9 @@ BB16_3:
 	add.s32 	%r38, %r9, %r38;
 	setp.lt.u32	%p3, %r38, %r4;
 	mov.f64 	%fd72, %fd73;
-	@%p3 bra 	BB16_3;
+	@%p3 bra 	BB17_3;
 
-BB16_4:
+BB17_4:
 	mov.f64 	%fd70, %fd72;
 	mov.u32 	%r10, %tid.x;
 	mul.wide.u32 	%rd6, %r10, 8;
@@ -2622,130 +2696,130 @@ BB16_4:
 	bar.sync 	0;
 	mov.u32 	%r11, %ntid.x;
 	setp.lt.u32	%p4, %r11, 1024;
-	@%p4 bra 	BB16_8;
+	@%p4 bra 	BB17_8;
 
 	setp.gt.u32	%p5, %r10, 511;
 	mov.f64 	%fd71, %fd70;
-	@%p5 bra 	BB16_7;
+	@%p5 bra 	BB17_7;
 
 	ld.shared.f64 	%fd29, [%rd8+4096];
 	max.f64 	%fd71, %fd70, %fd29;
 	st.shared.f64 	[%rd8], %fd71;
 
-BB16_7:
+BB17_7:
 	mov.f64 	%fd70, %fd71;
 	bar.sync 	0;
 
-BB16_8:
+BB17_8:
 	mov.f64 	%fd68, %fd70;
 	setp.lt.u32	%p6, %r11, 512;
-	@%p6 bra 	BB16_12;
+	@%p6 bra 	BB17_12;
 
 	setp.gt.u32	%p7, %r10, 255;
 	mov.f64 	%fd69, %fd68;
-	@%p7 bra 	BB16_11;
+	@%p7 bra 	BB17_11;
 
 	ld.shared.f64 	%fd30, [%rd8+2048];
 	max.f64 	%fd69, %fd68, %fd30;
 	st.shared.f64 	[%rd8], %fd69;
 
-BB16_11:
+BB17_11:
 	mov.f64 	%fd68, %fd69;
 	bar.sync 	0;
 
-BB16_12:
+BB17_12:
 	mov.f64 	%fd66, %fd68;
 	setp.lt.u32	%p8, %r11, 256;
-	@%p8 bra 	BB16_16;
+	@%p8 bra 	BB17_16;
 
 	setp.gt.u32	%p9, %r10, 127;
 	mov.f64 	%fd67, %fd66;
-	@%p9 bra 	BB16_15;
+	@%p9 bra 	BB17_15;
 
 	ld.shared.f64 	%fd31, [%rd8+1024];
 	max.f64 	%fd67, %fd66, %fd31;
 	st.shared.f64 	[%rd8], %fd67;
 
-BB16_15:
+BB17_15:
 	mov.f64 	%fd66, %fd67;
 	bar.sync 	0;
 
-BB16_16:
+BB17_16:
 	mov.f64 	%fd64, %fd66;
 	setp.lt.u32	%p10, %r11, 128;
-	@%p10 bra 	BB16_20;
+	@%p10 bra 	BB17_20;
 
 	setp.gt.u32	%p11, %r10, 63;
 	mov.f64 	%fd65, %fd64;
-	@%p11 bra 	BB16_19;
+	@%p11 bra 	BB17_19;
 
 	ld.shared.f64 	%fd32, [%rd8+512];
 	max.f64 	%fd65, %fd64, %fd32;
 	st.shared.f64 	[%rd8], %fd65;
 
-BB16_19:
+BB17_19:
 	mov.f64 	%fd64, %fd65;
 	bar.sync 	0;
 
-BB16_20:
+BB17_20:
 	mov.f64 	%fd63, %fd64;
 	setp.gt.u32	%p12, %r10, 31;
-	@%p12 bra 	BB16_33;
+	@%p12 bra 	BB17_33;
 
 	setp.lt.u32	%p13, %r11, 64;
-	@%p13 bra 	BB16_23;
+	@%p13 bra 	BB17_23;
 
 	ld.volatile.shared.f64 	%fd33, [%rd8+256];
 	max.f64 	%fd63, %fd63, %fd33;
 	st.volatile.shared.f64 	[%rd8], %fd63;
 
-BB16_23:
+BB17_23:
 	mov.f64 	%fd62, %fd63;
 	setp.lt.u32	%p14, %r11, 32;
-	@%p14 bra 	BB16_25;
+	@%p14 bra 	BB17_25;
 
 	ld.volatile.shared.f64 	%fd34, [%rd8+128];
 	max.f64 	%fd62, %fd62, %fd34;
 	st.volatile.shared.f64 	[%rd8], %fd62;
 
-BB16_25:
+BB17_25:
 	mov.f64 	%fd61, %fd62;
 	setp.lt.u32	%p15, %r11, 16;
-	@%p15 bra 	BB16_27;
+	@%p15 bra 	BB17_27;
 
 	ld.volatile.shared.f64 	%fd35, [%rd8+64];
 	max.f64 	%fd61, %fd61, %fd35;
 	st.volatile.shared.f64 	[%rd8], %fd61;
 
-BB16_27:
+BB17_27:
 	mov.f64 	%fd60, %fd61;
 	setp.lt.u32	%p16, %r11, 8;
-	@%p16 bra 	BB16_29;
+	@%p16 bra 	BB17_29;
 
 	ld.volatile.shared.f64 	%fd36, [%rd8+32];
 	max.f64 	%fd60, %fd60, %fd36;
 	st.volatile.shared.f64 	[%rd8], %fd60;
 
-BB16_29:
+BB17_29:
 	mov.f64 	%fd59, %fd60;
 	setp.lt.u32	%p17, %r11, 4;
-	@%p17 bra 	BB16_31;
+	@%p17 bra 	BB17_31;
 
 	ld.volatile.shared.f64 	%fd37, [%rd8+16];
 	max.f64 	%fd59, %fd59, %fd37;
 	st.volatile.shared.f64 	[%rd8], %fd59;
 
-BB16_31:
+BB17_31:
 	setp.lt.u32	%p18, %r11, 2;
-	@%p18 bra 	BB16_33;
+	@%p18 bra 	BB17_33;
 
 	ld.volatile.shared.f64 	%fd38, [%rd8+8];
 	max.f64 	%fd39, %fd59, %fd38;
 	st.volatile.shared.f64 	[%rd8], %fd39;
 
-BB16_33:
+BB17_33:
 	setp.ne.s32	%p19, %r10, 0;
-	@%p19 bra 	BB16_35;
+	@%p19 bra 	BB17_35;
 
 	ld.shared.f64 	%fd40, [sdata];
 	cvta.to.global.u64 	%rd39, %rd2;
@@ -2753,7 +2827,7 @@ BB16_33:
 	add.s64 	%rd41, %rd39, %rd40;
 	st.global.f64 	[%rd41], %fd40;
 
-BB16_35:
+BB17_35:
 	ret;
 }
 
@@ -2780,18 +2854,18 @@ BB16_35:
 	mov.u32 	%r9, %tid.x;
 	mad.lo.s32 	%r1, %r7, %r8, %r9;
 	setp.ge.u32	%p1, %r1, %r6;
-	@%p1 bra 	BB17_5;
+	@%p1 bra 	BB18_5;
 
 	cvta.to.global.u64 	%rd1, %rd2;
 	mul.lo.s32 	%r2, %r6, %r5;
 	mov.f64 	%fd8, 0dFFEFFFFFFFFFFFFF;
 	mov.f64 	%fd9, %fd8;
 	setp.ge.u32	%p2, %r1, %r2;
-	@%p2 bra 	BB17_4;
+	@%p2 bra 	BB18_4;
 
 	mov.u32 	%r10, %r1;
 
-BB17_3:
+BB18_3:
 	mov.u32 	%r3, %r10;
 	mul.wide.u32 	%rd4, %r3, 8;
 	add.s64 	%rd5, %rd1, %rd4;
@@ -2801,15 +2875,15 @@ BB17_3:
 	setp.lt.u32	%p3, %r4, %r2;
 	mov.u32 	%r10, %r4;
 	mov.f64 	%fd8, %fd9;
-	@%p3 bra 	BB17_3;
+	@%p3 bra 	BB18_3;
 
-BB17_4:
+BB18_4:
 	cvta.to.global.u64 	%rd6, %rd3;
 	mul.wide.u32 	%rd7, %r1, 8;
 	add.s64 	%rd8, %rd6, %rd7;
 	st.global.f64 	[%rd8], %fd8;
 
-BB17_5:
+BB18_5:
 	ret;
 }
 
@@ -2837,9 +2911,9 @@ BB17_5:
 	mov.f64 	%fd76, 0d7FEFFFFFFFFFFFFF;
 	mov.f64 	%fd77, %fd76;
 	setp.ge.u32	%p1, %r32, %r5;
-	@%p1 bra 	BB18_4;
+	@%p1 bra 	BB19_4;
 
-BB18_1:
+BB19_1:
 	mov.f64 	%fd1, %fd77;
 	cvta.to.global.u64 	%rd4, %rd2;
 	mul.wide.u32 	%rd5, %r32, 8;
@@ -2848,23 +2922,23 @@ BB18_1:
 	min.f64 	%fd78, %fd1, %fd30;
 	add.s32 	%r3, %r32, %r9;
 	setp.ge.u32	%p2, %r3, %r5;
-	@%p2 bra 	BB18_3;
+	@%p2 bra 	BB19_3;
 
 	mul.wide.u32 	%rd8, %r3, 8;
 	add.s64 	%rd9, %rd4, %rd8;
 	ld.global.f64 	%fd31, [%rd9];
 	min.f64 	%fd78, %fd78, %fd31;
 
-BB18_3:
+BB19_3:
 	mov.f64 	%fd77, %fd78;
 	shl.b32 	%r12, %r9, 1;
 	mov.u32 	%r13, %nctaid.x;
 	mad.lo.s32 	%r32, %r12, %r13, %r32;
 	setp.lt.u32	%p3, %r32, %r5;
 	mov.f64 	%fd76, %fd77;
-	@%p3 bra 	BB18_1;
+	@%p3 bra 	BB19_1;
 
-BB18_4:
+BB19_4:
 	mov.f64 	%fd74, %fd76;
 	mul.wide.u32 	%rd10, %r6, 8;
 	mov.u64 	%rd11, sdata;
@@ -2872,130 +2946,130 @@ BB18_4:
 	st.shared.f64 	[%rd1], %fd74;
 	bar.sync 	0;
 	setp.lt.u32	%p4, %r9, 1024;
-	@%p4 bra 	BB18_8;
+	@%p4 bra 	BB19_8;
 
 	setp.gt.u32	%p5, %r6, 511;
 	mov.f64 	%fd75, %fd74;
-	@%p5 bra 	BB18_7;
+	@%p5 bra 	BB19_7;
 
 	ld.shared.f64 	%fd32, [%rd1+4096];
 	min.f64 	%fd75, %fd74, %fd32;
 	st.shared.f64 	[%rd1], %fd75;
 
-BB18_7:
+BB19_7:
 	mov.f64 	%fd74, %fd75;
 	bar.sync 	0;
 
-BB18_8:
+BB19_8:
 	mov.f64 	%fd72, %fd74;
 	setp.lt.u32	%p6, %r9, 512;
-	@%p6 bra 	BB18_12;
+	@%p6 bra 	BB19_12;
 
 	setp.gt.u32	%p7, %r6, 255;
 	mov.f64 	%fd73, %fd72;
-	@%p7 bra 	BB18_11;
+	@%p7 bra 	BB19_11;
 
 	ld.shared.f64 	%fd33, [%rd1+2048];
 	min.f64 	%fd73, %fd72, %fd33;
 	st.shared.f64 	[%rd1], %fd73;
 
-BB18_11:
+BB19_11:
 	mov.f64 	%fd72, %fd73;
 	bar.sync 	0;
 
-BB18_12:
+BB19_12:
 	mov.f64 	%fd70, %fd72;
 	setp.lt.u32	%p8, %r9, 256;
-	@%p8 bra 	BB18_16;
+	@%p8 bra 	BB19_16;
 
 	setp.gt.u32	%p9, %r6, 127;
 	mov.f64 	%fd71, %fd70;
-	@%p9 bra 	BB18_15;
+	@%p9 bra 	BB19_15;
 
 	ld.shared.f64 	%fd34, [%rd1+1024];
 	min.f64 	%fd71, %fd70, %fd34;
 	st.shared.f64 	[%rd1], %fd71;
 
-BB18_15:
+BB19_15:
 	mov.f64 	%fd70, %fd71;
 	bar.sync 	0;
 
-BB18_16:
+BB19_16:
 	mov.f64 	%fd68, %fd70;
 	setp.lt.u32	%p10, %r9, 128;
-	@%p10 bra 	BB18_20;
+	@%p10 bra 	BB19_20;
 
 	setp.gt.u32	%p11, %r6, 63;
 	mov.f64 	%fd69, %fd68;
-	@%p11 bra 	BB18_19;
+	@%p11 bra 	BB19_19;
 
 	ld.shared.f64 	%fd35, [%rd1+512];
 	min.f64 	%fd69, %fd68, %fd35;
 	st.shared.f64 	[%rd1], %fd69;
 
-BB18_19:
+BB19_19:
 	mov.f64 	%fd68, %fd69;
 	bar.sync 	0;
 
-BB18_20:
+BB19_20:
 	mov.f64 	%fd67, %fd68;
 	setp.gt.u32	%p12, %r6, 31;
-	@%p12 bra 	BB18_33;
+	@%p12 bra 	BB19_33;
 
 	setp.lt.u32	%p13, %r9, 64;
-	@%p13 bra 	BB18_23;
+	@%p13 bra 	BB19_23;
 
 	ld.volatile.shared.f64 	%fd36, [%rd1+256];
 	min.f64 	%fd67, %fd67, %fd36;
 	st.volatile.shared.f64 	[%rd1], %fd67;
 
-BB18_23:
+BB19_23:
 	mov.f64 	%fd66, %fd67;
 	setp.lt.u32	%p14, %r9, 32;
-	@%p14 bra 	BB18_25;
+	@%p14 bra 	BB19_25;
 
 	ld.volatile.shared.f64 	%fd37, [%rd1+128];
 	min.f64 	%fd66, %fd66, %fd37;
 	st.volatile.shared.f64 	[%rd1], %fd66;
 
-BB18_25:
+BB19_25:
 	mov.f64 	%fd65, %fd66;
 	setp.lt.u32	%p15, %r9, 16;
-	@%p15 bra 	BB18_27;
+	@%p15 bra 	BB19_27;
 
 	ld.volatile.shared.f64 	%fd38, [%rd1+64];
 	min.f64 	%fd65, %fd65, %fd38;
 	st.volatile.shared.f64 	[%rd1], %fd65;
 
-BB18_27:
+BB19_27:
 	mov.f64 	%fd64, %fd65;
 	setp.lt.u32	%p16, %r9, 8;
-	@%p16 bra 	BB18_29;
+	@%p16 bra 	BB19_29;
 
 	ld.volatile.shared.f64 	%fd39, [%rd1+32];
 	min.f64 	%fd64, %fd64, %fd39;
 	st.volatile.shared.f64 	[%rd1], %fd64;
 
-BB18_29:
+BB19_29:
 	mov.f64 	%fd63, %fd64;
 	setp.lt.u32	%p17, %r9, 4;
-	@%p17 bra 	BB18_31;
+	@%p17 bra 	BB19_31;
 
 	ld.volatile.shared.f64 	%fd40, [%rd1+16];
 	min.f64 	%fd63, %fd63, %fd40;
 	st.volatile.shared.f64 	[%rd1], %fd63;
 
-BB18_31:
+BB19_31:
 	setp.lt.u32	%p18, %r9, 2;
-	@%p18 bra 	BB18_33;
+	@%p18 bra 	BB19_33;
 
 	ld.volatile.shared.f64 	%fd41, [%rd1+8];
 	min.f64 	%fd42, %fd63, %fd41;
 	st.volatile.shared.f64 	[%rd1], %fd42;
 
-BB18_33:
+BB19_33:
 	setp.ne.s32	%p19, %r6, 0;
-	@%p19 bra 	BB18_35;
+	@%p19 bra 	BB19_35;
 
 	ld.shared.f64 	%fd43, [sdata];
 	cvta.to.global.u64 	%rd12, %rd3;
@@ -3003,7 +3077,7 @@ BB18_33:
 	add.s64 	%rd14, %rd12, %rd13;
 	st.global.f64 	[%rd14], %fd43;
 
-BB18_35:
+BB19_35:
 	ret;
 }
 
@@ -3027,17 +3101,17 @@ BB18_35:
 	ld.param.u32 	%r4, [reduce_row_min_param_3];
 	mov.u32 	%r6, %ctaid.x;
 	setp.ge.u32	%p1, %r6, %r5;
-	@%p1 bra 	BB19_35;
+	@%p1 bra 	BB20_35;
 
 	mov.u32 	%r38, %tid.x;
 	mov.f64 	%fd72, 0d7FEFFFFFFFFFFFFF;
 	mov.f64 	%fd73, %fd72;
 	setp.ge.u32	%p2, %r38, %r4;
-	@%p2 bra 	BB19_4;
+	@%p2 bra 	BB20_4;
 
 	cvta.to.global.u64 	%rd3, %rd1;
 
-BB19_3:
+BB20_3:
 	mad.lo.s32 	%r8, %r6, %r4, %r38;
 	mul.wide.u32 	%rd4, %r8, 8;
 	add.s64 	%rd5, %rd3, %rd4;
@@ -3047,9 +3121,9 @@ BB19_3:
 	add.s32 	%r38, %r9, %r38;
 	setp.lt.u32	%p3, %r38, %r4;
 	mov.f64 	%fd72, %fd73;
-	@%p3 bra 	BB19_3;
+	@%p3 bra 	BB20_3;
 
-BB19_4:
+BB20_4:
 	mov.f64 	%fd70, %fd72;
 	mov.u32 	%r10, %tid.x;
 	mul.wide.u32 	%rd6, %r10, 8;
@@ -3059,130 +3133,130 @@ BB19_4:
 	bar.sync 	0;
 	mov.u32 	%r11, %ntid.x;
 	setp.lt.u32	%p4, %r11, 1024;
-	@%p4 bra 	BB19_8;
+	@%p4 bra 	BB20_8;
 
 	setp.gt.u32	%p5, %r10, 511;
 	mov.f64 	%fd71, %fd70;
-	@%p5 bra 	BB19_7;
+	@%p5 bra 	BB20_7;
 
 	ld.shared.f64 	%fd29, [%rd8+4096];
 	min.f64 	%fd71, %fd70, %fd29;
 	st.shared.f64 	[%rd8], %fd71;
 
-BB19_7:
+BB20_7:
 	mov.f64 	%fd70, %fd71;
 	bar.sync 	0;
 
-BB19_8:
+BB20_8:
 	mov.f64 	%fd68, %fd70;
 	setp.lt.u32	%p6, %r11, 512;
-	@%p6 bra 	BB19_12;
+	@%p6 bra 	BB20_12;
 
 	setp.gt.u32	%p7, %r10, 255;
 	mov.f64 	%fd69, %fd68;
-	@%p7 bra 	BB19_11;
+	@%p7 bra 	BB20_11;
 
 	ld.shared.f64 	%fd30, [%rd8+2048];
 	min.f64 	%fd69, %fd68, %fd30;
 	st.shared.f64 	[%rd8], %fd69;
 
-BB19_11:
+BB20_11:
 	mov.f64 	%fd68, %fd69;
 	bar.sync 	0;
 
-BB19_12:
+BB20_12:
 	mov.f64 	%fd66, %fd68;
 	setp.lt.u32	%p8, %r11, 256;
-	@%p8 bra 	BB19_16;
+	@%p8 bra 	BB20_16;
 
 	setp.gt.u32	%p9, %r10, 127;
 	mov.f64 	%fd67, %fd66;
-	@%p9 bra 	BB19_15;
+	@%p9 bra 	BB20_15;
 
 	ld.shared.f64 	%fd31, [%rd8+1024];
 	min.f64 	%fd67, %fd66, %fd31;
 	st.shared.f64 	[%rd8], %fd67;
 
-BB19_15:
+BB20_15:
 	mov.f64 	%fd66, %fd67;
 	bar.sync 	0;
 
-BB19_16:
+BB20_16:
 	mov.f64 	%fd64, %fd66;
 	setp.lt.u32	%p10, %r11, 128;
-	@%p10 bra 	BB19_20;
+	@%p10 bra 	BB20_20;
 
 	setp.gt.u32	%p11, %r10, 63;
 	mov.f64 	%fd65, %fd64;
-	@%p11 bra 	BB19_19;
+	@%p11 bra 	BB20_19;
 
 	ld.shared.f64 	%fd32, [%rd8+512];
 	min.f64 	%fd65, %fd64, %fd32;
 	st.shared.f64 	[%rd8], %fd65;
 
-BB19_19:
+BB20_19:
 	mov.f64 	%fd64, %fd65;
 	bar.sync 	0;
 
-BB19_20:
+BB20_20:
 	mov.f64 	%fd63, %fd64;
 	setp.gt.u32	%p12, %r10, 31;
-	@%p12 bra 	BB19_33;
+	@%p12 bra 	BB20_33;
 
 	setp.lt.u32	%p13, %r11, 64;
-	@%p13 bra 	BB19_23;
+	@%p13 bra 	BB20_23;
 
 	ld.volatile.shared.f64 	%fd33, [%rd8+256];
 	min.f64 	%fd63, %fd63, %fd33;
 	st.volatile.shared.f64 	[%rd8], %fd63;
 
-BB19_23:
+BB20_23:
 	mov.f64 	%fd62, %fd63;
 	setp.lt.u32	%p14, %r11, 32;
-	@%p14 bra 	BB19_25;
+	@%p14 bra 	BB20_25;
 
 	ld.volatile.shared.f64 	%fd34, [%rd8+128];
 	min.f64 	%fd62, %fd62, %fd34;
 	st.volatile.shared.f64 	[%rd8], %fd62;
 
-BB19_25:
+BB20_25:
 	mov.f64 	%fd61, %fd62;
 	setp.lt.u32	%p15, %r11, 16;
-	@%p15 bra 	BB19_27;
+	@%p15 bra 	BB20_27;
 
 	ld.volatile.shared.f64 	%fd35, [%rd8+64];
 	min.f64 	%fd61, %fd61, %fd35;
 	st.volatile.shared.f64 	[%rd8], %fd61;
 
-BB19_27:
+BB20_27:
 	mov.f64 	%fd60, %fd61;
 	setp.lt.u32	%p16, %r11, 8;
-	@%p16 bra 	BB19_29;
+	@%p16 bra 	BB20_29;
 
 	ld.volatile.shared.f64 	%fd36, [%rd8+32];
 	min.f64 	%fd60, %fd60, %fd36;
 	st.volatile.shared.f64 	[%rd8], %fd60;
 
-BB19_29:
+BB20_29:
 	mov.f64 	%fd59, %fd60;
 	setp.lt.u32	%p17, %r11, 4;
-	@%p17 bra 	BB19_31;
+	@%p17 bra 	BB20_31;
 
 	ld.volatile.shared.f64 	%fd37, [%rd8+16];
 	min.f64 	%fd59, %fd59, %fd37;
 	st.volatile.shared.f64 	[%rd8], %fd59;
 
-BB19_31:
+BB20_31:
 	setp.lt.u32	%p18, %r11, 2;
-	@%p18 bra 	BB19_33;
+	@%p18 bra 	BB20_33;
 
 	ld.volatile.shared.f64 	%fd38, [%rd8+8];
 	min.f64 	%fd39, %fd59, %fd38;
 	st.volatile.shared.f64 	[%rd8], %fd39;
 
-BB19_33:
+BB20_33:
 	setp.ne.s32	%p19, %r10, 0;
-	@%p19 bra 	BB19_35;
+	@%p19 bra 	BB20_35;
 
 	ld.shared.f64 	%fd40, [sdata];
 	cvta.to.global.u64 	%rd39, %rd2;
@@ -3190,7 +3264,7 @@ BB19_33:
 	add.s64 	%rd41, %rd39, %rd40;
 	st.global.f64 	[%rd41], %fd40;
 
-BB19_35:
+BB20_35:
 	ret;
 }
 
@@ -3217,18 +3291,18 @@ BB19_35:
 	mov.u32 	%r9, %tid.x;
 	mad.lo.s32 	%r1, %r7, %r8, %r9;
 	setp.ge.u32	%p1, %r1, %r6;
-	@%p1 bra 	BB20_5;
+	@%p1 bra 	BB21_5;
 
 	cvta.to.global.u64 	%rd1, %rd2;
 	mul.lo.s32 	%r2, %r6, %r5;
 	mov.f64 	%fd8, 0d7FEFFFFFFFFFFFFF;
 	mov.f64 	%fd9, %fd8;
 	setp.ge.u32	%p2, %r1, %r2;
-	@%p2 bra 	BB20_4;
+	@%p2 bra 	BB21_4;
 
 	mov.u32 	%r10, %r1;
 
-BB20_3:
+BB21_3:
 	mov.u32 	%r3, %r10;
 	mul.wide.u32 	%rd4, %r3, 8;
 	add.s64 	%rd5, %rd1, %rd4;
@@ -3238,15 +3312,15 @@ BB20_3:
 	setp.lt.u32	%p3, %r4, %r2;
 	mov.u32 	%r10, %r4;
 	mov.f64 	%fd8, %fd9;
-	@%p3 bra 	BB20_3;
+	@%p3 bra 	BB21_3;
 
-BB20_4:
+BB21_4:
 	cvta.to.global.u64 	%rd6, %rd3;
 	mul.wide.u32 	%rd7, %r1, 8;
 	add.s64 	%rd8, %rd6, %rd7;
 	st.global.f64 	[%rd8], %fd8;
 
-BB20_5:
+BB21_5:
 	ret;
 }
 
@@ -3274,9 +3348,9 @@ BB20_5:
 	mov.f64 	%fd76, 0d3FF0000000000000;
 	mov.f64 	%fd77, %fd76;
 	setp.ge.u32	%p1, %r32, %r5;
-	@%p1 bra 	BB21_4;
+	@%p1 bra 	BB22_4;
 
-BB21_1:
+BB22_1:
 	mov.f64 	%fd1, %fd77;
 	cvta.to.global.u64 	%rd4, %rd2;
 	mul.wide.u32 	%rd5, %r32, 8;
@@ -3285,23 +3359,23 @@ BB21_1:
 	mul.f64 	%fd78, %fd1, %fd30;
 	add.s32 	%r3, %r32, %r9;
 	setp.ge.u32	%p2, %r3, %r5;
-	@%p2 bra 	BB21_3;
+	@%p2 bra 	BB22_3;
 
 	mul.wide.u32 	%rd8, %r3, 8;
 	add.s64 	%rd9, %rd4, %rd8;
 	ld.global.f64 	%fd31, [%rd9];
 	mul.f64 	%fd78, %fd78, %fd31;
 
-BB21_3:
+BB22_3:
 	mov.f64 	%fd77, %fd78;
 	shl.b32 	%r12, %r9, 1;
 	mov.u32 	%r13, %nctaid.x;
 	mad.lo.s32 	%r32, %r12, %r13, %r32;
 	setp.lt.u32	%p3, %r32, %r5;
 	mov.f64 	%fd76, %fd77;
-	@%p3 bra 	BB21_1;
+	@%p3 bra 	BB22_1;
 
-BB21_4:
+BB22_4:
 	mov.f64 	%fd74, %fd76;
 	mul.wide.u32 	%rd10, %r6, 8;
 	mov.u64 	%rd11, sdata;
@@ -3309,130 +3383,130 @@ BB21_4:
 	st.shared.f64 	[%rd1], %fd74;
 	bar.sync 	0;
 	setp.lt.u32	%p4, %r9, 1024;
-	@%p4 bra 	BB21_8;
+	@%p4 bra 	BB22_8;
 
 	setp.gt.u32	%p5, %r6, 511;
 	mov.f64 	%fd75, %fd74;
-	@%p5 bra 	BB21_7;
+	@%p5 bra 	BB22_7;
 
 	ld.shared.f64 	%fd32, [%rd1+4096];
 	mul.f64 	%fd75, %fd74, %fd32;
 	st.shared.f64 	[%rd1], %fd75;
 
-BB21_7:
+BB22_7:
 	mov.f64 	%fd74, %fd75;
 	bar.sync 	0;
 
-BB21_8:
+BB22_8:
 	mov.f64 	%fd72, %fd74;
 	setp.lt.u32	%p6, %r9, 512;
-	@%p6 bra 	BB21_12;
+	@%p6 bra 	BB22_12;
 
 	setp.gt.u32	%p7, %r6, 255;
 	mov.f64 	%fd73, %fd72;
-	@%p7 bra 	BB21_11;
+	@%p7 bra 	BB22_11;
 
 	ld.shared.f64 	%fd33, [%rd1+2048];
 	mul.f64 	%fd73, %fd72, %fd33;
 	st.shared.f64 	[%rd1], %fd73;
 
-BB21_11:
+BB22_11:
 	mov.f64 	%fd72, %fd73;
 	bar.sync 	0;
 
-BB21_12:
+BB22_12:
 	mov.f64 	%fd70, %fd72;
 	setp.lt.u32	%p8, %r9, 256;
-	@%p8 bra 	BB21_16;
+	@%p8 bra 	BB22_16;
 
 	setp.gt.u32	%p9, %r6, 127;
 	mov.f64 	%fd71, %fd70;
-	@%p9 bra 	BB21_15;
+	@%p9 bra 	BB22_15;
 
 	ld.shared.f64 	%fd34, [%rd1+1024];
 	mul.f64 	%fd71, %fd70, %fd34;
 	st.shared.f64 	[%rd1], %fd71;
 
-BB21_15:
+BB22_15:
 	mov.f64 	%fd70, %fd71;
 	bar.sync 	0;
 
-BB21_16:
+BB22_16:
 	mov.f64 	%fd68, %fd70;
 	setp.lt.u32	%p10, %r9, 128;
-	@%p10 bra 	BB21_20;
+	@%p10 bra 	BB22_20;
 
 	setp.gt.u32	%p11, %r6, 63;
 	mov.f64 	%fd69, %fd68;
-	@%p11 bra 	BB21_19;
+	@%p11 bra 	BB22_19;
 
 	ld.shared.f64 	%fd35, [%rd1+512];
 	mul.f64 	%fd69, %fd68, %fd35;
 	st.shared.f64 	[%rd1], %fd69;
 
-BB21_19:
+BB22_19:
 	mov.f64 	%fd68, %fd69;
 	bar.sync 	0;
 
-BB21_20:
+BB22_20:
 	mov.f64 	%fd67, %fd68;
 	setp.gt.u32	%p12, %r6, 31;
-	@%p12 bra 	BB21_33;
+	@%p12 bra 	BB22_33;
 
 	setp.lt.u32	%p13, %r9, 64;
-	@%p13 bra 	BB21_23;
+	@%p13 bra 	BB22_23;
 
 	ld.volatile.shared.f64 	%fd36, [%rd1+256];
 	mul.f64 	%fd67, %fd67, %fd36;
 	st.volatile.shared.f64 	[%rd1], %fd67;
 
-BB21_23:
+BB22_23:
 	mov.f64 	%fd66, %fd67;
 	setp.lt.u32	%p14, %r9, 32;
-	@%p14 bra 	BB21_25;
+	@%p14 bra 	BB22_25;
 
 	ld.volatile.shared.f64 	%fd37, [%rd1+128];
 	mul.f64 	%fd66, %fd66, %fd37;
 	st.volatile.shared.f64 	[%rd1], %fd66;
 
-BB21_25:
+BB22_25:
 	mov.f64 	%fd65, %fd66;
 	setp.lt.u32	%p15, %r9, 16;
-	@%p15 bra 	BB21_27;
+	@%p15 bra 	BB22_27;
 
 	ld.volatile.shared.f64 	%fd38, [%rd1+64];
 	mul.f64 	%fd65, %fd65, %fd38;
 	st.volatile.shared.f64 	[%rd1], %fd65;
 
-BB21_27:
+BB22_27:
 	mov.f64 	%fd64, %fd65;
 	setp.lt.u32	%p16, %r9, 8;
-	@%p16 bra 	BB21_29;
+	@%p16 bra 	BB22_29;
 
 	ld.volatile.shared.f64 	%fd39, [%rd1+32];
 	mul.f64 	%fd64, %fd64, %fd39;
 	st.volatile.shared.f64 	[%rd1], %fd64;
 
-BB21_29:
+BB22_29:
 	mov.f64 	%fd63, %fd64;
 	setp.lt.u32	%p17, %r9, 4;
-	@%p17 bra 	BB21_31;
+	@%p17 bra 	BB22_31;
 
 	ld.volatile.shared.f64 	%fd40, [%rd1+16];
 	mul.f64 	%fd63, %fd63, %fd40;
 	st.volatile.shared.f64 	[%rd1], %fd63;
 
-BB21_31:
+BB22_31:
 	setp.lt.u32	%p18, %r9, 2;
-	@%p18 bra 	BB21_33;
+	@%p18 bra 	BB22_33;
 
 	ld.volatile.shared.f64 	%fd41, [%rd1+8];
 	mul.f64 	%fd42, %fd63, %fd41;
 	st.volatile.shared.f64 	[%rd1], %fd42;
 
-BB21_33:
+BB22_33:
 	setp.ne.s32	%p19, %r6, 0;
-	@%p19 bra 	BB21_35;
+	@%p19 bra 	BB22_35;
 
 	ld.shared.f64 	%fd43, [sdata];
 	cvta.to.global.u64 	%rd12, %rd3;
@@ -3440,7 +3514,7 @@ BB21_33:
 	add.s64 	%rd14, %rd12, %rd13;
 	st.global.f64 	[%rd14], %fd43;
 
-BB21_35:
+BB22_35:
 	ret;
 }
 
@@ -3464,17 +3538,17 @@ BB21_35:
 	ld.param.u32 	%r4, [reduce_row_mean_param_3];
 	mov.u32 	%r6, %ctaid.x;
 	setp.ge.u32	%p1, %r6, %r5;
-	@%p1 bra 	BB22_35;
+	@%p1 bra 	BB23_35;
 
 	mov.u32 	%r38, %tid.x;
 	mov.f64 	%fd74, 0d0000000000000000;
 	mov.f64 	%fd75, %fd74;
 	setp.ge.u32	%p2, %r38, %r4;
-	@%p2 bra 	BB22_4;
+	@%p2 bra 	BB23_4;
 
 	cvta.to.global.u64 	%rd3, %rd1;
 
-BB22_3:
+BB23_3:
 	mad.lo.s32 	%r8, %r6, %r4, %r38;
 	mul.wide.u32 	%rd4, %r8, 8;
 	add.s64 	%rd5, %rd3, %rd4;
@@ -3484,9 +3558,9 @@ BB22_3:
 	add.s32 	%r38, %r9, %r38;
 	setp.lt.u32	%p3, %r38, %r4;
 	mov.f64 	%fd74, %fd75;
-	@%p3 bra 	BB22_3;
+	@%p3 bra 	BB23_3;
 
-BB22_4:
+BB23_4:
 	mov.f64 	%fd72, %fd74;
 	mov.u32 	%r10, %tid.x;
 	mul.wide.u32 	%rd6, %r10, 8;
@@ -3496,130 +3570,130 @@ BB22_4:
 	bar.sync 	0;
 	mov.u32 	%r11, %ntid.x;
 	setp.lt.u32	%p4, %r11, 1024;
-	@%p4 bra 	BB22_8;
+	@%p4 bra 	BB23_8;
 
 	setp.gt.u32	%p5, %r10, 511;
 	mov.f64 	%fd73, %fd72;
-	@%p5 bra 	BB22_7;
+	@%p5 bra 	BB23_7;
 
 	ld.shared.f64 	%fd29, [%rd8+4096];
 	add.f64 	%fd73, %fd72, %fd29;
 	st.shared.f64 	[%rd8], %fd73;
 
-BB22_7:
+BB23_7:
 	mov.f64 	%fd72, %fd73;
 	bar.sync 	0;
 
-BB22_8:
+BB23_8:
 	mov.f64 	%fd70, %fd72;
 	setp.lt.u32	%p6, %r11, 512;
-	@%p6 bra 	BB22_12;
+	@%p6 bra 	BB23_12;
 
 	setp.gt.u32	%p7, %r10, 255;
 	mov.f64 	%fd71, %fd70;
-	@%p7 bra 	BB22_11;
+	@%p7 bra 	BB23_11;
 
 	ld.shared.f64 	%fd30, [%rd8+2048];
 	add.f64 	%fd71, %fd70, %fd30;
 	st.shared.f64 	[%rd8], %fd71;
 
-BB22_11:
+BB23_11:
 	mov.f64 	%fd70, %fd71;
 	bar.sync 	0;
 
-BB22_12:
+BB23_12:
 	mov.f64 	%fd68, %fd70;
 	setp.lt.u32	%p8, %r11, 256;
-	@%p8 bra 	BB22_16;
+	@%p8 bra 	BB23_16;
 
 	setp.gt.u32	%p9, %r10, 127;
 	mov.f64 	%fd69, %fd68;
-	@%p9 bra 	BB22_15;
+	@%p9 bra 	BB23

<TRUNCATED>

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

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/systemml/blob/628ffad1/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 62c0e0d..92a5546 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
@@ -112,6 +112,7 @@ import org.apache.sysml.runtime.matrix.operators.CMOperator;
 import org.apache.sysml.runtime.matrix.operators.LeftScalarOperator;
 import org.apache.sysml.runtime.matrix.operators.RightScalarOperator;
 import org.apache.sysml.runtime.matrix.operators.ScalarOperator;
+import org.apache.sysml.runtime.util.IndexRange;
 import org.apache.sysml.utils.GPUStatistics;
 import org.apache.sysml.utils.Statistics;
 
@@ -148,7 +149,7 @@ public class LibMatrixCUDA {
 
 	private static final Log LOG = LogFactory.getLog(LibMatrixCUDA.class.getName());
 
-    // Assume Compute Capability 3.0
+	// Assume Compute Capability 3.0
 	// MAX BLOCKS is 2^31 - 1 For compute capability > 3.0
 	// MAX_THREADS is 1024 For compute capability > 3.0
 	private static int _MAX_THREADS = -1;
@@ -163,7 +164,7 @@ public class LibMatrixCUDA {
 	static GPUContext gCtx throws DMLRuntimeException {
 			return GPUContext.gCtx;
 	}
-	*/
+	 */
 
 	/**
 	 * Utility function to get maximum number of threads supported by the active CUDA device.
@@ -336,7 +337,6 @@ public class LibMatrixCUDA {
 	 * @return a sparse matrix pointer
 	 * @throws DMLRuntimeException if error occurs
 	 */
-	@SuppressWarnings("unused")
 	private static CSRPointer getSparsePointer(GPUContext gCtx, MatrixObject input, String instName) throws DMLRuntimeException {
 		if(!isInSparseFormat(gCtx, input)) {
 			input.getGPUObject(gCtx).denseToSparse();
@@ -405,7 +405,7 @@ public class LibMatrixCUDA {
 		biasAdd(instName, tmp, biasPointer, outputPointer, rows, cols, (int)k1);
 
 		cudaFreeHelper(tmp);
-		*/
+		 */
 		LOG.trace("GPU : conv2dBiasAdd" + ", GPUContext=" + gCtx);
 		conv2d(gCtx, instName, image, filter, output, N, C, H, W, K, R, S, pad_h, pad_w, stride_h, stride_w, P, Q);
 		//cudaDeviceSynchronize;
@@ -413,7 +413,7 @@ public class LibMatrixCUDA {
 	}
 
 	public static void conv2d(GPUContext gCtx, String instName, MatrixObject image, MatrixObject filter, MatrixObject outputBlock, int N, int C, int H, int W,
-														int K, int R, int S, int pad_h, int pad_w, int stride_h, int stride_w, int P, int Q)
+			int K, int R, int S, int pad_h, int pad_w, int stride_h, int stride_w, int P, int Q)
 					throws DMLRuntimeException {
 		Pointer imagePointer = getDensePointer(gCtx, image, true, instName);
 		Pointer filterPointer = getDensePointer(gCtx, filter, true, instName);
@@ -448,10 +448,10 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if error
 	 */
 	public static void conv2d(GPUContext gCtx, String instName, Pointer image, Pointer filter, Pointer output, int N,
-														 int C, int H, int W, int K, int R, int S, int pad_h, int pad_w, int stride_h, int stride_w, int P, int Q)
+			int C, int H, int W, int K, int R, int S, int pad_h, int pad_w, int stride_h, int stride_w, int P, int Q)
 					throws DMLRuntimeException {
-        LOG.trace("GPU : conv2d" + ", GPUContext=" + gCtx);
-        cudnnFilterDescriptor filterDesc = null;
+		LOG.trace("GPU : conv2d" + ", GPUContext=" + gCtx);
+		cudnnFilterDescriptor filterDesc = null;
 		cudnnConvolutionDescriptor convDesc = null;
 		Pointer workSpace = null;
 		long sizeInBytes = 0;
@@ -480,7 +480,7 @@ public class LibMatrixCUDA {
 				// Also ensure that GPU has enough memory to allocate memory
 				long sizeInBytesArray[] = {0};
 				jcuda.jcudnn.JCudnn.cudnnGetConvolutionForwardAlgorithm(getCudnnHandle(gCtx), srcTensorDesc, filterDesc, convDesc, dstTensorDesc,
-								CONVOLUTION_PREFERENCE, sizeInBytesArray[0], algos);
+						CONVOLUTION_PREFERENCE, sizeInBytesArray[0], algos);
 				cudnnGetConvolutionForwardWorkspaceSize(getCudnnHandle(gCtx), srcTensorDesc, filterDesc, convDesc, dstTensorDesc, algos[0], sizeInBytesArray);
 				if (sizeInBytesArray[0] != 0)
 					workSpace = gCtx.allocate(sizeInBytesArray[0]);
@@ -494,10 +494,10 @@ public class LibMatrixCUDA {
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
 			if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
 			int status = cudnnConvolutionForward(getCudnnHandle(gCtx), one(),
-							srcTensorDesc, image,
-							filterDesc, filter,
-							convDesc, algo, workSpace, sizeInBytes, zero(),
-							dstTensorDesc, output);
+					srcTensorDesc, image,
+					filterDesc, filter,
+					convDesc, algo, workSpace, sizeInBytes, zero(),
+					dstTensorDesc, output);
 			if (GPUStatistics.DISPLAY_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CONVOLUTION_FORWARD_LIB, System.nanoTime() - t2);
 			if (status != cudnnStatus.CUDNN_STATUS_SUCCESS) {
@@ -564,8 +564,8 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void reluBackward(GPUContext gCtx, String instName, MatrixObject input, MatrixObject dout, MatrixObject outputBlock) throws DMLRuntimeException {
-        LOG.trace("GPU : reluBackward" + ", GPUContext=" + gCtx);
-        long rows = input.getNumRows();
+		LOG.trace("GPU : reluBackward" + ", GPUContext=" + gCtx);
+		long rows = input.getNumRows();
 		long cols = input.getNumColumns();
 		Pointer imagePointer = getDensePointer(gCtx, input, instName);
 		Pointer doutPointer = getDensePointer(gCtx, dout, instName);
@@ -574,8 +574,8 @@ public class LibMatrixCUDA {
 		long t1=0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("relu_backward",
-						ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(rows), toInt(cols)),
-						imagePointer, doutPointer, outputPointer, toInt(rows), toInt(cols));
+				ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(rows), toInt(cols)),
+				imagePointer, doutPointer, outputPointer, toInt(rows), toInt(cols));
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_BIAS_ADD_LIB, System.nanoTime() - t1);
 
 	}
@@ -593,8 +593,8 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void biasMultiply(GPUContext gCtx, String instName, MatrixObject input, MatrixObject bias, MatrixObject outputBlock) throws DMLRuntimeException {
-        LOG.trace("GPU : biasMultiply" + ", GPUContext=" + gCtx);
-        if(isInSparseFormat(gCtx, input)) {
+		LOG.trace("GPU : biasMultiply" + ", GPUContext=" + gCtx);
+		if(isInSparseFormat(gCtx, input)) {
 			input.getGPUObject(gCtx).sparseToDense(instName);
 		}
 		if(isInSparseFormat(gCtx, bias)) {
@@ -613,8 +613,8 @@ public class LibMatrixCUDA {
 		long t1 = 0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("bias_multiply",
-						ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(rows), toInt(cols)),
-						imagePointer, biasPointer, outputPointer, toInt(rows), toInt(cols), toInt(PQ));
+				ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(rows), toInt(cols)),
+				imagePointer, biasPointer, outputPointer, toInt(rows), toInt(cols), toInt(PQ));
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RELU_BACKWARD_KERNEL, System.nanoTime() - t1);
 
 	}
@@ -660,13 +660,13 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException
 	 */
 	private static void biasAdd(GPUContext gCtx, String instName, Pointer image, Pointer bias, Pointer output, int rows, int cols, int k) throws DMLRuntimeException {
-        LOG.trace("GPU : biasAdd" + ", GPUContext=" + gCtx);
-        int PQ = cols / k;
+		LOG.trace("GPU : biasAdd" + ", GPUContext=" + gCtx);
+		int PQ = cols / k;
 		long t1 = 0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("bias_add",
-						ExecutionConfig.getConfigForSimpleMatrixOperations(rows, cols),
-						image, bias, output, rows, cols, PQ);
+				ExecutionConfig.getConfigForSimpleMatrixOperations(rows, cols),
+				image, bias, output, rows, cols, PQ);
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RELU_BACKWARD_KERNEL, System.nanoTime() - t1);
 	}
 
@@ -701,8 +701,8 @@ public class LibMatrixCUDA {
 	public static void batchNormalizationForwardInference(GPUContext gCtx, String instName, MatrixObject image,
 			MatrixObject scale, MatrixObject bias, MatrixObject runningMean, MatrixObject runningVar,
 			MatrixObject ret, double epsilon) throws DMLRuntimeException {
-        LOG.trace("GPU : batchNormalizationForwardInference" + ", GPUContext=" + gCtx);
-        int mode = cudnnBatchNormMode.CUDNN_BATCHNORM_SPATIAL;
+		LOG.trace("GPU : batchNormalizationForwardInference" + ", GPUContext=" + gCtx);
+		int mode = cudnnBatchNormMode.CUDNN_BATCHNORM_SPATIAL;
 
 		int N = toInt(image.getNumRows());
 		int C = toInt(scale.getNumColumns());
@@ -724,8 +724,8 @@ public class LibMatrixCUDA {
 
 		checkStatus(cudnnBatchNormalizationForwardInference(getCudnnHandle(gCtx), mode, one(), zero(),
 				nCHWDescriptor, imagePtr, nCHWDescriptor, retPtr,
-			scaleTensorDesc, scalePtr, biasPtr,
-			runningMeanPtr, runningVarPtr, epsilon));
+				scaleTensorDesc, scalePtr, biasPtr,
+				runningMeanPtr, runningVarPtr, epsilon));
 	}
 
 	/**
@@ -747,8 +747,8 @@ public class LibMatrixCUDA {
 	public static void batchNormalizationForwardTraining(GPUContext gCtx, String instName, MatrixObject image,
 			MatrixObject scale,  MatrixObject bias, MatrixObject runningMean, MatrixObject runningVar,
 			MatrixObject ret, MatrixObject retRunningMean, MatrixObject retRunningVar, double epsilon, double exponentialAverageFactor) throws DMLRuntimeException {
-        LOG.trace("GPU : batchNormalizationForwardTraining" + ", GPUContext=" + gCtx);
-        int mode = cudnnBatchNormMode.CUDNN_BATCHNORM_SPATIAL;
+		LOG.trace("GPU : batchNormalizationForwardTraining" + ", GPUContext=" + gCtx);
+		int mode = cudnnBatchNormMode.CUDNN_BATCHNORM_SPATIAL;
 
 		int N = toInt(image.getNumRows());
 		int C = toInt(scale.getNumColumns());
@@ -777,8 +777,8 @@ public class LibMatrixCUDA {
 		// ignoring resultSaveMean and resultSaveVariance as it requires state management
 		checkStatus(cudnnBatchNormalizationForwardTraining(getCudnnHandle(gCtx), mode, one(), zero(),
 				nCHWDescriptor, imagePtr, nCHWDescriptor, retPtr,
-			scaleTensorDesc, scalePtr, biasPtr, exponentialAverageFactor,
-			retRunningMeanPtr, retRunningVarPtr, epsilon, new Pointer(), new Pointer()));
+				scaleTensorDesc, scalePtr, biasPtr, exponentialAverageFactor,
+				retRunningMeanPtr, retRunningVarPtr, epsilon, new Pointer(), new Pointer()));
 	}
 
 	/**
@@ -852,8 +852,8 @@ public class LibMatrixCUDA {
 	public static void batchNormalizationBackward(GPUContext gCtx, String instName, MatrixObject image, MatrixObject dout,
 			MatrixObject scale, MatrixObject ret, MatrixObject retScale, MatrixObject retBias,
 			double epsilon) throws DMLRuntimeException {
-        LOG.trace("GPU : batchNormalizationBackward" + ", GPUContext=" + gCtx);
-        int mode = cudnnBatchNormMode.CUDNN_BATCHNORM_SPATIAL;
+		LOG.trace("GPU : batchNormalizationBackward" + ", GPUContext=" + gCtx);
+		int mode = cudnnBatchNormMode.CUDNN_BATCHNORM_SPATIAL;
 
 		int N = toInt(image.getNumRows());
 		int C = toInt(scale.getNumColumns());
@@ -902,11 +902,11 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void conv2dBackwardFilter(GPUContext gCtx, String instName, MatrixObject image, MatrixObject dout,
-																					MatrixObject outputBlock, int N, int C, int H, int W, int K, int R,
-																					int S, int pad_h, int pad_w, int stride_h, int stride_w, int P,
-																					int Q) throws DMLRuntimeException {
-        LOG.trace("GPU : conv2dBackwardFilter" + ", GPUContext=" + gCtx);
-        cudnnFilterDescriptor dwDesc = null;
+			MatrixObject outputBlock, int N, int C, int H, int W, int K, int R,
+			int S, int pad_h, int pad_w, int stride_h, int stride_w, int P,
+			int Q) throws DMLRuntimeException {
+		LOG.trace("GPU : conv2dBackwardFilter" + ", GPUContext=" + gCtx);
+		cudnnFilterDescriptor dwDesc = null;
 		cudnnConvolutionDescriptor convDesc = null;
 
 		Pointer workSpace = null;
@@ -934,13 +934,13 @@ public class LibMatrixCUDA {
 
 			workSpace = new Pointer();
 			cudnnGetConvolutionBackwardFilterWorkspaceSize(getCudnnHandle(gCtx),
-							xTensorDesc, doutTensorDesc, convDesc, dwDesc, algo, sizeInBytesArray);
+					xTensorDesc, doutTensorDesc, convDesc, dwDesc, algo, sizeInBytesArray);
 			if (GPUStatistics.DISPLAY_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
 
 			if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
 			int status = cudnnConvolutionBackwardFilter(getCudnnHandle(gCtx), one(), xTensorDesc, imagePointer,
-							doutTensorDesc, doutPointer, convDesc, algo, workSpace, sizeInBytes, zero(), dwDesc, dwPointer);
+					doutTensorDesc, doutPointer, convDesc, algo, workSpace, sizeInBytes, zero(), dwDesc, dwPointer);
 			if (GPUStatistics.DISPLAY_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CONVOLUTION_BACKWARD_FILTER_LIB, System.nanoTime() - t2);
 
@@ -948,7 +948,7 @@ public class LibMatrixCUDA {
 				throw new DMLRuntimeException("Could not executed cudnnConvolutionBackwardFilter: " + jcuda.jcudnn.cudnnStatus.stringFor(status));
 			}
 		} catch (CudaException e) {
-				throw new DMLRuntimeException("Error in conv2d in GPUContext " + gCtx.toString() + " from Thread " + Thread.currentThread().toString(), e);
+			throw new DMLRuntimeException("Error in conv2d in GPUContext " + gCtx.toString() + " from Thread " + Thread.currentThread().toString(), e);
 		} finally {
 			long t3=0;
 			if (GPUStatistics.DISPLAY_STATISTICS) t3 = System.nanoTime();
@@ -989,11 +989,11 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void conv2dBackwardData(GPUContext gCtx, String instName, MatrixObject filter, MatrixObject dout,
-																				MatrixObject output, int N, int C, int H, int W, int K, int R,
-																				int S, int pad_h, int pad_w, int stride_h, int stride_w, int P,
-																				int Q) throws DMLRuntimeException {
-        LOG.trace("GPU : conv2dBackwardData" + ", GPUContext=" + gCtx);
-        cudnnFilterDescriptor wDesc = null;
+			MatrixObject output, int N, int C, int H, int W, int K, int R,
+			int S, int pad_h, int pad_w, int stride_h, int stride_w, int P,
+			int Q) throws DMLRuntimeException {
+		LOG.trace("GPU : conv2dBackwardData" + ", GPUContext=" + gCtx);
+		cudnnFilterDescriptor wDesc = null;
 		cudnnConvolutionDescriptor convDesc = null;
 
 		Pointer workSpace = null;
@@ -1020,12 +1020,12 @@ public class LibMatrixCUDA {
 			int algo = jcuda.jcudnn.cudnnConvolutionBwdDataAlgo.CUDNN_CONVOLUTION_BWD_DATA_ALGO_0;
 			workSpace = new Pointer();
 			cudnnGetConvolutionBackwardDataWorkspaceSize(getCudnnHandle(gCtx),
-							wDesc, dyDesc, convDesc, dxDesc, algo, sizeInBytesArray);
+					wDesc, dyDesc, convDesc, dxDesc, algo, sizeInBytesArray);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
 
 			if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
 			int status = cudnnConvolutionBackwardData(getCudnnHandle(gCtx), one(), wDesc, w,
-							dyDesc, dy, convDesc, algo, workSpace, sizeInBytes, zero(), dxDesc, dx);
+					dyDesc, dy, convDesc, algo, workSpace, sizeInBytes, zero(), dxDesc, dx);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CONVOLUTION_BACKWARD_DATA_LIB, System.nanoTime() - t2);
 
 			if(status != jcuda.jcudnn.cudnnStatus.CUDNN_STATUS_SUCCESS) {
@@ -1074,7 +1074,7 @@ public class LibMatrixCUDA {
 			MatrixObject outputBlock, int N, int C, int H, int W, int K, int R,
 			int S, int pad_h, int pad_w, int stride_h, int stride_w, int P,
 			int Q) throws DMLRuntimeException {
-        Pointer x = getDensePointer(gCtx, image, true, instName);
+		Pointer x = getDensePointer(gCtx, image, true, instName);
 		cudnnTensorDescriptor xDesc = allocateTensorDescriptor(gCtx, image, N, C, H, W);
 		performMaxpooling(gCtx, instName, x, xDesc, outputBlock, N, C, H, W, K, R, S, pad_h, pad_w, stride_h, stride_w, P, Q);
 	}
@@ -1083,8 +1083,8 @@ public class LibMatrixCUDA {
 			MatrixObject outputBlock, int N, int C, int H, int W, int K, int R,
 			int S, int pad_h, int pad_w, int stride_h, int stride_w, int P,
 			int Q) throws DMLRuntimeException {
-        LOG.trace("GPU : performMaxpooling" + ", GPUContext=" + gCtx);
-        Pointer y = getDensePointer(gCtx, outputBlock, true, instName);
+		LOG.trace("GPU : performMaxpooling" + ", GPUContext=" + gCtx);
+		Pointer y = getDensePointer(gCtx, outputBlock, true, instName);
 		cudnnPoolingDescriptor poolingDesc = null;
 
 		try {
@@ -1138,11 +1138,11 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void maxpoolingBackward(GPUContext gCtx, String instName, MatrixObject image, MatrixObject dout,
-																				MatrixObject outputBlock, int N, int C, int H, int W, int K, int R,
-																				int S, int pad_h, int pad_w, int stride_h, int stride_w, int P,
-																				int Q) throws DMLRuntimeException {
-        LOG.trace("GPU : maxpoolingBackward" + ", GPUContext=" + gCtx);
-        Pointer y = null;
+			MatrixObject outputBlock, int N, int C, int H, int W, int K, int R,
+			int S, int pad_h, int pad_w, int stride_h, int stride_w, int P,
+			int Q) throws DMLRuntimeException {
+		LOG.trace("GPU : maxpoolingBackward" + ", GPUContext=" + gCtx);
+		Pointer y = null;
 		cudnnPoolingDescriptor poolingDesc = null;
 
 		try {
@@ -1202,8 +1202,8 @@ public class LibMatrixCUDA {
 	private static void performCuDNNReLU(GPUContext gCtx, String instName, MatrixObject in, Pointer dstData, cudnnTensorDescriptor srcTensorDesc) throws DMLRuntimeException {
 		long t0=0;
 		try {
-            LOG.trace("GPU : performCuDNNReLU" + ", GPUContext=" + gCtx);
-            cudnnTensorDescriptor dstTensorDesc = srcTensorDesc;
+			LOG.trace("GPU : performCuDNNReLU" + ", GPUContext=" + gCtx);
+			cudnnTensorDescriptor dstTensorDesc = srcTensorDesc;
 
 			Pointer srcData = getDensePointer(gCtx, in, true, instName);
 			cudnnActivationDescriptor activationDescriptor = new cudnnActivationDescriptor();
@@ -1212,8 +1212,8 @@ public class LibMatrixCUDA {
 			cudnnSetActivationDescriptor(activationDescriptor, CUDNN_ACTIVATION_RELU, CUDNN_PROPAGATE_NAN, dummy);
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			cudnnActivationForward(getCudnnHandle(gCtx), activationDescriptor,
-							one(), srcTensorDesc, srcData,
-							zero(), dstTensorDesc, dstData);
+					one(), srcTensorDesc, srcData,
+					zero(), dstTensorDesc, dstData);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ACTIVATION_FORWARD_LIB, System.nanoTime() - t0);
 		} catch (CudaException e) {
 			throw new DMLRuntimeException("Error in conv2d in GPUContext " + gCtx.toString() + " from Thread " + Thread.currentThread().toString(), e);
@@ -1245,14 +1245,14 @@ public class LibMatrixCUDA {
 		long t0=0;
 		cudnnTensorDescriptor srcTensorDesc = in.getGPUObject(gCtx).getTensorDescriptor();
 		if(N*CHW >= numDoublesIn2GB ||  srcTensorDesc == null) {
-            LOG.trace("GPU : relu custom kernel" + ", GPUContext=" + gCtx);
-            // Invokes relu(double* A,  double* ret, int rlen, int clen)
+			LOG.trace("GPU : relu custom kernel" + ", GPUContext=" + gCtx);
+			// Invokes relu(double* A,  double* ret, int rlen, int clen)
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			Pointer dstData = getDensePointer(gCtx, output, instName);
 			Pointer srcData = getDensePointer(gCtx, in, instName); // TODO: FIXME: Add sparse kernel support for relu
 			getCudaKernels(gCtx).launchKernel("relu",
-							ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(N), toInt(CHW)),
-							srcData, dstData, toInt(N), toInt(CHW));
+					ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(N), toInt(CHW)),
+					srcData, dstData, toInt(N), toInt(CHW));
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RELU_KERNEL, System.nanoTime() - t0);
 		}
 		else {
@@ -1287,7 +1287,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void matmultTSMM(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject left, String outputName,
-																 boolean isLeftTransposed) throws DMLRuntimeException {
+			boolean isLeftTransposed) throws DMLRuntimeException {
 		LOG.trace("GPU : matmultTSMM" + ", GPUContext=" + gCtx);
 		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");
@@ -1303,7 +1303,7 @@ public class LibMatrixCUDA {
 		// Note: the dimensions are swapped
 		int m = toInt(isLeftTransposed ? left.getNumColumns() : left.getNumRows());
 		int k = toInt(isLeftTransposed ? left.getNumRows() : left.getNumColumns());
-				
+
 		// For dense TSMM, exploit cublasDsyrk(...) and call custom kernel to flip the matrix
 		MatrixObject output = getDenseMatrixOutputForGPUInstruction(ec, instName, outputName, m, m);	// Allocated the dense output matrix
 
@@ -1343,8 +1343,8 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void copyUpperToLowerTriangle(GPUContext gCtx, String instName, MatrixObject ret) throws DMLRuntimeException {
-        LOG.trace("GPU : copyUpperToLowerTriangle" + ", GPUContext=" + gCtx);
-        if(isInSparseFormat(gCtx, ret)) {
+		LOG.trace("GPU : copyUpperToLowerTriangle" + ", GPUContext=" + gCtx);
+		if(isInSparseFormat(gCtx, ret)) {
 			throw new DMLRuntimeException("Sparse GPU copyUpperToLowerTriangle is not implemented");
 		}
 		if(ret.getNumRows() != ret.getNumColumns()) {
@@ -1352,8 +1352,8 @@ public class LibMatrixCUDA {
 		}
 		int dim = toInt(ret.getNumRows());
 		getCudaKernels(gCtx).launchKernel("copy_u2l_dense",
-						ExecutionConfig.getConfigForSimpleMatrixOperations(dim, dim),
-						getDensePointer(gCtx, ret, instName), dim, dim*dim);
+				ExecutionConfig.getConfigForSimpleMatrixOperations(dim, dim),
+				getDensePointer(gCtx, ret, instName), dim, dim*dim);
 	}
 
 
@@ -1389,7 +1389,7 @@ public class LibMatrixCUDA {
 	 * @return output of matrix multiply
 	 */
 	public static MatrixObject matmult(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject left, MatrixObject right, String outputName,
-																		 boolean isLeftTransposed, boolean isRightTransposed) throws DMLRuntimeException {
+			boolean isLeftTransposed, boolean isRightTransposed) throws DMLRuntimeException {
 		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");
 		LOG.trace("GPU : matmult" + ", GPUContext=" + gCtx);
@@ -1400,7 +1400,7 @@ public class LibMatrixCUDA {
 		boolean bothSparse = left.getGPUObject(gCtx).isSparse() && right.getGPUObject(gCtx).isSparse();
 
 		MatrixObject output = ec.getMatrixObject(outputName);
-		
+
 		long outRLen = isLeftTransposed ? left.getNumColumns() : left.getNumRows();
 		long outCLen = isRightTransposed ? right.getNumRows() : right.getNumColumns();
 
@@ -1436,7 +1436,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void eitherSparseMatmult(GPUContext gCtx, String instName, MatrixObject output, MatrixObject left, MatrixObject right,
-																						boolean isLeftTransposed, boolean isRightTransposed) throws DMLRuntimeException {
+			boolean isLeftTransposed, boolean isRightTransposed) throws DMLRuntimeException {
 
 		int m = toInt(isLeftTransposed ? left.getNumColumns() : left.getNumRows()) ;
 		int n = toInt(isRightTransposed ? right.getNumRows() : right.getNumColumns());
@@ -1476,15 +1476,15 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void denseSparseMatmult(GPUContext gCtx, String instName, MatrixObject left, MatrixObject right, MatrixObject output,
-                                             boolean isLeftTransposed, boolean isRightTransposed, int m, int n, int k)
+			boolean isLeftTransposed, boolean isRightTransposed, int m, int n, int k)
 					throws DMLRuntimeException {
 		// right sparse, left dense
 		CSRPointer B = right.getGPUObject(gCtx).getJcudaSparseMatrixPtr();
 		Pointer ADense = getDensePointer(gCtx, left, instName);
 		if (B.isUltraSparse(k, n)){
-            LOG.trace(" GPU : Convert d M %*% sp M --> sp M %*% sp M)" + ", GPUContext=" + gCtx);
+			LOG.trace(" GPU : Convert d M %*% sp M --> sp M %*% sp M)" + ", GPUContext=" + gCtx);
 
-            // Convert left to CSR and do cuSparse matmul
+			// Convert left to CSR and do cuSparse matmul
 			int rowsA = (int)left.getNumRows();
 			int colsA = (int)left.getNumColumns();
 
@@ -1497,8 +1497,8 @@ public class LibMatrixCUDA {
 			CSRPointer A = GPUObject.columnMajorDenseToRowMajorSparse(gCtx, getCusparseHandle(gCtx), AT, rowsA, colsA);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DENSE_TO_SPARSE, System.nanoTime() - t1);
 
-			if (DMLScript.STATISTICS) GPUStatistics.cudaDenseToSparseTime.getAndAdd(System.nanoTime() - t0);
-			if (DMLScript.STATISTICS) GPUStatistics.cudaDenseToSparseCount.getAndAdd(1);
+			if (DMLScript.STATISTICS) GPUStatistics.cudaDenseToSparseTime.add(System.nanoTime() - t0);
+			if (DMLScript.STATISTICS) GPUStatistics.cudaDenseToSparseCount.add(1);
 			sparseSparseMatmult(gCtx, instName, A, B, output, isLeftTransposed, isRightTransposed, m, n, k);
 
 			if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
@@ -1507,7 +1507,7 @@ public class LibMatrixCUDA {
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDA_FREE, System.nanoTime() - t2, 2);
 
 		} else {
-            LOG.trace(" GPU : Convert d M %*% sp M --> d M %*% d M" + ", GPUContext=" + gCtx);
+			LOG.trace(" GPU : Convert d M %*% sp M --> d M %*% d M" + ", GPUContext=" + gCtx);
 			// Convert right to dense and do a cuBlas matmul
 			// BDenseTransposed is a column major matrix
 			// Note the arguments to denseDenseMatmult to accommodate for this.
@@ -1515,8 +1515,8 @@ public class LibMatrixCUDA {
 			if (DMLScript.STATISTICS) t0 = System.nanoTime();
 			Pointer BDenseTransposed = B.toColumnMajorDenseMatrix(getCusparseHandle(gCtx), getCublasHandle(gCtx), (int)right.getNumRows(), (int)right.getNumColumns());
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, System.nanoTime() - t0);
-			if (DMLScript.STATISTICS) GPUStatistics.cudaSparseToDenseTime.getAndAdd(System.nanoTime() - t0);
-			if (DMLScript.STATISTICS) GPUStatistics.cudaSparseToDenseCount.getAndAdd(System.nanoTime() - t0);
+			if (DMLScript.STATISTICS) GPUStatistics.cudaSparseToDenseTime.add(System.nanoTime() - t0);
+			if (DMLScript.STATISTICS) GPUStatistics.cudaSparseToDenseCount.add(System.nanoTime() - t0);
 
 			if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
 			boolean allocated = output.getGPUObject(gCtx).acquireDeviceModifyDense();	// To allocate the dense matrix
@@ -1550,7 +1550,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void sparseDenseMatmult(GPUContext gCtx, String instName, MatrixObject output, MatrixObject left, MatrixObject right,
-																					 boolean isLeftTransposed, boolean isRightTransposed, int m, int n, int k)
+			boolean isLeftTransposed, boolean isRightTransposed, int m, int n, int k)
 					throws DMLRuntimeException {
 		CSRPointer A = left.getGPUObject(gCtx).getJcudaSparseMatrixPtr();
 		Pointer BDense = getDensePointer(gCtx, right, instName);
@@ -1577,8 +1577,8 @@ public class LibMatrixCUDA {
 				CSRPointer B = GPUObject.columnMajorDenseToRowMajorSparse(gCtx, getCusparseHandle(gCtx), BT, rowsB, colsB);
 				if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DENSE_TO_SPARSE, System.nanoTime() - t1);
 
-				if (DMLScript.STATISTICS) GPUStatistics.cudaDenseToSparseTime.getAndAdd(System.nanoTime() - t0);
-				if (DMLScript.STATISTICS) GPUStatistics.cudaDenseToSparseCount.getAndAdd(1);
+				if (DMLScript.STATISTICS) GPUStatistics.cudaDenseToSparseTime.add(System.nanoTime() - t0);
+				if (DMLScript.STATISTICS) GPUStatistics.cudaDenseToSparseCount.add(1);
 
 				sparseSparseMatmult(gCtx, instName, A, B, output, isLeftTransposed, isRightTransposed, m, n, k);
 
@@ -1588,15 +1588,15 @@ public class LibMatrixCUDA {
 				if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDA_FREE, System.nanoTime() - t2, 2);
 
 			} else {
-                LOG.trace(" GPU : Convert sp M %*% d M --> d M %*% d M" + ", GPUContext=" + gCtx);
+				LOG.trace(" GPU : Convert sp M %*% d M --> d M %*% d M" + ", GPUContext=" + gCtx);
 				// Convert left to dense and do a cuBlas matmul
 				// ADenseTransposed is a column major matrix
 				// Note the arguments to denseDenseMatmult to accommodate for this.
 				if (DMLScript.STATISTICS) t0 = System.nanoTime();
 				Pointer ADenseTransposed = A.toColumnMajorDenseMatrix(getCusparseHandle(gCtx), getCublasHandle(gCtx), (int)left.getNumRows(), (int)left.getNumColumns());
 				if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, System.nanoTime() - t0);
-				if (DMLScript.STATISTICS) GPUStatistics.cudaSparseToDenseTime.getAndAdd(System.nanoTime() - t0);
-				if (DMLScript.STATISTICS) GPUStatistics.cudaSparseToDenseCount.getAndAdd(System.nanoTime() - t0);
+				if (DMLScript.STATISTICS) GPUStatistics.cudaSparseToDenseTime.add(System.nanoTime() - t0);
+				if (DMLScript.STATISTICS) GPUStatistics.cudaSparseToDenseCount.add(System.nanoTime() - t0);
 
 				if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
 				boolean allocated = output.getGPUObject(gCtx).acquireDeviceModifyDense();	// To allocate the dense matrix
@@ -1629,13 +1629,13 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void sparseMatrixDenseVectorMult(GPUContext gCtx, String instName, MatrixObject output, CSRPointer A, Pointer B_dense, boolean isATranposed,
-																										int m, int k) throws DMLRuntimeException {
-        LOG.trace("GPU : sp M %*% dense V" + ", GPUContext=" + gCtx);
-        int transA = CUSPARSE_OPERATION_NON_TRANSPOSE;
+			int m, int k) throws DMLRuntimeException {
+		LOG.trace("GPU : sp M %*% dense V" + ", GPUContext=" + gCtx);
+		int transA = CUSPARSE_OPERATION_NON_TRANSPOSE;
 		long size = m * Sizeof.DOUBLE;
 		if (isATranposed){
 			size = k * Sizeof.DOUBLE;
-            transA = CUSPARSE_OPERATION_TRANSPOSE;
+			transA = CUSPARSE_OPERATION_TRANSPOSE;
 		}
 		Pointer C_dense = gCtx.allocate(instName, (int)size);
 		long t1=0;
@@ -1662,8 +1662,8 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void bothSparseMatmult(GPUContext gCtx, String instName, MatrixObject output, MatrixObject left, MatrixObject right,
-																					boolean isLeftTransposed, boolean isRightTransposed) throws DMLRuntimeException {
-        int m = toInt(isLeftTransposed ? left.getNumColumns() : left.getNumRows()) ;
+			boolean isLeftTransposed, boolean isRightTransposed) throws DMLRuntimeException {
+		int m = toInt(isLeftTransposed ? left.getNumColumns() : left.getNumRows()) ;
 		int n = toInt(isRightTransposed ? right.getNumRows() : right.getNumColumns());
 		int k = toInt(isLeftTransposed ? left.getNumRows() :  left.getNumColumns());
 		int k1 = toInt(isRightTransposed ? right.getNumColumns() : right.getNumRows());
@@ -1701,7 +1701,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void sparseMatrixVectorMult(GPUContext gCtx, String instName, MatrixObject output, boolean isATranposed, int m, int n, int k,
-																							 CSRPointer A, CSRPointer B) throws DMLRuntimeException {
+			CSRPointer A, CSRPointer B) throws DMLRuntimeException {
 		long t0=0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 		Pointer BDenseVector = B.toColumnMajorDenseMatrix(getCusparseHandle(gCtx), getCublasHandle(gCtx), k, 1);
@@ -1726,11 +1726,11 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void sparseSparseMatmult(GPUContext gCtx, String instName, CSRPointer A, CSRPointer B, MatrixObject output,
-                                              boolean isLeftTransposed, boolean isRightTransposed, int m, int n, int k) throws DMLRuntimeException {
-        LOG.trace("GPU : sp M %*% sp M" + ", GPUContext=" + gCtx);
+			boolean isLeftTransposed, boolean isRightTransposed, int m, int n, int k) throws DMLRuntimeException {
+		LOG.trace("GPU : sp M %*% sp M" + ", GPUContext=" + gCtx);
 
-        int transA = isLeftTransposed ? CUSPARSE_OPERATION_TRANSPOSE : CUSPARSE_OPERATION_NON_TRANSPOSE;
-        int transB = isRightTransposed ? CUSPARSE_OPERATION_TRANSPOSE : CUSPARSE_OPERATION_NON_TRANSPOSE;
+		int transA = isLeftTransposed ? CUSPARSE_OPERATION_TRANSPOSE : CUSPARSE_OPERATION_NON_TRANSPOSE;
+		int transB = isRightTransposed ? CUSPARSE_OPERATION_TRANSPOSE : CUSPARSE_OPERATION_NON_TRANSPOSE;
 
 		long t0=0, t1=0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
@@ -1741,9 +1741,9 @@ public class LibMatrixCUDA {
 
 		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
 		cusparseDcsrgemm(getCusparseHandle(gCtx), transA, transB, m, n, k,
-						A.descr, (int)A.nnz, A.val, A.rowPtr, A.colInd,
-						B.descr, (int)B.nnz, B.val, B.rowPtr, B.colInd,
-						C.descr, C.val, C.rowPtr, C.colInd);
+				A.descr, (int)A.nnz, A.val, A.rowPtr, A.colInd,
+				B.descr, (int)B.nnz, B.val, B.rowPtr, B.colInd,
+				C.descr, C.val, C.rowPtr, C.colInd);
 		//cudaDeviceSynchronize;
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_MATRIX_SPARSE_MATRIX_LIB, System.nanoTime() - t1);
 	}
@@ -1762,7 +1762,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void denseDenseMatmult(GPUContext gCtx, String instName, MatrixObject output, MatrixObject left, MatrixObject right,
-																					boolean isLeftTransposed, boolean isRightTransposed) throws DMLRuntimeException {
+			boolean isLeftTransposed, boolean isRightTransposed) throws DMLRuntimeException {
 
 		Pointer leftPtr = getDensePointer(gCtx, left, instName);
 		Pointer rightPtr = getDensePointer(gCtx, right, instName);
@@ -1773,7 +1773,7 @@ public class LibMatrixCUDA {
 		int rightCols = toInt(right.getNumColumns());
 		Pointer C = getDensePointer(gCtx, output, instName);
 		denseDenseMatmult(gCtx, instName, C, leftRows, leftCols, rightRows, rightCols, isLeftTransposed, isRightTransposed,
-						leftPtr, rightPtr);
+				leftPtr, rightPtr);
 	}
 
 	/**
@@ -1799,9 +1799,9 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void denseDenseMatmult(GPUContext gCtx, String instName, Pointer output, int leftRows1, int leftCols1, int rightRows1,
-																			 int rightCols1, boolean isLeftTransposed1, boolean isRightTransposed1, Pointer leftPtr, Pointer rightPtr)
+			int rightCols1, boolean isLeftTransposed1, boolean isRightTransposed1, Pointer leftPtr, Pointer rightPtr)
 					throws DMLRuntimeException {
-        LOG.trace("GPU : d M %*% d M" + ", GPUContext=" + gCtx);
+		LOG.trace("GPU : d M %*% d M" + ", GPUContext=" + gCtx);
 
 		Pointer A = rightPtr;
 		Pointer B = leftPtr;
@@ -1892,7 +1892,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if {@link DMLRuntimeException} occurs
 	 */
 	public static void unaryAggregate(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, String output, AggregateUnaryOperator op)
-					throws DMLRuntimeException {
+			throws DMLRuntimeException {
 		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");
 		LOG.trace("GPU : unaryAggregate" + ", GPUContext=" + gCtx);
@@ -1955,12 +1955,12 @@ public class LibMatrixCUDA {
 		} else if (aggOp.increOp.fn instanceof Builtin) {
 			Builtin b = (Builtin)aggOp.increOp.fn;
 			switch(b.bFunc) {
-				case MAX: opIndex = OP_MAX; break;
-				case MIN: opIndex = OP_MIN; break;
-				case MAXINDEX: opIndex = OP_MAXINDEX; break;
-				case MININDEX: opIndex = OP_MININDEX;break;
-				default:
-					new DMLRuntimeException("Internal Error - Unsupported Builtin Function for Aggregate unary being done on GPU");
+			case MAX: opIndex = OP_MAX; break;
+			case MIN: opIndex = OP_MIN; break;
+			case MAXINDEX: opIndex = OP_MAXINDEX; break;
+			case MININDEX: opIndex = OP_MININDEX;break;
+			default:
+				new DMLRuntimeException("Internal Error - Unsupported Builtin Function for Aggregate unary being done on GPU");
 			}
 		} else {
 			throw new DMLRuntimeException("Internal Error - Aggregate operator has invalid Value function");
@@ -1980,7 +1980,7 @@ public class LibMatrixCUDA {
 			// throw new DMLRuntimeException("Internal Error - Not implemented");
 
 		}
-		
+
 		long outRLen = -1;
 		long outCLen = -1;
 		if (indexFn instanceof ReduceRow) { // COL{SUM, MAX...}
@@ -2004,210 +2004,210 @@ public class LibMatrixCUDA {
 
 		// For scalars, set the scalar output in the Execution Context object
 		switch (opIndex){
-			case OP_PLUS: {
-				switch(reductionDirection) {
-					case REDUCTION_ALL : {
-						double result = reduceAll(gCtx, instName, "reduce_sum", in, size);
-						ec.setScalarOutput(output, new DoubleObject(result));
-						break;
-					}
-					case REDUCTION_COL : {	// The names are a bit misleading, REDUCTION_COL refers to the direction (reduce all elements in a column)
-						reduceRow(gCtx, instName, "reduce_row_sum", in, out, rlen, clen);
-						break;
-					}
-					case REDUCTION_ROW : {
-						reduceCol(gCtx, instName, "reduce_col_sum", in, out, rlen, clen);
-						break;
-					}
-					case REDUCTION_DIAG :
-						throw new DMLRuntimeException("Internal Error - Row, Column and Diag summation not implemented yet");
-				}
+		case OP_PLUS: {
+			switch(reductionDirection) {
+			case REDUCTION_ALL : {
+				double result = reduceAll(gCtx, instName, "reduce_sum", in, size);
+				ec.setScalarOutput(output, new DoubleObject(result));
 				break;
 			}
-			case OP_PLUS_SQ : {
-				// Calculate the squares in a temporary object tmp
-				Pointer tmp = gCtx.allocate(instName, size * Sizeof.DOUBLE);
-
-				squareMatrix(gCtx, instName, in, tmp, rlen, clen);
-				// Then do the sum on the temporary object and free it
-				switch(reductionDirection) {
-					case REDUCTION_ALL : {
-						double result = reduceAll(gCtx, instName, "reduce_sum", tmp, size);
-						ec.setScalarOutput(output, new DoubleObject(result));
-						break;
-					}
-					case REDUCTION_COL : {	// The names are a bit misleading, REDUCTION_COL refers to the direction (reduce all elements in a column)
-						reduceRow(gCtx, instName, "reduce_row_sum", tmp, out, rlen, clen);
-						break;
-					}
-					case REDUCTION_ROW : {
-						reduceCol(gCtx, instName, "reduce_col_sum", tmp, out, rlen, clen);
-						break;
-					}
-					default:
-						throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for summation squared");
-				}
-				gCtx.cudaFreeHelper(instName, tmp);
+			case REDUCTION_COL : {	// The names are a bit misleading, REDUCTION_COL refers to the direction (reduce all elements in a column)
+				reduceRow(gCtx, instName, "reduce_row_sum", in, out, rlen, clen);
 				break;
 			}
-			case OP_MEAN:{
-				switch(reductionDirection) {
-					case REDUCTION_ALL: {
-						double result = reduceAll(gCtx, instName, "reduce_sum", in, size);
-						double mean = result / size;
-						ec.setScalarOutput(output, new DoubleObject(mean));
-						break;
-					}
-					case REDUCTION_COL: {
-						reduceRow(gCtx, instName, "reduce_row_mean", in, out, rlen, clen);
-						break;
-					}
-					case REDUCTION_ROW: {
-						reduceCol(gCtx, instName, "reduce_col_mean", in, out, rlen, clen);
-						break;
-					}
-					default:
-						throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for mean");
-				}
+			case REDUCTION_ROW : {
+				reduceCol(gCtx, instName, "reduce_col_sum", in, out, rlen, clen);
 				break;
 			}
-			case OP_MULTIPLY : {
-				switch (reductionDirection) {
-					case REDUCTION_ALL: {
-						double result = reduceAll(gCtx, instName, "reduce_prod", in, size);
-						ec.setScalarOutput(output, new DoubleObject(result));
-						break;
-					}
-					default:
-						throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for multiplication");
-				}
+			case REDUCTION_DIAG :
+				throw new DMLRuntimeException("Internal Error - Row, Column and Diag summation not implemented yet");
+			}
+			break;
+		}
+		case OP_PLUS_SQ : {
+			// Calculate the squares in a temporary object tmp
+			Pointer tmp = gCtx.allocate(instName, size * Sizeof.DOUBLE);
+
+			squareMatrix(gCtx, instName, in, tmp, rlen, clen);
+			// Then do the sum on the temporary object and free it
+			switch(reductionDirection) {
+			case REDUCTION_ALL : {
+				double result = reduceAll(gCtx, instName, "reduce_sum", tmp, size);
+				ec.setScalarOutput(output, new DoubleObject(result));
 				break;
 			}
-			case OP_MAX :{
-				switch(reductionDirection) {
-					case REDUCTION_ALL: {
-						double result = reduceAll(gCtx, instName, "reduce_max", in, size);
-						ec.setScalarOutput(output, new DoubleObject(result));
-						break;
-					}
-					case REDUCTION_COL: {
-						reduceRow(gCtx, instName, "reduce_row_max", in, out, rlen, clen);
-						break;
-					}
-					case REDUCTION_ROW: {
-						reduceCol(gCtx, instName, "reduce_col_max", in, out, rlen, clen);
-						break;
-					}
-					default:
-						throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for max");
-				}
+			case REDUCTION_COL : {	// The names are a bit misleading, REDUCTION_COL refers to the direction (reduce all elements in a column)
+				reduceRow(gCtx, instName, "reduce_row_sum", tmp, out, rlen, clen);
 				break;
 			}
-			case OP_MIN :{
-				switch(reductionDirection) {
-					case REDUCTION_ALL: {
-						double result = reduceAll(gCtx, instName, "reduce_min", in, size);
-						ec.setScalarOutput(output, new DoubleObject(result));
-						break;
-					}
-					case REDUCTION_COL: {
-						reduceRow(gCtx, instName, "reduce_row_min", in, out, rlen, clen);
-						break;
-					}
-					case REDUCTION_ROW: {
-						reduceCol(gCtx, instName, "reduce_col_min", in, out, rlen, clen);
-						break;
-					}
-					default:
-						throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for min");
-				}
+			case REDUCTION_ROW : {
+				reduceCol(gCtx, instName, "reduce_col_sum", tmp, out, rlen, clen);
+				break;
+			}
+			default:
+				throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for summation squared");
+			}
+			gCtx.cudaFreeHelper(instName, tmp);
+			break;
+		}
+		case OP_MEAN:{
+			switch(reductionDirection) {
+			case REDUCTION_ALL: {
+				double result = reduceAll(gCtx, instName, "reduce_sum", in, size);
+				double mean = result / size;
+				ec.setScalarOutput(output, new DoubleObject(mean));
+				break;
+			}
+			case REDUCTION_COL: {
+				reduceRow(gCtx, instName, "reduce_row_mean", in, out, rlen, clen);
+				break;
+			}
+			case REDUCTION_ROW: {
+				reduceCol(gCtx, instName, "reduce_col_mean", in, out, rlen, clen);
+				break;
+			}
+			default:
+				throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for mean");
+			}
+			break;
+		}
+		case OP_MULTIPLY : {
+			switch (reductionDirection) {
+			case REDUCTION_ALL: {
+				double result = reduceAll(gCtx, instName, "reduce_prod", in, size);
+				ec.setScalarOutput(output, new DoubleObject(result));
+				break;
+			}
+			default:
+				throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for multiplication");
+			}
+			break;
+		}
+		case OP_MAX :{
+			switch(reductionDirection) {
+			case REDUCTION_ALL: {
+				double result = reduceAll(gCtx, instName, "reduce_max", in, size);
+				ec.setScalarOutput(output, new DoubleObject(result));
+				break;
+			}
+			case REDUCTION_COL: {
+				reduceRow(gCtx, instName, "reduce_row_max", in, out, rlen, clen);
+				break;
+			}
+			case REDUCTION_ROW: {
+				reduceCol(gCtx, instName, "reduce_col_max", in, out, rlen, clen);
+				break;
+			}
+			default:
+				throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for max");
+			}
+			break;
+		}
+		case OP_MIN :{
+			switch(reductionDirection) {
+			case REDUCTION_ALL: {
+				double result = reduceAll(gCtx, instName, "reduce_min", in, size);
+				ec.setScalarOutput(output, new DoubleObject(result));
 				break;
 			}
-			case OP_VARIANCE : {
-				// Temporary GPU array for
-				Pointer tmp = gCtx.allocate(instName, size * Sizeof.DOUBLE);
-				Pointer tmp2 = gCtx.allocate(instName, size * Sizeof.DOUBLE);
+			case REDUCTION_COL: {
+				reduceRow(gCtx, instName, "reduce_row_min", in, out, rlen, clen);
+				break;
+			}
+			case REDUCTION_ROW: {
+				reduceCol(gCtx, instName, "reduce_col_min", in, out, rlen, clen);
+				break;
+			}
+			default:
+				throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for min");
+			}
+			break;
+		}
+		case OP_VARIANCE : {
+			// Temporary GPU array for
+			Pointer tmp = gCtx.allocate(instName, size * Sizeof.DOUBLE);
+			Pointer tmp2 = gCtx.allocate(instName, size * Sizeof.DOUBLE);
 
-				switch(reductionDirection) {
+			switch(reductionDirection) {
 
-					case REDUCTION_ALL: {
-						double result = reduceAll(gCtx, instName, "reduce_sum", in, size);
-						double mean = result / size;
+			case REDUCTION_ALL: {
+				double result = reduceAll(gCtx, instName, "reduce_sum", in, size);
+				double mean = result / size;
 
-						// Subtract mean from every element in the matrix
-						ScalarOperator minusOp = new RightScalarOperator(Minus.getMinusFnObject(), mean);
-						matrixScalarOp(gCtx, instName, in, mean, rlen, clen, tmp, minusOp);
+				// Subtract mean from every element in the matrix
+				ScalarOperator minusOp = new RightScalarOperator(Minus.getMinusFnObject(), mean);
+				matrixScalarOp(gCtx, instName, in, mean, rlen, clen, tmp, minusOp);
 
-						squareMatrix(gCtx, instName, tmp, tmp2, rlen, clen);
+				squareMatrix(gCtx, instName, tmp, tmp2, rlen, clen);
 
-						double result2 = reduceAll(gCtx, instName, "reduce_sum", tmp2, size);
-						double variance = result2 / (size - 1);
-						ec.setScalarOutput(output, new DoubleObject(variance));
+				double result2 = reduceAll(gCtx, instName, "reduce_sum", tmp2, size);
+				double variance = result2 / (size - 1);
+				ec.setScalarOutput(output, new DoubleObject(variance));
 
-						break;
-					}
-					case REDUCTION_COL: {
-						reduceRow(gCtx, instName, "reduce_row_mean", in, out, rlen, clen);
-						// Subtract the row-wise mean from every element in the matrix
-						BinaryOperator minusOp = new BinaryOperator(Minus.getMinusFnObject());
-						matrixMatrixOp(gCtx, instName, in, out, rlen, clen, VectorShape.NONE.code(), VectorShape.COLUMN.code(), tmp, minusOp);
+				break;
+			}
+			case REDUCTION_COL: {
+				reduceRow(gCtx, instName, "reduce_row_mean", in, out, rlen, clen);
+				// Subtract the row-wise mean from every element in the matrix
+				BinaryOperator minusOp = new BinaryOperator(Minus.getMinusFnObject());
+				matrixMatrixOp(gCtx, instName, in, out, rlen, clen, VectorShape.NONE.code(), VectorShape.COLUMN.code(), tmp, minusOp);
 
-						squareMatrix(gCtx, instName, tmp, tmp2, rlen, clen);
+				squareMatrix(gCtx, instName, tmp, tmp2, rlen, clen);
 
-						Pointer tmpRow = gCtx.allocate(instName, rlen * Sizeof.DOUBLE);
-						reduceRow(gCtx, instName, "reduce_row_sum", tmp2, tmpRow, rlen, clen);
+				Pointer tmpRow = gCtx.allocate(instName, rlen * Sizeof.DOUBLE);
+				reduceRow(gCtx, instName, "reduce_row_sum", tmp2, tmpRow, rlen, clen);
 
-						ScalarOperator divideOp = new RightScalarOperator(Divide.getDivideFnObject(), clen - 1);
-						matrixScalarOp(gCtx, instName, tmpRow, clen - 1, rlen, 1, out, divideOp);
+				ScalarOperator divideOp = new RightScalarOperator(Divide.getDivideFnObject(), clen - 1);
+				matrixScalarOp(gCtx, instName, tmpRow, clen - 1, rlen, 1, out, divideOp);
 
-						gCtx.cudaFreeHelper(instName, tmpRow);
+				gCtx.cudaFreeHelper(instName, tmpRow);
 
-						break;
-					}
-					case REDUCTION_ROW: {
-						reduceCol(gCtx, instName, "reduce_col_mean", in, out, rlen, clen);
-						// Subtract the columns-wise mean from every element in the matrix
-						BinaryOperator minusOp = new BinaryOperator(Minus.getMinusFnObject());
-						matrixMatrixOp(gCtx, instName, in, out, rlen, clen, VectorShape.NONE.code(), VectorShape.ROW.code(), tmp, minusOp);
+				break;
+			}
+			case REDUCTION_ROW: {
+				reduceCol(gCtx, instName, "reduce_col_mean", in, out, rlen, clen);
+				// Subtract the columns-wise mean from every element in the matrix
+				BinaryOperator minusOp = new BinaryOperator(Minus.getMinusFnObject());
+				matrixMatrixOp(gCtx, instName, in, out, rlen, clen, VectorShape.NONE.code(), VectorShape.ROW.code(), tmp, minusOp);
 
-						squareMatrix(gCtx, instName, tmp, tmp2, rlen, clen);
+				squareMatrix(gCtx, instName, tmp, tmp2, rlen, clen);
 
-						Pointer tmpCol = gCtx.allocate(instName, clen * Sizeof.DOUBLE);
-						reduceCol(gCtx, instName, "reduce_col_sum", tmp2, tmpCol, rlen, clen);
+				Pointer tmpCol = gCtx.allocate(instName, clen * Sizeof.DOUBLE);
+				reduceCol(gCtx, instName, "reduce_col_sum", tmp2, tmpCol, rlen, clen);
 
-						ScalarOperator divideOp = new RightScalarOperator(Divide.getDivideFnObject(), rlen - 1);
-						matrixScalarOp(gCtx, instName, tmpCol, rlen - 1, 1, clen, out, divideOp);
+				ScalarOperator divideOp = new RightScalarOperator(Divide.getDivideFnObject(), rlen - 1);
+				matrixScalarOp(gCtx, instName, tmpCol, rlen - 1, 1, clen, out, divideOp);
 
-						gCtx.cudaFreeHelper(instName, tmpCol);
+				gCtx.cudaFreeHelper(instName, tmpCol);
 
-						break;
-					}
-					default:
-						throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for variance");
-				}
-				gCtx.cudaFreeHelper(instName, tmp);
-				gCtx.cudaFreeHelper(instName, tmp2);
 				break;
 			}
-			case OP_MAXINDEX : {
-				switch(reductionDirection) {
-					case REDUCTION_COL:
-						throw new DMLRuntimeException("Internal Error - Column maxindex of matrix not implemented yet for GPU ");
-					default:
-						throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for maxindex");
-				}
-				// break;
+			default:
+				throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for variance");
 			}
-			case OP_MININDEX : {
-				switch(reductionDirection) {
-					case REDUCTION_COL:
-						throw new DMLRuntimeException("Internal Error - Column minindex of matrix not implemented yet for GPU ");
-					default:
-						throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for minindex");
-				}
-				// break;
+			gCtx.cudaFreeHelper(instName, tmp);
+			gCtx.cudaFreeHelper(instName, tmp2);
+			break;
+		}
+		case OP_MAXINDEX : {
+			switch(reductionDirection) {
+			case REDUCTION_COL:
+				throw new DMLRuntimeException("Internal Error - Column maxindex of matrix not implemented yet for GPU ");
+			default:
+				throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for maxindex");
+			}
+			// break;
+		}
+		case OP_MININDEX : {
+			switch(reductionDirection) {
+			case REDUCTION_COL:
+				throw new DMLRuntimeException("Internal Error - Column minindex of matrix not implemented yet for GPU ");
+			default:
+				throw new DMLRuntimeException("Internal Error - Unsupported reduction direction for minindex");
 			}
-			default : throw new DMLRuntimeException("Internal Error - Invalid GPU Unary aggregate function!");
+			// break;
+		}
+		default : throw new DMLRuntimeException("Internal Error - Invalid GPU Unary aggregate function!");
 		}
 	}
 
@@ -2222,7 +2222,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if error
 	 */
 	private static void squareMatrix(GPUContext gCtx, String instName, Pointer in, Pointer out, int rlen, int clen) throws DMLRuntimeException {
-        ScalarOperator power2op = new RightScalarOperator(Power.getPowerFnObject(), 2);
+		ScalarOperator power2op = new RightScalarOperator(Power.getPowerFnObject(), 2);
 		matrixScalarOp(gCtx, instName, in, 2, rlen, clen, out, power2op);
 	}
 
@@ -2236,9 +2236,9 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static double reduceAll(GPUContext gCtx, String instName, String kernelFunction, Pointer in, int n) throws DMLRuntimeException {
-        LOG.trace("GPU : reduceAll for " + kernelFunction + ", GPUContext=" + gCtx);
+		LOG.trace("GPU : reduceAll for " + kernelFunction + ", GPUContext=" + gCtx);
 
-        int[] tmp = getKernelParamsForReduceAll(gCtx, n);
+		int[] tmp = getKernelParamsForReduceAll(gCtx, n);
 		int blocks = tmp[0], threads = tmp[1], sharedMem = tmp[2];
 
 		Pointer tempOut = gCtx.allocate(instName, n * Sizeof.DOUBLE);
@@ -2256,7 +2256,7 @@ public class LibMatrixCUDA {
 			blocks = tmp[0]; threads = tmp[1]; sharedMem = tmp[2];
 			if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
 			getCudaKernels(gCtx).launchKernel(kernelFunction, new ExecutionConfig(blocks, threads, sharedMem),
-							tempOut, tempOut, s);
+					tempOut, tempOut, s);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_ALL_KERNEL, System.nanoTime() - t2);
 			s = (s + (threads*2-1)) / (threads*2);
 		}
@@ -2282,15 +2282,15 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void reduceRow(GPUContext gCtx, String instName, String kernelFunction, Pointer in, Pointer out, int rows, int cols) throws DMLRuntimeException {
-        LOG.trace("GPU : reduceRow for " + kernelFunction + ", GPUContext=" + gCtx);
+		LOG.trace("GPU : reduceRow for " + kernelFunction + ", GPUContext=" + gCtx);
 
-        int[] tmp = getKernelParamsForReduceByRow(gCtx, rows, cols);
+		int[] tmp = getKernelParamsForReduceByRow(gCtx, rows, cols);
 		int blocks = tmp[0], threads = tmp[1], sharedMem = tmp[2];
 
 		long t0=0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel(kernelFunction, new ExecutionConfig(blocks, threads, sharedMem),
-						in, out, rows, cols);
+				in, out, rows, cols);
 		//cudaDeviceSynchronize;
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_ROW_KERNEL, System.nanoTime() - t0);
 
@@ -2308,15 +2308,15 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void reduceCol(GPUContext gCtx, String instName, String kernelFunction, Pointer in, Pointer out, int rows, int cols) throws DMLRuntimeException {
-        LOG.trace("GPU : reduceCol for " + kernelFunction + ", GPUContext=" + gCtx);
+		LOG.trace("GPU : reduceCol for " + kernelFunction + ", GPUContext=" + gCtx);
 
-        int[] tmp = getKernelParamsForReduceByCol(gCtx, rows, cols);
+		int[] tmp = getKernelParamsForReduceByCol(gCtx, rows, cols);
 		int blocks = tmp[0], threads = tmp[1], sharedMem = tmp[2];
 
 		long t0=0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel(kernelFunction, new ExecutionConfig(blocks, threads, sharedMem),
-						in, out, rows, cols);
+				in, out, rows, cols);
 		//cudaDeviceSynchronize;
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_COL_KERNEL, System.nanoTime() - t0);
 	}
@@ -2448,51 +2448,51 @@ public class LibMatrixCUDA {
 			throw new DMLRuntimeException("GPU : Invalid internal state, the GPUContext set with the ExecutionContext is not the same used to run this LibMatrixCUDA function");
 		double constant = op.getConstant();
 		LOG.trace("GPU : matrixScalarArithmetic, scalar: " + constant + ", GPUContext=" + gCtx);
-		
+
 		int outRLen = isInputTransposed ? (int) in.getNumColumns() : (int) in.getNumRows();
 		int outCLen = isInputTransposed ? (int) in.getNumRows() : (int) in.getNumColumns();
-		
+
 		//boolean isCUDALibAvailable = (op.fn instanceof Multiply
 		//		|| (op.fn instanceof Divide && op instanceof RightScalarOperator && constant != 0)) && !isSparseAndEmpty(gCtx, in);
 		//if(!isCUDALibAvailable) {
-			if(constant == 0) {
-                if(op.fn instanceof Plus || (op.fn instanceof Minus && op instanceof RightScalarOperator) || op.fn instanceof Or) {
-					deviceCopy(ec, gCtx, instName, in, outputName, isInputTransposed);
-				}
-				else if(op.fn instanceof Multiply || op.fn instanceof And) {
-					setOutputToConstant(ec, gCtx, instName, 0.0, outputName, outRLen, outCLen);
-				}
-				else if(op.fn instanceof Power) {
-					setOutputToConstant(ec, gCtx, instName, 1.0, outputName, outRLen, outCLen);
-				}
-                // TODO:
-                // x/0.0 is either +Infinity or -Infinity according to Java.
-                // In the context of a matrix, different elements of the matrix
-                // could have different values.
-                // If the IEEE 754 standard defines otherwise, this logic needs
-                // to be re-enabled and the Java computation logic for divide by zero
-                // needs to be revisited
-                //else if(op.fn instanceof Divide && isSparseAndEmpty(gCtx, in)) {
-                //	setOutputToConstant(ec, gCtx, instName, Double.NaN, outputName);
-                //}
-                //else if(op.fn instanceof Divide) {
-                //	//For division, IEEE 754 defines x/0.0 as INFINITY and 0.0/0.0 as NaN.
-                //	compareAndSet(ec, gCtx, instName, in, outputName, 0.0, 1e-6, Double.NaN, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY);
-                //}
-                else {
-					// TODO: Potential to optimize
-					matrixScalarOp(ec, gCtx, instName, in, outputName, isInputTransposed, op);
-				}
+		if(constant == 0) {
+			if(op.fn instanceof Plus || (op.fn instanceof Minus && op instanceof RightScalarOperator) || op.fn instanceof Or) {
+				deviceCopy(ec, gCtx, instName, in, outputName, isInputTransposed);
 			}
-			else if(constant == 1.0 && op.fn instanceof Or) {
-				setOutputToConstant(ec, gCtx, instName, 1.0, outputName, outRLen, outCLen);
+			else if(op.fn instanceof Multiply || op.fn instanceof And) {
+				setOutputToConstant(ec, gCtx, instName, 0.0, outputName, outRLen, outCLen);
 			}
-			else if(constant == 1.0 && (op.fn instanceof And || op.fn instanceof Power)) {
-				deviceCopy(ec, gCtx, instName, in, outputName, isInputTransposed);
+			else if(op.fn instanceof Power) {
+				setOutputToConstant(ec, gCtx, instName, 1.0, outputName, outRLen, outCLen);
 			}
+			// TODO:
+				// x/0.0 is either +Infinity or -Infinity according to Java.
+			// In the context of a matrix, different elements of the matrix
+			// could have different values.
+			// If the IEEE 754 standard defines otherwise, this logic needs
+			// to be re-enabled and the Java computation logic for divide by zero
+			// needs to be revisited
+			//else if(op.fn instanceof Divide && isSparseAndEmpty(gCtx, in)) {
+			//	setOutputToConstant(ec, gCtx, instName, Double.NaN, outputName);
+			//}
+			//else if(op.fn instanceof Divide) {
+			//	//For division, IEEE 754 defines x/0.0 as INFINITY and 0.0/0.0 as NaN.
+			//	compareAndSet(ec, gCtx, instName, in, outputName, 0.0, 1e-6, Double.NaN, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY);
+			//}
 			else {
+				// TODO: Potential to optimize
 				matrixScalarOp(ec, gCtx, instName, in, outputName, isInputTransposed, op);
 			}
+		}
+		else if(constant == 1.0 && op.fn instanceof Or) {
+			setOutputToConstant(ec, gCtx, instName, 1.0, outputName, outRLen, outCLen);
+		}
+		else if(constant == 1.0 && (op.fn instanceof And || op.fn instanceof Power)) {
+			deviceCopy(ec, gCtx, instName, in, outputName, isInputTransposed);
+		}
+		else {
+			matrixScalarOp(ec, gCtx, instName, in, outputName, isInputTransposed, op);
+		}
 		// }
 		//else {
 		//	double alpha = 0;
@@ -2506,8 +2506,8 @@ public class LibMatrixCUDA {
 		//		throw new DMLRuntimeException("Unsupported op");
 		//	}
 
-			// TODO: Performance optimization: Call cublasDaxpy if(in.getNumRows() == 1 || in.getNumColumns() == 1)
-			// C = alpha* op( A ) + beta* op ( B )
+		// TODO: Performance optimization: Call cublasDaxpy if(in.getNumRows() == 1 || in.getNumColumns() == 1)
+		// C = alpha* op( A ) + beta* op ( B )
 		//	dgeam(ec, gCtx, instName, in, in, outputName, isInputTransposed, isInputTransposed, alpha, 0.0);
 		//}
 	}
@@ -2563,15 +2563,15 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void matrixMatrixArithmetic(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2,
-																						String outputName, boolean isLeftTransposed, boolean isRightTransposed, BinaryOperator op) throws DMLRuntimeException {
+			String outputName, boolean isLeftTransposed, boolean isRightTransposed, BinaryOperator op) throws DMLRuntimeException {
 		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");
 		boolean isCUDALibAvailable = (op.fn instanceof Plus || op.fn instanceof Minus) && !isSparseAndEmpty(gCtx, in1) && !isSparseAndEmpty(gCtx, in2) && !isVector(in1) && !isVector(in2);
 		if(!isCUDALibAvailable) {
-            matrixMatrixOp(ec, gCtx, instName, in1, in2, outputName, isLeftTransposed, isRightTransposed, op);
+			matrixMatrixOp(ec, gCtx, instName, in1, in2, outputName, isLeftTransposed, isRightTransposed, op);
 		}
 		else {
-            double alpha;
+			double alpha;
 			double beta;
 			if(op.fn instanceof Plus) {
 				alpha = 1.0;
@@ -2602,7 +2602,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void matrixScalarOp(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in, String outputName, boolean isInputTransposed,
-																		 ScalarOperator op) throws DMLRuntimeException {
+			ScalarOperator op) throws DMLRuntimeException {
 		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");
 		if(isInputTransposed)
@@ -2638,11 +2638,11 @@ public class LibMatrixCUDA {
 		int isLeftScalar = (op instanceof LeftScalarOperator) ? 1 : 0;
 		int size = rlenA * clenA;
 		long t0=0;
-        if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
-            getCudaKernels(gCtx).launchKernel("matrix_scalar_op",
-                            ExecutionConfig.getConfigForSimpleVectorOperations(size),
-                            a, scalar, c, size, getBinaryOp(op.fn), isLeftScalar);
-            if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MATRIX_SCALAR_OP_KERNEL, System.nanoTime() - t0);
+		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		getCudaKernels(gCtx).launchKernel("matrix_scalar_op",
+				ExecutionConfig.getConfigForSimpleVectorOperations(size),
+				a, scalar, c, size, getBinaryOp(op.fn), isLeftScalar);
+		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MATRIX_SCALAR_OP_KERNEL, System.nanoTime() - t0);
 	}
 
 	/**
@@ -2660,7 +2660,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void matrixMatrixOp(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2,
-																		 String outputName, boolean isLeftTransposed, boolean isRightTransposed, BinaryOperator op) throws DMLRuntimeException {
+			String outputName, boolean isLeftTransposed, boolean isRightTransposed, BinaryOperator op) throws DMLRuntimeException {
 		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");
 		boolean isEmpty1 = isSparseAndEmpty(gCtx, in1);
@@ -2671,7 +2671,7 @@ public class LibMatrixCUDA {
 		int clenB = toInt(in2.getNumColumns());
 		int vecStatusA = getVectorStatus(rlenA, clenA).code();
 		int vecStatusB = getVectorStatus(rlenB, clenB).code();
-		
+
 		if(isLeftTransposed || isRightTransposed) {
 			throw new DMLRuntimeException("Unsupported operator: GPU transposed binary op " + isLeftTransposed + " " + isRightTransposed);
 		}
@@ -2742,8 +2742,8 @@ public class LibMatrixCUDA {
 		long t0=0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("matrix_matrix_cellwise_op",
-            ExecutionConfig.getConfigForSimpleMatrixOperations(maxRlen, maxClen),
-						a, b, c, maxRlen, maxClen, vecStatusA, vecStatusB, getBinaryOp(op.fn));
+				ExecutionConfig.getConfigForSimpleMatrixOperations(maxRlen, maxClen),
+				a, b, c, maxRlen, maxClen, vecStatusA, vecStatusB, getBinaryOp(op.fn));
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MATRIX_MATRIX_CELLWISE_OP_KERNEL, System.nanoTime() - t0);
 	}
 
@@ -2816,7 +2816,7 @@ public class LibMatrixCUDA {
 
 	@SuppressWarnings("unused")
 	private static void compareAndSet(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in, String outputName, double compareVal,  double tolerance,
-																		double ifEqualsVal, double ifLessThanVal, double ifGreaterThanVal) throws DMLRuntimeException {
+			double ifEqualsVal, double ifLessThanVal, double ifGreaterThanVal) throws DMLRuntimeException {
 		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");
 		Pointer A = getDensePointer(gCtx, in, instName); // TODO: FIXME: Implement sparse kernel
@@ -2825,14 +2825,14 @@ public class LibMatrixCUDA {
 		int clen = toInt(out.getNumColumns());
 		getDenseMatrixOutputForGPUInstruction(ec, instName, outputName, rlen, clen);	// Allocated the dense output matrix
 		Pointer ret = getDensePointer(gCtx, out, instName);
-		
+
 		// out.getMatrixCharacteristics().setNonZeros(rlen*clen);
 		// compareAndSet(double* A,  double* ret, int rlen, int clen, double compareVal, double ifEqualsVal, double ifNotEqualsVal)
 		long t0=0;
 		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("compare_and_set",
-						ExecutionConfig.getConfigForSimpleMatrixOperations(rlen, clen),
-						A, ret, rlen, clen, compareVal, tolerance, ifEqualsVal, ifLessThanVal, ifGreaterThanVal);
+				ExecutionConfig.getConfigForSimpleMatrixOperations(rlen, clen),
+				A, ret, rlen, clen, compareVal, tolerance, ifEqualsVal, ifLessThanVal, ifGreaterThanVal);
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_COMPARE_AND_SET_KERNEL, System.nanoTime() - t0);
 	}
 
@@ -2935,7 +2935,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	private static void dgeam(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2, String outputName,
-														boolean isLeftTransposed, boolean isRightTransposed, double alpha, double beta) throws DMLRuntimeException {
+			boolean isLeftTransposed, boolean isRightTransposed, double alpha, double beta) throws DMLRuntimeException {
 		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");
 		LOG.trace("GPU : dgeam" + ", GPUContext=" + gCtx);
@@ -2944,7 +2944,7 @@ public class LibMatrixCUDA {
 		Pointer betaPtr = pointerTo(beta);
 		int transa = isLeftTransposed ? CUBLAS_OP_T : CUBLAS_OP_N;
 		int transb = isRightTransposed ? CUBLAS_OP_T : CUBLAS_OP_N;
-		
+
 		long outRLen = isLeftTransposed ? in1.getNumColumns() : in1.getNumRows();
 		long outCLen = isLeftTransposed ? in1.getNumRows() : in1.getNumColumns();
 
@@ -3086,6 +3086,70 @@ public class LibMatrixCUDA {
 	//**************** Matrix Manipulation Functions *********************/
 	//********************************************************************/
 
+	/**
+	 * Method to perform rangeReIndex operation for a given lower and upper bounds in row and column dimensions.
+	 *  
+	 * @param ec current execution context
+	 * @param gCtx current gpu context
+	 * @param instName name of the instruction for maintaining statistics
+	 * @param in1 input matrix object
+	 * @param ixrange index range (0-based)
+	 * @param outputName output matrix object
+	 * @throws DMLRuntimeException if error occurs
+	 */
+	public static void sliceOperations(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1,
+			IndexRange ixrange, String outputName) throws DMLRuntimeException {
+		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");
+		LOG.trace("GPU : sliceOperations" + ", GPUContext=" + gCtx);
+
+		int rl = (int) ixrange.rowStart;
+		int ru = (int) ixrange.rowEnd;
+		int cl = (int) ixrange.colStart;
+		int cu = (int) ixrange.colEnd;
+		if (rl < 0 || rl >= in1.getNumRows() || ru < rl || ru >= in1.getNumRows() || cl < 0
+				|| cu >= in1.getNumColumns() || cu < cl || cu >= in1.getNumColumns()) {
+			throw new DMLRuntimeException("Invalid values for matrix indexing: [" + (rl + 1) + ":" + (ru + 1) + ","
+					+ (cl + 1) + ":" + (cu + 1) + "] " + "must be within matrix dimensions [" + in1.getNumRows() + ","
+					+ in1.getNumColumns() + "]");
+		}
+
+		int len1 = toInt(in1.getNumColumns());
+		int len2 = toInt(ec.getMatrixObject(outputName).getNumColumns());
+		if(isInSparseFormat(gCtx, in1)) {
+			// Input in1 is in sparse format and output is in dense format
+			MatrixObject out = getDenseMatrixOutputForGPUInstruction(ec, instName, outputName, ru - rl + 1, cu - cl + 1);
+			CSRPointer inPointer = getSparsePointer(gCtx, in1, instName);
+			Pointer outPointer = getDensePointer(gCtx, out, instName);
+			int size = ru - rl + 1;
+			long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			// Performs a slice operation where the input matrix is sparse and the output matrix is dense.
+			// This function avoids unnecessary sparse to dense conversion of the input matrix.
+			// We can generalize this later to output sparse matrix.
+			getCudaKernels(gCtx).launchKernel("slice_sparse_dense", ExecutionConfig.getConfigForSimpleVectorOperations(size),
+					inPointer.val, inPointer.rowPtr, inPointer.colInd, outPointer, rl, ru, cl, cu);
+			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RIX_SPARSE_DENSE_OP, System.nanoTime() - t0);
+		}
+		else {
+			// Input in1 is in dense format (see inPointer)
+			MatrixObject out = getDenseMatrixOutputForGPUInstruction(ec, instName, outputName, ru - rl + 1, cu - cl + 1);
+
+			Pointer inPointer = getDensePointer(gCtx, in1, instName);
+			Pointer outPointer = getDensePointer(gCtx, out, instName);
+			long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			if (len1 == len2) {
+				cudaMemcpy(outPointer, inPointer.withByteOffset(rl * len1 * Sizeof.DOUBLE), (ru - rl + 1) * len1
+						* Sizeof.DOUBLE, cudaMemcpyDeviceToDevice);
+			} else {
+				for (int i = rl, ix1 = rl * len1 + cl, ix2 = 0; i <= ru; i++, ix1 += len1, ix2 += len2) {
+					cudaMemcpy(outPointer.withByteOffset(ix2 * Sizeof.DOUBLE),
+							inPointer.withByteOffset(ix1 * Sizeof.DOUBLE), len2 * Sizeof.DOUBLE, cudaMemcpyDeviceToDevice);
+				}
+			}
+			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RIX_DENSE_OP, System.nanoTime() - t0);
+		}
+	}
 
 	public static void cbind(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2, String outputName) throws DMLRuntimeException {
 		if (ec.getGPUContext(0) != gCtx)
@@ -3093,7 +3157,7 @@ public class LibMatrixCUDA {
 		LOG.trace("GPU : cbind" + ", GPUContext=" + gCtx);
 
 		long t1 = 0;
-		
+
 		long rowsA = toInt(in1.getNumRows());
 		long colsA = toInt(in1.getNumColumns());
 		long rowsB = toInt(in2.getNumRows());
@@ -3114,8 +3178,8 @@ public class LibMatrixCUDA {
 
 		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx)
-				.launchKernel("cbind", ExecutionConfig.getConfigForSimpleMatrixOperations(maxRows, maxCols), A, B, C,
-						rowsA, colsA, rowsB, colsB);
+		.launchKernel("cbind", ExecutionConfig.getConfigForSimpleMatrixOperations(maxRows, maxCols), A, B, C,
+				rowsA, colsA, rowsB, colsB);
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CBIND_KERNEL, System.nanoTime() - t1);
 
 	}
@@ -3126,7 +3190,7 @@ public class LibMatrixCUDA {
 		LOG.trace("GPU : rbind" + ", GPUContext=" + gCtx);
 
 		long t1 = 0;
-		
+
 		int rowsA = toInt(in1.getNumRows());
 		int colsA = toInt(in1.getNumColumns());
 		int rowsB = toInt(in2.getNumRows());
@@ -3147,8 +3211,8 @@ public class LibMatrixCUDA {
 
 		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx)
-				.launchKernel("rbind", ExecutionConfig.getConfigForSimpleMatrixOperations(maxRows, maxCols), A, B, C,
-						rowsA, colsA, rowsB, colsB);
+		.launchKernel("rbind", ExecutionConfig.getConfigForSimpleMatrixOperations(maxRows, maxCols), A, B, C,
+				rowsA, colsA, rowsB, colsB);
 		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RBIND_KERNEL, System.nanoTime() - t1);
 
 	}
@@ -3422,7 +3486,7 @@ public class LibMatrixCUDA {
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public static void axpy(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2,
-													String outputName,  double constant) throws DMLRuntimeException {
+			String outputName,  double constant) throws DMLRuntimeException {
 		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");
 		Pointer A = getDensePointer(gCtx, in1, instName);
@@ -3433,9 +3497,9 @@ public class LibMatrixCUDA {
 
 		long t1=0, t2=0;
 		if(in1.getNumRows() == in2.getNumRows() && in1.getNumColumns() == in2.getNumColumns()) {
-            LOG.trace("GPU : cublasDaxpy" + ", GPUContext=" + gCtx);
+			LOG.trace("GPU : cublasDaxpy" + ", GPUContext=" + gCtx);
 
-            // Matrix-Matrix daxpy
+			// Matrix-Matrix daxpy
 			long n = in1.getNumRows()*in2.getNumColumns(); // Since A is always a matrix
 			Pointer alphaPtr = pointerTo(constant);
 			// C <- A + alpha*B
@@ -3451,9 +3515,9 @@ public class LibMatrixCUDA {
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DAXPY_LIB, System.nanoTime() - t2);
 		}
 		else {
-            LOG.trace("GPU : daxpy_matrix_vector" + ", GPUContext=" + gCtx);
+			LOG.trace("GPU : daxpy_matrix_vector" + ", GPUContext=" + gCtx);
 
-            // Matrix-Vector daxpy
+			// Matrix-Vector daxpy
 			// Note: Vector-Matrix operation is not supported
 			// daxpy_matrix_vector(double* A,  double* B, double alpha, double* ret, int rlenA, int clenA, int rlenB, int clenB)
 			if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
@@ -3466,51 +3530,51 @@ public class LibMatrixCUDA {
 	}
 
 
-    /**
-     * Implements the "solve" function for systemml Ax = B (A is of size m*n, B is of size m*1, x is of size n*1)
-     *
-     * @param ec         a valid {@link ExecutionContext}
-     * @param gCtx       a valid {@link GPUContext}
-     * @param instName   the invoking instruction's name for record {@link Statistics}.
-     * @param in1        input matrix A
-     * @param in2        input matrix B
-     * @param outputName name of the output matrix
-     * @throws DMLRuntimeException if an error occurs
-     */
-    public static void solve(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2, String outputName) throws DMLRuntimeException {
-        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");
+	/**
+	 * Implements the "solve" function for systemml Ax = B (A is of size m*n, B is of size m*1, x is of size n*1)
+	 *
+	 * @param ec         a valid {@link ExecutionContext}
+	 * @param gCtx       a valid {@link GPUContext}
+	 * @param instName   the invoking instruction's name for record {@link Statistics}.
+	 * @param in1        input matrix A
+	 * @param in2        input matrix B
+	 * @param outputName name of the output matrix
+	 * @throws DMLRuntimeException if an error occurs
+	 */
+	public static void solve(ExecutionContext ec, GPUContext gCtx, String instName, MatrixObject in1, MatrixObject in2, String outputName) throws DMLRuntimeException {
+		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");
 
-        // x = solve(A, b)
+		// x = solve(A, b)
 		LOG.trace("GPU : solve" + ", GPUContext=" + gCtx);
 
 		long t0 = -1;
 
-        if (!isInSparseFormat(gCtx, in1) && !isInSparseFormat(gCtx, in2)) {    // Both dense
-            GPUObject Aobj = in1.getGPUObject(gCtx);
-            GPUObject bobj = in2.getGPUObject(gCtx);
-            int m = toInt(in1.getNumRows());
-            int n = toInt(in1.getNumColumns());
-            if (in2.getNumRows() != m)
-                throw new DMLRuntimeException("GPU : Incorrect input for solve(), rows in A should be the same as rows in B");
-            if (in2.getNumColumns() != 1)
-                throw new DMLRuntimeException("GPU : Incorrect input for solve(), columns in B should be 1");
-
-
-            // Copy over matrices and
-            // convert dense matrices to row major
-            // Operation in cuSolver and cuBlas are for column major dense matrices
-            // and are destructive to the original input
+		if (!isInSparseFormat(gCtx, in1) && !isInSparseFormat(gCtx, in2)) {    // Both dense
+			GPUObject Aobj = in1.getGPUObject(gCtx);
+			GPUObject bobj = in2.getGPUObject(gCtx);
+			int m = toInt(in1.getNumRows());
+			int n = toInt(in1.getNumColumns());
+			if (in2.getNumRows() != m)
+				throw new DMLRuntimeException("GPU : Incorrect input for solve(), rows in A should be the same as rows in B");
+			if (in2.getNumColumns() != 1)
+				throw new DMLRuntimeException("GPU : Incorrect input for solve(), columns in B should be 1");
+
+
+			// Copy over matrices and
+			// convert dense matrices to row major
+			// Operation in cuSolver and cuBlas are for column major dense matrices
+			// and are destructive to the original input
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
-            GPUObject ATobj = (GPUObject) Aobj.clone();
+			GPUObject ATobj = (GPUObject) Aobj.clone();
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_OBJECT_CLONE, System.nanoTime() - t0);
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			ATobj.denseRowMajorToColumnMajor();
-            if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ROW_TO_COLUMN_MAJOR, System.nanoTime() - t0);
-            Pointer A = ATobj.getJcudaDenseMatrixPtr();
+			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ROW_TO_COLUMN_MAJOR, System.nanoTime() - t0);
+			Pointer A = ATobj.getJcudaDenseMatrixPtr();
 
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
-            GPUObject bTobj = (GPUObject) bobj.clone();
+			GPUObject bTobj = (GPUObject) bobj.clone();
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_OBJECT_CLONE, System.nanoTime() - t0);
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			bTobj.denseRowMajorToColumnMajor();
@@ -3518,76 +3582,76 @@ public class LibMatrixCUDA {
 
 			Pointer b = bTobj.getJcudaDenseMatrixPtr();
 
-            // The following set of operations is done following the example in the cusolver documentation
-            // http://docs.nvidia.com/cuda/cusolver/#ormqr-example1
+			// The following set of operations is done following the example in the cusolver documentation
+			// http://docs.nvidia.com/cuda/cusolver/#ormqr-example1
 
-            // step 3: query working space of geqrf and ormqr
+			// step 3: query working space of geqrf and ormqr
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			int[] lwork = {0};
-            JCusolverDn.cusolverDnDgeqrf_bufferSize(gCtx.getCusolverDnHandle(), m, n, A, m, lwork);
+			JCusolverDn.cusolverDnDgeqrf_bufferSize(gCtx.getCusolverDnHandle(), m, n, A, m, lwork);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_QR_BUFFER, System.nanoTime() - t0);
 
 
 			// step 4: compute QR factorization
-            Pointer work = gCtx.allocate(instName, lwork[0] * Sizeof.DOUBLE);
-            Pointer tau = gCtx.allocate(instName, m * Sizeof.DOUBLE);
-            Pointer devInfo = gCtx.allocate(Sizeof.INT);
+			Pointer work = gCtx.allocate(instName, lwork[0] * Sizeof.DOUBLE);
+			Pointer tau = gCtx.allocate(instName, m * Sizeof.DOUBLE);
+			Pointer devInfo = gCtx.allocate(Sizeof.INT);
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			JCusolverDn.cusolverDnDgeqrf(gCtx.getCusolverDnHandle(), m, n, A, m, tau, work, lwork[0], devInfo);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_QR, System.nanoTime() - t0);
 
 
 			int[] qrError = {-1};
-            cudaMemcpy(Pointer.to(qrError), devInfo, Sizeof.INT, cudaMemcpyDeviceToHost);
-            if (qrError[0] != 0) {
-                throw new DMLRuntimeException("GPU : Error in call to geqrf (QR factorization) as part of solve, argument " + qrError[0] + " was wrong");
-            }
+			cudaMemcpy(Pointer.to(qrError), devInfo, Sizeof.INT, cudaMemcpyDeviceToHost);
+			if (qrError[0] != 0) {
+				throw new DMLRuntimeException("GPU : Error in call to geqrf (QR factorization) as part of solve, argument " + qrError[0] + " was wrong");
+			}
 
-            // step 5: compute Q^T*B
+			// step 5: compute Q^T*B
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			JCusolverDn.cusolverDnDormqr(gCtx.getCusolverDnHandle(), cublasSideMode.CUBLAS_SIDE_LEFT, cublasOperation.CUBLAS_OP_T, m, 1, n, A, m, tau, b, m, work, lwork[0], devInfo);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ORMQR, System.nanoTime() - t0);
 			cudaMemcpy(Pointer.to(qrError), devInfo, Sizeof.INT, cudaMemcpyDeviceToHost);
-            if (qrError[0] != 0) {
-                throw new DMLRuntimeException("GPU : Error in call to ormqr (to compuete Q^T*B after QR factorization) as part of solve, argument " + qrError[0] + " was wrong");
-            }
+			if (qrError[0] != 0) {
+				throw new DMLRuntimeException("GPU : Error in call to ormqr (to compuete Q^T*B after QR factorization) as part of solve, argument " + qrError[0] + " was wrong");
+			}
 
-            // step 6: compute x = R \ Q^T*B
+			// step 6: compute x = R \ Q^T*B
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			JCublas2.cublasDtrsm(gCtx.getCublasHandle(),
-                cublasSideMode.CUBLAS_SIDE_LEFT, cublasFillMode.CUBLAS_FILL_MODE_UPPER, cublasOperation.CUBLAS_OP_N, cublasDiagType.CUBLAS_DIAG_NON_UNIT,
-                n, 1, pointerTo(1.0), A, m, b, m);
+					cublasSideMode.CUBLAS_SIDE_LEFT, cublasFillMode.CUBLAS_FILL_MODE_UPPER, cublasOperation.CUBLAS_OP_N, cublasDiagType.CUBLAS_DIAG_NON_UNIT,
+					n, 1, pointerTo(1.0), A, m, b, m);
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_TRSM, System.nanoTime() - t0);
 
 			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
 			bTobj.denseColumnMajorToRowMajor();
 			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_COLUMN_TO_ROW_MAJOR, System.nanoTime() - t0);
 
-            // TODO  : Find a way to assign bTobj directly to the output and set the correct flags so as to not crash
-            // There is an avoidable copy happening here
+			// TODO  : Find a way to assign bTobj directly to the output and set the correct flags so as to not crash
+			// There is an avoidable copy happening here
 			MatrixObject out = getDenseMatrixOutputForGPUInstruction(ec, instName, outputName, in1.getNumColumns(), 1);
-            cudaMemcpy(out.getGPUObject(gCtx).getJcudaDenseMatrixPtr(), bTobj.getJcudaDenseMatrixPtr(), n * 1 * Sizeof.DOUBLE, cudaMemcpyDeviceToDevice);
+			cudaMemcpy(out.getGPUObject(gCtx).getJcudaDenseMatrixPtr(), bTobj.getJcudaDenseMatrixPtr(), n * 1 * Sizeof.DOUBLE, cudaMemcpyDeviceToDevice);
 
-            gCtx.cudaFreeHelper(instName, work);
-            gCtx.cudaFreeHelper(instName, tau);
-            ATobj.clearData();
-            bTobj.clearData();
+			gCtx.cudaFreeHelper(instName, work);
+			gCtx.cudaFreeHelper(instName, tau);
+			ATobj.clearData();
+			bTobj.clearData();
 
-            //debugPrintMatrix(b, n, 1);
+			//debugPrintMatrix(b, n, 1);
 
 
-        } else if (isInSparseFormat(gCtx, in1) && isInSparseFormat(gCtx, in2)) { // Both sparse
-            throw new DMLRuntimeException("GPU : solve on sparse inputs not supported");
-        } else if (!isInSparseFormat(gCtx, in1) && isInSparseFormat(gCtx, in2)) { // A is dense, b is sparse
-            // Pointer A = getDensePointer(gCtx, in1, instName);
-            // Pointer B = getDensePointer(gCtx, in2, instName);
-            throw new DMLRuntimeException("GPU : solve on sparse inputs not supported");
-        } else if (isInSparseFormat(gCtx, in1) && !isInSparseFormat(gCtx, in2)) { // A is sparse, b is dense
-            throw new DMLRuntimeException("GPU : solve on sparse inputs not supported");
-        }
+		} else if (isInSparseFormat(gCtx, in1) && isInSparseFormat(gCtx, in2)) { // Both sparse
+			throw new DMLRuntimeException("GPU : solve on sparse inputs not supported");
+		} else if (!isInSparseFormat(gCtx, in1) && isInSparseFormat(gCtx, in2)) { // A is dense, b is sparse
+			// Pointer A = getDensePointer(gCtx, in1, instName);
+			// Pointer B = getDensePointer(gCtx, in2, instName);
+			throw new DMLRuntimeException("GPU : solv

<TRUNCATED>