Repository: systemml
Updated Branches:
  refs/heads/master f1bf97baf -> 8af7a9ea3


[SYSTEMML-2438] Performance matrix compression column group partitioning

This patch fixes performance issues with the default bin-packing-based
column group partitioning when compression matrices with many columns (>
millions). In detail, the first fit decreasing heuristic with O(n^2)
complexity was the main bottleneck. We now (1) run this heuristic over
max size sequences of items to ensure robustness, and (2) use native
arrays to encode the individual bins. 

When compression Freescale1 (Florida matrix collection, 3.4M x 3.4m,
nnz=19M) this patch improved the time for column group partitioning from
191s to 304ms. In detail, the native arrays improved 190s->41s, while
the partitioning further improved it from 41->304ms.


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

Branch: refs/heads/master
Commit: 8af7a9ea32d9b0672d819aa8a8e891506ad9107f
Parents: f1bf97b
Author: Matthias Boehm <[email protected]>
Authored: Thu Jul 12 20:39:17 2018 -0700
Committer: Matthias Boehm <[email protected]>
Committed: Thu Jul 12 20:39:17 2018 -0700

----------------------------------------------------------------------
 .../compress/cocode/ColumnGroupPartitioner.java |  2 +-
 .../ColumnGroupPartitionerBinPacking.java       | 53 +++++++++++++-------
 .../cocode/ColumnGroupPartitionerStatic.java    | 11 ++--
 .../compress/cocode/PlanningCoCoder.java        | 16 +++---
 .../runtime/compress/utils/IntArrayList.java    |  5 ++
 5 files changed, 54 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/8af7a9ea/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitioner.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitioner.java
 
b/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitioner.java
index 17fe58b..2b1f61f 100644
--- 
a/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitioner.java
+++ 
b/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitioner.java
@@ -34,5 +34,5 @@ public abstract class ColumnGroupPartitioner
         * @param groupColsInfo list of column infos
         * @return list of partitions (where each partition is a list of 
columns)
         */
-       public abstract List<List<Integer>> partitionColumns(List<Integer> 
groupCols, HashMap<Integer, GroupableColInfo> groupColsInfo);
+       public abstract List<int[]> partitionColumns(List<Integer> groupCols, 
HashMap<Integer, GroupableColInfo> groupColsInfo);
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/8af7a9ea/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java
 
b/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java
index 8b849d2..b6dccf6 100644
--- 
a/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java
+++ 
b/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java
@@ -23,9 +23,11 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.apache.commons.lang.ArrayUtils;
 import 
org.apache.sysml.runtime.compress.cocode.PlanningCoCoder.GroupableColInfo;
+import org.apache.sysml.runtime.compress.utils.IntArrayList;
 import org.apache.sysml.runtime.util.SortUtils;
 
 /**
@@ -35,14 +37,15 @@ import org.apache.sysml.runtime.util.SortUtils;
 public class ColumnGroupPartitionerBinPacking extends ColumnGroupPartitioner
 {
        private static final boolean FIRST_FIT_DEC = true;
-       private static final int MAX_COL_PER_GROUP = Integer.MAX_VALUE;
+       private static final int MAX_COL_FIRST_FIT = 16384;
+       private static final int MAX_COL_PER_GROUP = 1024;
 
        //we use a constant partition size (independent of the number of rows
        //in order to ensure constant compression speed independent of blocking)
        public static double BIN_CAPACITY = 0.000032; //higher values, more 
grouping
        
        @Override
-       public List<List<Integer>> partitionColumns(List<Integer> groupCols, 
HashMap<Integer, GroupableColInfo> groupColsInfo) 
+       public List<int[]> partitionColumns(List<Integer> groupCols, 
HashMap<Integer, GroupableColInfo> groupColsInfo) 
        {
                //obtain column weights
                int[] items = new int[groupCols.size()];
@@ -53,15 +56,29 @@ public class ColumnGroupPartitionerBinPacking extends 
ColumnGroupPartitioner
                        itemWeights[i] = groupColsInfo.get(col).cardRatio;
                } 
                
-               //sort items (first fit decreasing)
-               if( FIRST_FIT_DEC ) {
-                       SortUtils.sortByValue(0, items.length, itemWeights, 
items);
-                       ArrayUtils.reverse(items);
-                       ArrayUtils.reverse(itemWeights);
+               //run first fit heuristic over sequences of at most 
MAX_COL_FIRST_FIT
+               //items to ensure robustness for matrices with many columns due 
to O(n^2)
+               List<IntArrayList> bins = new ArrayList<>();
+               for(int i=0; i<items.length; i+=MAX_COL_FIRST_FIT) {
+                       //extract sequence of items and item weights
+                       int iu = Math.min(i+MAX_COL_FIRST_FIT, items.length);
+                       int[] litems = Arrays.copyOfRange(items, i, iu);
+                       double[] litemWeights = Arrays.copyOfRange(itemWeights, 
i, iu);
+                       
+                       //sort items (first fit decreasing)
+                       if( FIRST_FIT_DEC ) {
+                               SortUtils.sortByValue(0, litems.length, 
litemWeights, litems);
+                               ArrayUtils.reverse(litems);
+                               ArrayUtils.reverse(litemWeights);
+                       }
+                       
+                       //partition columns via bin packing
+                       bins.addAll(packFirstFit(litems, litemWeights));
                }
                
-               //partition columns via bin packing
-               return packFirstFit(items, itemWeights);
+               //extract native int arrays for individual bins 
+               return bins.stream().map(b -> b.extractValues())
+                       .collect(Collectors.toList());
        }
 
        /**
@@ -71,27 +88,29 @@ public class ColumnGroupPartitionerBinPacking extends 
ColumnGroupPartitioner
         * @param itemWeights the weights of the items
         * @return
         */
