Github user rxin commented on the pull request:
https://github.com/apache/spark/pull/11017#issuecomment-178384561
Generated code:
```java
scala> sqlContext.range(1, 1000).count()
/* 001 */
/* 002 */ public Object generate(Object[] references) {
/* 003 */ return new GeneratedIterator(references);
/* 004 */ }
/* 005 */
/* 006 */ class GeneratedIterator extends
org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */
/* 008 */ private Object[] references;
/* 009 */ private boolean agg_initAgg0;
/* 010 */ private boolean agg_bufIsNull1;
/* 011 */ private long agg_bufValue2;
/* 012 */ private boolean range_initRange6;
/* 013 */ private long range_partitionEnd7;
/* 014 */ private long range_number8;
/* 015 */ private boolean range_overflow9;
/* 016 */ private UnsafeRow agg_result19;
/* 017 */ private
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder20;
/* 018 */ private
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
agg_rowWriter21;
/* 019 */
/* 020 */ private void initRange(int idx) {
/* 021 */ java.math.BigInteger index =
java.math.BigInteger.valueOf(idx);
/* 022 */ java.math.BigInteger numSlice =
java.math.BigInteger.valueOf(1L);
/* 023 */ java.math.BigInteger numElement =
java.math.BigInteger.valueOf(999L);
/* 024 */ java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 025 */ java.math.BigInteger start = java.math.BigInteger.valueOf(1L);
/* 026 */
/* 027 */ java.math.BigInteger st =
index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 028 */ if
(st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 029 */ range_number8 = Long.MAX_VALUE;
/* 030 */ } else if
(st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 031 */ range_number8 = Long.MIN_VALUE;
/* 032 */ } else {
/* 033 */ range_number8 = st.longValue();
/* 034 */ }
/* 035 */
/* 036 */ java.math.BigInteger end =
index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 037 */ .multiply(step).add(start);
/* 038 */ if
(end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 039 */ range_partitionEnd7 = Long.MAX_VALUE;
/* 040 */ } else if
(end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 041 */ range_partitionEnd7 = Long.MIN_VALUE;
/* 042 */ } else {
/* 043 */ range_partitionEnd7 = end.longValue();
/* 044 */ }
/* 045 */ }
/* 046 */
/* 047 */
/* 048 */ private void agg_doAggregateWithoutKey5() throws
java.io.IOException {
/* 049 */ // initialize aggregation buffer
/* 050 */ /* 0 */
/* 051 */
/* 052 */ agg_bufIsNull1 = false;
/* 053 */ agg_bufValue2 = 0L;
/* 054 */
/* 055 */
/* 056 */
/* 057 */ // initialize Range
/* 058 */ if (!range_initRange6) {
/* 059 */ range_initRange6 = true;
/* 060 */ if (input.hasNext()) {
/* 061 */ initRange(((InternalRow) input.next()).getInt(0));
/* 062 */ } else {
/* 063 */ return;
/* 064 */ }
/* 065 */ }
/* 066 */
/* 067 */ while (!range_overflow9 && range_number8 <
range_partitionEnd7) {
/* 068 */ long range_value10 = range_number8;
/* 069 */ range_number8 += 1L;
/* 070 */ if (range_number8 < range_value10 ^ 1L < 0) {
/* 071 */ range_overflow9 = true;
/* 072 */ }
/* 073 */
/* 074 */
/* 075 */
/* 076 */
/* 077 */ // do aggregate
/* 078 */ /* (input[0, bigint] + 1) */
/* 079 */ /* input[0, bigint] */
/* 080 */
/* 081 */ /* 1 */
/* 082 */
/* 083 */ long agg_value12 = -1L;
/* 084 */ agg_value12 = agg_bufValue2 + 1L;
/* 085 */ // update aggregation buffer
/* 086 */
/* 087 */ agg_bufIsNull1 = false;
/* 088 */ agg_bufValue2 = agg_value12;
/* 089 */
/* 090 */
/* 091 */
/* 092 */ }
/* 093 */
/* 094 */ }
/* 095 */
/* 096 */
/* 097 */ public GeneratedIterator(Object[] references) {
/* 098 */ this.references = references;
/* 099 */ agg_initAgg0 = false;
/* 100 */
/* 101 */
/* 102 */ range_initRange6 = false;
/* 103 */ range_partitionEnd7 = 0L;
/* 104 */ range_number8 = 0L;
/* 105 */ range_overflow9 = false;
/* 106 */ agg_result19 = new UnsafeRow(1);
/* 107 */ this.agg_holder20 = new
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result19, 0);
/* 108 */ this.agg_rowWriter21 = new
org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder20,
1);
/* 109 */ }
/* 110 */
/* 111 */ protected void processNext() throws java.io.IOException {
/* 112 */
/* 113 */ if (!agg_initAgg0) {
/* 114 */ agg_initAgg0 = true;
/* 115 */ agg_doAggregateWithoutKey5();
/* 116 */
/* 117 */ // output the result
/* 118 */
/* 119 */
/* 120 */
/* 121 */ agg_rowWriter21.zeroOutNullBytes();
/* 122 */
/* 123 */ /* input[0, bigint] */
/* 124 */
/* 125 */ if (agg_bufIsNull1) {
/* 126 */ agg_rowWriter21.setNullAt(0);
/* 127 */ } else {
/* 128 */ agg_rowWriter21.write(0, agg_bufValue2);
/* 129 */ }
/* 130 */ currentRow = agg_result19;
/* 131 */ return;
/* 132 */
/* 133 */ }
/* 134 */
/* 135 */ }
/* 136 */ }
/* 137 */
```
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]