Skip to content

Conversation

@sameeragarwal
Copy link
Member

What changes were proposed in this pull request?

This PR adds support for generating the AggregateHashMap class in TungstenAggregate if the aggregate group by keys/value are of LongType. Note that currently this generate aggregate is not actually used.

NB: This currently only supports LongType keys/values (please see isAggregateHashMapSupported in TungstenAggregate) and will be generalized to other data types in a subsequent PR.

How was this patch tested?

Manually inspected the generated code. This is what the generated map looks like for 2 keys:

/* 068 */   public class agg_GeneratedAggregateHashMap {
/* 069 */     private org.apache.spark.sql.execution.vectorized.ColumnarBatch batch;
/* 070 */     private int[] buckets;
/* 071 */     private int numBuckets;
/* 072 */     private int maxSteps;
/* 073 */     private int numRows = 0;
/* 074 */     private org.apache.spark.sql.types.StructType schema =
/* 075 */     new org.apache.spark.sql.types.StructType()
/* 076 */     .add("k1", org.apache.spark.sql.types.DataTypes.LongType)
/* 077 */     .add("k2", org.apache.spark.sql.types.DataTypes.LongType)
/* 078 */     .add("sum", org.apache.spark.sql.types.DataTypes.LongType);
/* 079 */     
/* 080 */     public agg_GeneratedAggregateHashMap(int capacity, double loadFactor, int maxSteps) {
/* 081 */       assert (capacity > 0 && ((capacity & (capacity - 1)) == 0));
/* 082 */       this.maxSteps = maxSteps;
/* 083 */       numBuckets = (int) (capacity / loadFactor);
/* 084 */       batch = org.apache.spark.sql.execution.vectorized.ColumnarBatch.allocate(schema,
/* 085 */         org.apache.spark.memory.MemoryMode.ON_HEAP, capacity);
/* 086 */       buckets = new int[numBuckets];
/* 087 */       java.util.Arrays.fill(buckets, -1);
/* 088 */     }
/* 089 */     
/* 090 */     public agg_GeneratedAggregateHashMap() {
/* 091 */       new agg_GeneratedAggregateHashMap(1 << 16, 0.25, 5);
/* 092 */     }
/* 093 */     
/* 094 */     public org.apache.spark.sql.execution.vectorized.ColumnarBatch.Row findOrInsert(long agg_key, long agg_key1) {
/* 095 */       long h = hash(agg_key, agg_key1);
/* 096 */       int step = 0;
/* 097 */       int idx = (int) h & (numBuckets - 1);
/* 098 */       while (step < maxSteps) {
/* 099 */         // Return bucket index if it's either an empty slot or already contains the key
/* 100 */         if (buckets[idx] == -1) {
/* 101 */           batch.column(0).putLong(numRows, agg_key);
/* 102 */           batch.column(1).putLong(numRows, agg_key1);
/* 103 */           batch.column(2).putLong(numRows, 0);
/* 104 */           buckets[idx] = numRows++;
/* 105 */           return batch.getRow(buckets[idx]);
/* 106 */         } else if (equals(idx, agg_key, agg_key1)) {
/* 107 */           return batch.getRow(buckets[idx]);
/* 108 */         }
/* 109 */         idx = (idx + 1) & (numBuckets - 1);
/* 110 */         step++;
/* 111 */       }
/* 112 */       // Didn't find it
/* 113 */       return null;
/* 114 */     }
/* 115 */     
/* 116 */     private boolean equals(int idx, long agg_key, long agg_key1) {
/* 117 */       return batch.column(0).getLong(buckets[idx]) == agg_key && batch.column(1).getLong(buckets[idx]) == agg_key1;
/* 118 */     }
/* 119 */     
/* 120 */     // TODO: Improve this Hash Function
/* 121 */     private long hash(long agg_key, long agg_key1) {
/* 122 */       return agg_key ^ agg_key1;
/* 123 */     }
/* 124 */     
/* 125 */   }

@SparkQA
Copy link

SparkQA commented Apr 5, 2016