-       private static List<List<Integer>> packFirstFit(int[] items, double[] 
itemWeights) 
+       private static List<IntArrayList> packFirstFit(int[] items, double[] 
itemWeights) 
        {
-               List<List<Integer>> bins = new ArrayList<>();
-               List<Double> binWeights = new ArrayList<>();
+               List<IntArrayList> bins = new ArrayList<>();
+               double[] binWeights = new double[16];
                
                for( int i = 0; i < items.length; i++ ) {
                        //add to existing bin
                        boolean assigned = false;
                        for( int j = 0; j < bins.size(); j++ ) {
-                               double newBinWeight = 
binWeights.get(j)-itemWeights[i];
+                               double newBinWeight = 
binWeights[j]-itemWeights[i];
                                if( newBinWeight >= 0 && bins.get(j).size() < 
MAX_COL_PER_GROUP-1 ){
-                                       bins.get(j).add(items[i]);
-                                       binWeights.set(j, newBinWeight);
+                                       bins.get(j).appendValue(items[i]);
+                                       binWeights[j] = newBinWeight;
                                        assigned = true; break;
                                }
                        }
                        
                        //create new bin at end of list
                        if( !assigned ) {
-                               bins.add(new 
ArrayList<>(Arrays.asList(items[i])));
-                               binWeights.add(BIN_CAPACITY-itemWeights[i]);
+                               if( bins.size() == binWeights.length )
+                                       binWeights = Arrays.copyOf(binWeights, 
2*binWeights.length);
+                               bins.add(new IntArrayList(items[i]));
+                               binWeights[bins.size()-1] = 
BIN_CAPACITY-itemWeights[i];
                        }
                }
                

http://git-wip-us.apache.org/repos/asf/systemml/blob/8af7a9ea/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerStatic.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerStatic.java
 
b/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerStatic.java
index d830133..52e51d1 100644
--- 
a/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerStatic.java
+++ 
b/src/main/java/org/apache/sysml/runtime/compress/cocode/ColumnGroupPartitionerStatic.java
@@ -34,19 +34,16 @@ public class ColumnGroupPartitionerStatic extends 
ColumnGroupPartitioner
        private static final int MAX_COL_PER_GROUP = 20;
 
        @Override
-       public List<List<Integer>> partitionColumns(List<Integer> groupCols, 
HashMap<Integer, GroupableColInfo> groupColsInfo) 
-       {
-               List<List<Integer>> ret = new ArrayList<>();
+       public List<int[]> partitionColumns(List<Integer> groupCols, 
HashMap<Integer, GroupableColInfo> groupColsInfo) {
+               List<int[]> ret = new ArrayList<>();
                int numParts = 
(int)Math.ceil((double)groupCols.size()/MAX_COL_PER_GROUP);
                int partSize = 
(int)Math.ceil((double)groupCols.size()/numParts);
-               
                for( int i=0, pos=0; i<numParts; i++, pos+=partSize ) {
-                       List<Integer> tmp = new ArrayList<>();
+                       int[] tmp = new int[Math.min(partSize, 
groupCols.size()-pos)];
                        for( int j=0; j<partSize && pos+j<groupCols.size(); j++ 
)
-                               tmp.add(groupCols.get(pos+j));
+                               tmp[j] = groupCols.get(pos+j);
                        ret.add(tmp);
                }
-               
                return ret;
        }
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/8af7a9ea/src/main/java/org/apache/sysml/runtime/compress/cocode/PlanningCoCoder.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/compress/cocode/PlanningCoCoder.java 
b/src/main/java/org/apache/sysml/runtime/compress/cocode/PlanningCoCoder.java
index d6f6566..7f48f14 100644
--- 
a/src/main/java/org/apache/sysml/runtime/compress/cocode/PlanningCoCoder.java
+++ 
b/src/main/java/org/apache/sysml/runtime/compress/cocode/PlanningCoCoder.java
@@ -63,7 +63,7 @@ public class PlanningCoCoder
                }
                
                // use column group partitioner to create partitions of columns
-               List<List<Integer>> bins = 
createColumnGroupPartitioner(COLUMN_PARTITIONER)
+               List<int[]> bins = 
createColumnGroupPartitioner(COLUMN_PARTITIONER)
                                .partitionColumns(groupCols, groupColsInfo);
 
                // brute force grouping within each partition
@@ -72,13 +72,13 @@ public class PlanningCoCoder
                                getCocodingGroupsBruteForce(bins, 
groupColsInfo, sizeEstimator, numRows);
        }
 