Test build #54918 has finished for PR 12161 at commit 8a47e1e.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@sameeragarwal sameeragarwal force-pushed the tungsten-aggregate branch 3 times, most recently from d0eddf7 to a31be48 Compare April 5, 2016 22:44
@sameeragarwal sameeragarwal changed the title [WIP][SPARK-14394][SQL] Generate AggregateHashMap class during TungstenAggregate codegen [SPARK-14394][SQL] Generate AggregateHashMap class during TungstenAggregate codegen Apr 5, 2016
@sameeragarwal
Copy link
Member Author

Partial Aggregate for a query of the form sqlContext.range(N).selectExpr("id", "(id & 65535) as k1", "(id & 65535) as k2").groupBy("k1", "k2").sum("id").collect() looks like:

/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */ 
/* 005 */ /** Codegened pipeline for:
/* 006 */ * TungstenAggregate(key=[k1#146L,k2#147L], functions=[(sum(id#143L),mode=Partial,isDistinct=false)], output=[k1#146L,k2#147L,sum#1...
/* 007 */   */
/* 008 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 009 */   private Object[] references;
/* 010 */   private boolean agg_initAgg;
/* 011 */   private agg_GeneratedAggregateHashMap agg_aggregateHashMap;
/* 012 */   private org.apache.spark.sql.execution.aggregate.TungstenAggregate agg_plan;
/* 013 */   private org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap;
/* 014 */   private org.apache.spark.sql.execution.UnsafeKVExternalSorter agg_sorter;
/* 015 */   private org.apache.spark.unsafe.KVIterator agg_mapIter;
/* 016 */   private org.apache.spark.sql.execution.metric.LongSQLMetric range_numOutputRows;
/* 017 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue range_metricValue;
/* 018 */   private boolean range_initRange;
/* 019 */   private long range_partitionEnd;
/* 020 */   private long range_number;
/* 021 */   private boolean range_overflow;
/* 022 */   private scala.collection.Iterator range_input;
/* 023 */   private UnsafeRow range_result;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder range_holder;
/* 025 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter range_rowWriter;
/* 026 */   private UnsafeRow project_result;
/* 027 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 028 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 029 */   private UnsafeRow agg_result;
/* 030 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 031 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 032 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowJoiner agg_unsafeRowJoiner;
/* 033 */   private org.apache.spark.sql.execution.metric.LongSQLMetric wholestagecodegen_numOutputRows;
/* 034 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue wholestagecodegen_metricValue;
/* 035 */   
/* 036 */   public GeneratedIterator(Object[] references) {
/* 037 */     this.references = references;
/* 038 */   }
/* 039 */   
/* 040 */   public void init(int index, scala.collection.Iterator inputs[]) {
/* 041 */     partitionIndex = index;
/* 042 */     agg_initAgg = false;
/* 043 */     agg_aggregateHashMap = new agg_GeneratedAggregateHashMap();
/* 044 */     this.agg_plan = (org.apache.spark.sql.execution.aggregate.TungstenAggregate) references[0];
/* 045 */     agg_hashMap = agg_plan.createHashMap();
/* 046 */     
/* 047 */     this.range_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1];
/* 048 */     range_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) range_numOutputRows.localValue();
/* 049 */     range_initRange = false;
/* 050 */     range_partitionEnd = 0L;
/* 051 */     range_number = 0L;
/* 052 */     range_overflow = false;
/* 053 */     range_input = inputs[0];
/* 054 */     range_result = new UnsafeRow(1);
/* 055 */     this.range_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(range_result, 0);
/* 056 */     this.range_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(range_holder, 1);
/* 057 */     project_result = new UnsafeRow(3);
/* 058 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 0);
/* 059 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 3);
/* 060 */     agg_result = new UnsafeRow(2);
/* 061 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 062 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 2);
/* 063 */     agg_unsafeRowJoiner = agg_plan.createUnsafeJoiner();
/* 064 */     this.wholestagecodegen_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[2];
/* 065 */     wholestagecodegen_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) wholestagecodegen_numOutputRows.localValue();
/* 066 */   }
/* 067 */   
/* 068 */   public class agg_GeneratedAggregateHashMap {
/* 069 */     private org.apache.spark.sql.execution.vectorized.ColumnarBatch batch;
/* 070 */     private int[] buckets;
/* 071 */     private int numBuckets;
/* 072 */     private int maxSteps;
/* 073 */     private int numRows = 0;
/* 074 */     private org.apache.spark.sql.types.StructType schema =
/* 075 */     new org.apache.spark.sql.types.StructType()
/* 076 */     .add("k1", org.apache.spark.sql.types.DataTypes.LongType)
/* 077 */     .add("k2", org.apache.spark.sql.types.DataTypes.LongType)
/* 078 */     .add("sum", org.apache.spark.sql.types.DataTypes.LongType);
/* 079 */     
/* 080 */     public agg_GeneratedAggregateHashMap(int capacity, double loadFactor, int maxSteps) {
/* 081 */       assert (capacity > 0 && ((capacity & (capacity - 1)) == 0));
/* 082 */       this.maxSteps = maxSteps;
/* 083 */       numBuckets = (int) (capacity / loadFactor);
/* 084 */       batch = org.apache.spark.sql.execution.vectorized.ColumnarBatch.allocate(schema,
/* 085 */         org.apache.spark.memory.MemoryMode.ON_HEAP, capacity);
/* 086 */       buckets = new int[numBuckets];
/* 087 */       java.util.Arrays.fill(buckets, -1);
/* 088 */     }
/* 089 */     
/* 090 */     public agg_GeneratedAggregateHashMap() {
/* 091 */       new agg_GeneratedAggregateHashMap(1 << 16, 0.25, 5);
/* 092 */     }
/* 093 */     
/* 094 */     public org.apache.spark.sql.execution.vectorized.ColumnarBatch.Row findOrInsert(long agg_key, long agg_key1) {
/* 095 */       long h = hash(agg_key, agg_key1);
/* 096 */       int step = 0;
/* 097 */       int idx = (int) h & (numBuckets - 1);
/* 098 */       while (step < maxSteps) {
/* 099 */         // Return bucket index if it's either an empty slot or already contains the key
/* 100 */         if (buckets[idx] == -1) {
/* 101 */           batch.column(0).putLong(numRows, agg_key);
/* 102 */           batch.column(1).putLong(numRows, agg_key1);
/* 103 */           batch.column(2).putLong(numRows, 0);
/* 104 */           buckets[idx] = numRows++;
/* 105 */           return batch.getRow(buckets[idx]);
/* 106 */         } else if (equals(idx, agg_key, agg_key1)) {
/* 107 */           return batch.getRow(buckets[idx]);
/* 108 */         }
/* 109 */         idx = (idx + 1) & (numBuckets - 1);
/* 110 */         step++;
/* 111 */       }
/* 112 */       // Didn't find it
/* 113 */       return null;
/* 114 */     }
/* 115 */     
/* 116 */     private boolean equals(int idx, long agg_key, long agg_key1) {
/* 117 */       return batch.column(0).getLong(buckets[idx]) == agg_key && batch.column(1).getLong(buckets[idx]) == agg_key1;
/* 118 */     }
/* 119 */     
/* 120 */     // TODO: Improve this Hash Function
/* 121 */     private long hash(long agg_key, long agg_key1) {
/* 122 */       return agg_key ^ agg_key1;
/* 123 */     }
/* 124 */     
/* 125 */   }
/* 126 */   
/* 127 */   private void agg_doAggregateWithKeys() throws java.io.IOException {
/* 128 */     /*** PRODUCE: Project [id#143L,(id#143L & 65535) AS k1#146L,(id#143L & 65535) AS k2#147L] */
/* 129 */     
/* 130 */     /*** PRODUCE: Range 0, 1, 1, 20971520, [id#143L] */
/* 131 */     
/* 132 */     // initialize Range
/* 133 */     if (!range_initRange) {
/* 134 */       range_initRange = true;
/* 135 */       initRange(partitionIndex);
/* 136 */     }
/* 137 */     
/* 138 */     while (!range_overflow && range_number < range_partitionEnd) {
/* 139 */       long range_value = range_number;
/* 140 */       range_number += 1L;
/* 141 */       if (range_number < range_value ^ 1L < 0) {
/* 142 */         range_overflow = true;
/* 143 */       }
/* 144 */       
/* 145 */       /*** CONSUME: Project [id#143L,(id#143L & 65535) AS k1#146L,(id#143L & 65535) AS k2#147L] */
/* 146 */       
/* 147 */       /*** CONSUME: TungstenAggregate(key=[k1#146L,k2#147L], functions=[(sum(id#143L),mode=Partial,isDistinct=false)], output=[k1#146L,k2#147L,sum#1... */
/* 148 */       /* (input[0, bigint] & 65535) */
/* 149 */       long project_value1 = -1L;
/* 150 */       project_value1 = range_value & 65535L;
/* 151 */       /* (input[0, bigint] & 65535) */
/* 152 */       long project_value4 = -1L;
/* 153 */       project_value4 = range_value & 65535L;
/* 154 */       
/* 155 */       // generate grouping key
/* 156 */       agg_rowWriter.write(0, project_value1);
/* 157 */       
/* 158 */       agg_rowWriter.write(1, project_value4);
/* 159 */       /* hash(input[1, bigint], input[2, bigint], 42) */
/* 160 */       int agg_value3 = 42;
/* 161 */       
/* 162 */       agg_value3 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(project_value1, agg_value3);
/* 163 */       
/* 164 */       agg_value3 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(project_value4, agg_value3);
/* 165 */       UnsafeRow agg_aggBuffer = null;
/* 166 */       if (true) {
/* 167 */         // try to get the buffer from hash map
/* 168 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value3);
/* 169 */       }
/* 170 */       if (agg_aggBuffer == null) {
/* 171 */         if (agg_sorter == null) {
/* 172 */           agg_sorter = agg_hashMap.destructAndCreateExternalSorter();
/* 173 */         } else {
/* 174 */           agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter());
/* 175 */         }
/* 176 */         
/* 177 */         // the hash map had be spilled, it should have enough memory now,
/* 178 */         // try  to allocate buffer again.
/* 179 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value3);
/* 180 */         if (agg_aggBuffer == null) {
/* 181 */           // failed to allocate the first page
/* 182 */           throw new OutOfMemoryError("No enough memory for aggregation");
/* 183 */         }
/* 184 */       }
/* 185 */       
/* 186 */       // evaluate aggregate function
/* 187 */       /* (coalesce(input[0, bigint], cast(0 as bigint)) + cast(input[1, bigint] as bigint)) */
/* 188 */       /* coalesce(input[0, bigint], cast(0 as bigint)) */
/* 189 */       /* input[0, bigint] */
/* 190 */       boolean agg_isNull7 = agg_aggBuffer.isNullAt(0);
/* 191 */       long agg_value8 = agg_isNull7 ? -1L : (agg_aggBuffer.getLong(0));
/* 192 */       boolean agg_isNull6 = agg_isNull7;
/* 193 */       long agg_value7 = agg_value8;
/* 194 */       
/* 195 */       if (agg_isNull6) {
/* 196 */         /* cast(0 as bigint) */
/* 197 */         boolean agg_isNull8 = false;
/* 198 */         long agg_value9 = -1L;
/* 199 */         if (!false) {
/* 200 */           agg_value9 = (long) 0;
/* 201 */         }
/* 202 */         if (!agg_isNull8) {
/* 203 */           agg_isNull6 = false;
/* 204 */           agg_value7 = agg_value9;
/* 205 */         }
/* 206 */       }
/* 207 */       /* cast(input[1, bigint] as bigint) */
/* 208 */       boolean agg_isNull10 = false;
/* 209 */       long agg_value11 = -1L;
/* 210 */       if (!false) {
/* 211 */         agg_value11 = range_value;
/* 212 */       }
/* 213 */       long agg_value6 = -1L;
/* 214 */       agg_value6 = agg_value7 + agg_value11;
/* 215 */       // update aggregate buffer
/* 216 */       agg_aggBuffer.setLong(0, agg_value6);
/* 217 */       
/* 218 */       if (shouldStop()) return;
/* 219 */     }
/* 220 */     
/* 221 */     agg_mapIter = agg_plan.finishAggregate(agg_hashMap, agg_sorter);
/* 222 */   }
/* 223 */   
/* 224 */   private void initRange(int idx) {
/* 225 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 226 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(1L);
/* 227 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(20971520L);
/* 228 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 229 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 230 */     
/* 231 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 232 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 233 */       range_number = Long.MAX_VALUE;
/* 234 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 235 */       range_number = Long.MIN_VALUE;
/* 236 */     } else {
/* 237 */       range_number = st.longValue();
/* 238 */     }
/* 239 */     
/* 240 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 241 */     .multiply(step).add(start);
/* 242 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 243 */       range_partitionEnd = Long.MAX_VALUE;
/* 244 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 245 */       range_partitionEnd = Long.MIN_VALUE;
/* 246 */     } else {
/* 247 */       range_partitionEnd = end.longValue();
/* 248 */     }
/* 249 */     
/* 250 */     range_metricValue.add((range_partitionEnd - range_number) / 1L);
/* 251 */   }
/* 252 */   
/* 253 */   protected void processNext() throws java.io.IOException {
/* 254 */     /*** PRODUCE: TungstenAggregate(key=[k1#146L,k2#147L], functions=[(sum(id#143L),mode=Partial,isDistinct=false)], output=[k1#146L,k2#147L,sum#1... */
/* 255 */     
/* 256 */     if (!agg_initAgg) {
/* 257 */       agg_initAgg = true;
/* 258 */       agg_doAggregateWithKeys();
/* 259 */     }
/* 260 */     
/* 261 */     // output the result
/* 262 */     while (agg_mapIter.next()) {
/* 263 */       wholestagecodegen_metricValue.add(1);
/* 264 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 265 */       UnsafeRow agg_aggBuffer1 = (UnsafeRow) agg_mapIter.getValue();
/* 266 */       
/* 267 */       UnsafeRow agg_resultRow = agg_unsafeRowJoiner.join(agg_aggKey, agg_aggBuffer1);
/* 268 */       
/* 269 */       /*** CONSUME: WholeStageCodegen */
/* 270 */       
/* 271 */       append(agg_resultRow);
/* 272 */       
/* 273 */       if (shouldStop()) return;
/* 274 */     }
/* 275 */     
/* 276 */     agg_mapIter.close();
/* 277 */     if (agg_sorter == null) {
/* 278 */       agg_hashMap.free();
/* 279 */     }
/* 280 */   }
/* 281 */ }

@SparkQA
Copy link

SparkQA commented Apr 5, 2016

Test build #55034 has finished for PR 12161 at commit a31be48.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class TungstenAggregateHashMap(
    • |public class $generatedClassName

@sameeragarwal
Copy link
Member Author

test this please

@SparkQA
Copy link

SparkQA commented Apr 5, 2016

Test build #55036 has finished for PR 12161 at commit a31be48.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class TungstenAggregateHashMap(
    • |public class $generatedClassName

@sameeragarwal
Copy link
Member Author

test this please

@sameeragarwal
Copy link
Member Author

cc @nongli

@SparkQA
Copy link

SparkQA commented Apr 5, 2016

Test build #55042 has finished for PR 12161 at commit a31be48.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class TungstenAggregateHashMap(
    • |public class $generatedClassName

@SparkQA
Copy link

SparkQA commented Apr 6, 2016

Test build #55068 has finished for PR 12161 at commit bd96657.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class TungstenAggregateHashMap(
    • |public class $generatedClassName

@SparkQA
Copy link

SparkQA commented Apr 6, 2016

Test build #55094 has finished for PR 12161 at commit e30d40d.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 6, 2016

Test build #55104 has finished for PR 12161 at commit 13b6b44.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's simplify this. The generated code only needs findOrInsert() and doesn't need find.