-       private static List<int[]> 
getCocodingGroupsBruteForce(List<List<Integer>> bins, HashMap<Integer, 
GroupableColInfo> groupColsInfo, CompressedSizeEstimator estim, int rlen) 
+       private static List<int[]> getCocodingGroupsBruteForce(List<int[]> 
bins, HashMap<Integer, GroupableColInfo> groupColsInfo, CompressedSizeEstimator 
estim, int rlen) 
        {
                List<int[]> retGroups = new ArrayList<>();
-               for (List<Integer> bin : bins) {
+               for( int[] bin : bins ) {
                        // building an array of singleton CoCodingGroup
                        ArrayList<PlanningCoCodingGroup> sgroups = new 
ArrayList<>();
-                       for (Integer col : bin)
+                       for( int col : bin )
                                sgroups.add(new PlanningCoCodingGroup(col, 
groupColsInfo.get(col)));
                        // brute force co-coding        
                        PlanningCoCodingGroup[] outputGroups = 
findCocodesBruteForce(
@@ -90,20 +90,20 @@ public class PlanningCoCoder
                return retGroups;
        }
 
-       private static List<int[]> 
getCocodingGroupsBruteForce(List<List<Integer>> bins, HashMap<Integer, 
GroupableColInfo> groupColsInfo, CompressedSizeEstimator estim, int rlen, int 
k) 
+       private static List<int[]> getCocodingGroupsBruteForce(List<int[]> 
bins, HashMap<Integer, GroupableColInfo> groupColsInfo, CompressedSizeEstimator 
estim, int rlen, int k) 
        {
                List<int[]> retGroups = new ArrayList<>();
                try {
                        ExecutorService pool = CommonThreadPool.get(k);
                        ArrayList<CocodeTask> tasks = new ArrayList<>();
-                       for (List<Integer> bin : bins) {
+                       for( int[] bin : bins ) {
                                // building an array of singleton CoCodingGroup
                                ArrayList<PlanningCoCodingGroup> sgroups = new 
ArrayList<>();
-                               for (Integer col : bin)
+                               for( int col : bin )
                                        sgroups.add(new 
PlanningCoCodingGroup(col, groupColsInfo.get(col)));
                                tasks.add(new CocodeTask(estim, sgroups, rlen));
                        }
-                       List<Future<PlanningCoCodingGroup[]>> rtask = 
pool.invokeAll(tasks);    
+                       List<Future<PlanningCoCodingGroup[]>> rtask = 
pool.invokeAll(tasks);
                        for( Future<PlanningCoCodingGroup[]> lrtask : rtask )
                                for (PlanningCoCodingGroup grp : lrtask.get())
                                        retGroups.add(grp.getColIndices());

http://git-wip-us.apache.org/repos/asf/systemml/blob/8af7a9ea/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java 
b/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java
index 93fcb70..e51f538 100644
--- a/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java
+++ b/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java
@@ -39,6 +39,11 @@ public class IntArrayList
                _data = null;
                _size = 0;
        }
+       
+       public IntArrayList(int value) {
+               this();
+               appendValue(value);
+       }
 
        public int size() {
                return _size;

Reply via email to