@SparkQA
Copy link

SparkQA commented Apr 7, 2016

Test build #55179 has finished for PR 12161 at commit cae66fd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 7, 2016

Test build #55180 has finished for PR 12161 at commit ff6ebbe.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@nongli
Copy link
Contributor

nongli commented Apr 7, 2016

The generated code takes a schema in the ctor and creates one as a member var. Let's just use the member var one like you had originally.

@sameeragarwal
Copy link
Member Author

sorry -- I hadn't updated the correct generated code in the PR description. Please let me know if this is OK or you still prefer the old version.

@nongli
Copy link
Contributor

nongli commented Apr 7, 2016

I think the old version makes more sense. The generated code only works for a particular schema so no reason to pass it in.

@sameeragarwal
Copy link
Member Author

sure, added it back

@nongli
Copy link
Contributor

nongli commented Apr 8, 2016

LGTM

@SparkQA
Copy link

SparkQA commented Apr 8, 2016

Test build #55272 has finished for PR 12161 at commit 071a900.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should document how this thing works in the classdoc (i.e. explain the physical layout).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also maybe this should be called ColumnarAggMapCodeGenerator?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

and this should be an object with a single public function, generate, which takes the constructor arguments?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added docs, renames etc. The reason I made it a class was because there was a lot of shared state that'd otherwise have to be passed around in all the functions (groupingKeys, bufferValues, groupingKeySignature etc.).

@sameeragarwal sameeragarwal changed the title [SPARK-14394][SQL] Generate AggregateHashMap class during TungstenAggregate codegen [SPARK-14394][SQL] Generate AggregateHashMap class for LongTypes during TungstenAggregate codegen Apr 8, 2016
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it'd be great to document the hash function (since it is more difficult to read the generated code)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

one thing that might be useful is to put the generated code actually in as comments.

same for the generateEquals and generateFindOrInsert

@SparkQA
Copy link

SparkQA commented Apr 8, 2016

Test build #55318 has finished for PR 12161 at commit eb8a020.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@sameeragarwal
Copy link
Member Author

test this please

@SparkQA
Copy link

SparkQA commented Apr 8, 2016

Test build #55327 has finished for PR 12161 at commit eb8a020.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@sameeragarwal
Copy link
Member Author

Seems like amp-jenkins-worker-06 is in a bad state. test this please

@SparkQA
Copy link

SparkQA commented Apr 8, 2016

Test build #55336 has finished for PR 12161 at commit eb8a020.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@sameeragarwal
Copy link
Member Author

test this please

@SparkQA
Copy link

SparkQA commented Apr 8, 2016

Test build #55354 has finished for PR 12161 at commit ec74328.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@sameeragarwal
Copy link
Member Author

test this please

2 similar comments
@sameeragarwal
Copy link
Member Author

test this please

@sameeragarwal
Copy link
Member Author

test this please

@SparkQA
Copy link

SparkQA commented Apr 8, 2016

Test build #55380 has finished for PR 12161 at commit ec74328.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@asfgit asfgit closed this in f8c9bec Apr 8, 2016
* less robust as compared to the `BytesToBytesMap` (especially for a large number of keys or even
* for certain distribution of keys) and requires us to fall back on the latter for correctness.
*/
class ColumnarAggMapCodeGenerator(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This class can be private, right ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

everything in execution is private

asfgit pushed a commit that referenced this pull request Apr 15, 2016
…dHashMap

## What changes were proposed in this pull request?

This patch speeds up group-by aggregates by around 3-5x by leveraging an in-memory `AggregateHashMap` (please see #12161), an append-only aggregate hash map that can act as a 'cache' for extremely fast key-value lookups while evaluating aggregates (and fall back to the `BytesToBytesMap` if a given key isn't found).

Architecturally, it is backed by a power-of-2-sized array for index lookups and a columnar batch that stores the key-value pairs. The index lookups in the array rely on linear probing (with a small number of maximum tries) and use an inexpensive hash function which makes it really efficient for a majority of lookups. However, using linear probing and an inexpensive hash function also makes it less robust as compared to the `BytesToBytesMap` (especially for a large number of keys or even for certain distribution of keys) and requires us to fall back on the latter for correctness.

## How was this patch tested?

    Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4
    Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz
    Aggregate w keys:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    codegen = F                              2124 / 2204          9.9         101.3       1.0X
    codegen = T hashmap = F                  1198 / 1364         17.5          57.1       1.8X
    codegen = T hashmap = T                   369 /  600         56.8          17.6       5.8X

Author: Sameer Agarwal <[email protected]>

Closes #12345 from sameeragarwal/tungsten-aggregate-integration.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants