Commit graph

3168 commits

Author SHA1 Message Date
Takeshi YAMAMURO dadf0138b3 [SPARK-14259][SQL] Add a FileSourceStrategy option for limiting #files in a partition
## What changes were proposed in this pull request?
This pr is to add a config to control the maximum number of files as even small files have a non-trivial fixed cost. The current packing can put a lot of small files together which cases straggler tasks.

## How was this patch tested?
I added tests to check if many files get split into partitions in FileSourceStrategySuite.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #12068 from maropu/SPARK-14259.
2016-03-30 16:02:48 -07:00
Wenchen Fan d46c71b39d [SPARK-14268][SQL] rename toRowExpressions and fromRowExpression to serializer and deserializer in ExpressionEncoder
## What changes were proposed in this pull request?

In `ExpressionEncoder`, we use `constructorFor` to build `fromRowExpression` as the `deserializer` in `ObjectOperator`. It's kind of confusing, we should make the name consistent.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12058 from cloud-fan/rename.
2016-03-30 11:03:15 -07:00
Wenchen Fan 816f359cf0 [SPARK-14114][SQL] implement buildReader for text data source
## What changes were proposed in this pull request?

This PR implements buildReader for text data source and enable it in the new data source code path.

## How was this patch tested?

Existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11934 from cloud-fan/text.
2016-03-30 17:32:53 +08:00
gatorsmile b66b97cd04 [SPARK-14124][SQL] Implement Database-related DDL Commands
#### What changes were proposed in this pull request?
This PR is to implement the following four Database-related DDL commands:
 - `CREATE DATABASE|SCHEMA [IF NOT EXISTS] database_name`
 - `DROP DATABASE [IF EXISTS] database_name [RESTRICT|CASCADE]`
 - `DESCRIBE DATABASE [EXTENDED] db_name`
 - `ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...)`

Another PR will be submitted to handle the unsupported commands. In the Database-related DDL commands, we will issue an error exception for `ALTER (DATABASE|SCHEMA) database_name SET OWNER [USER|ROLE] user_or_role`.

cc yhuai andrewor14 rxin Could you review the changes? Is it in the right direction? Thanks!

#### How was this patch tested?
Added a few test cases in `command/DDLSuite.scala` for testing DDL command execution in `SQLContext`. Since `HiveContext` also shares the same implementation, the existing test cases in `\hive` also verifies the correctness of these commands.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12009 from gatorsmile/dbDDL.
2016-03-29 17:39:52 -07:00
Sameer Agarwal 366cac6fb0 [SPARK-14225][SQL] Cap the length of toCommentSafeString at 128 chars
## What changes were proposed in this pull request?

Builds on https://github.com/apache/spark/pull/12022 and (a) appends "..." to truncated comment strings and (b) fixes indentation in lines after the commented strings if they happen to have a `(`, `{`, `)` or `}`

## How was this patch tested?

Manually examined the generated code.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12044 from sameeragarwal/comment.
2016-03-29 16:46:45 -07:00
Davies Liu a7a93a116d [SPARK-14215] [SQL] [PYSPARK] Support chained Python UDFs
## What changes were proposed in this pull request?

This PR brings the support for chained Python UDFs, for example

```sql
select udf1(udf2(a))
select udf1(udf2(a) + 3)
select udf1(udf2(a) + udf3(b))
```

Also directly chained unary Python UDFs are put in single batch of Python UDFs, others may require multiple batches.

For example,
```python
>>> sqlContext.sql("select double(double(1))").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [pythonUDF#10 AS double(double(1))#9]
:     +- INPUT
+- !BatchPythonEvaluation double(double(1)), [pythonUDF#10]
   +- Scan OneRowRelation[]
>>> sqlContext.sql("select double(double(1) + double(2))").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [pythonUDF#19 AS double((double(1) + double(2)))#16]
:     +- INPUT
+- !BatchPythonEvaluation double((pythonUDF#17 + pythonUDF#18)), [pythonUDF#17,pythonUDF#18,pythonUDF#19]
   +- !BatchPythonEvaluation double(2), [pythonUDF#17,pythonUDF#18]
      +- !BatchPythonEvaluation double(1), [pythonUDF#17]
         +- Scan OneRowRelation[]
```

TODO: will support multiple unrelated Python UDFs in one batch (another PR).

## How was this patch tested?

Added new unit tests for chained UDFs.

Author: Davies Liu <davies@databricks.com>

Closes #12014 from davies/py_udfs.
2016-03-29 15:06:29 -07:00
Eric Liang e58c4cb3c5 [SPARK-14227][SQL] Add method for printing out generated code for debugging
## What changes were proposed in this pull request?

This adds `debugCodegen` to the debug package for query execution.

## How was this patch tested?

Unit and manual testing. Output example:

```
scala> import org.apache.spark.sql.execution.debug._
import org.apache.spark.sql.execution.debug._

scala> sqlContext.range(100).groupBy("id").count().orderBy("id").debugCodegen()
Found 3 WholeStageCodegen subtrees.
== Subtree 1 / 3 ==
WholeStageCodegen
:  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L])
:     +- Range 0, 1, 1, 100, [id#0L]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ /** Codegened pipeline for:
/* 006 */ * TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L])
/* 007 */ +- Range 0, 1, 1, 100, [id#0L]
/* 008 */ */
/* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 010 */   private Object[] references;
/* 011 */   private boolean agg_initAgg;
/* 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 agg_result;
/* 027 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 028 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 029 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowJoiner agg_unsafeRowJoiner;
/* 030 */   private org.apache.spark.sql.execution.metric.LongSQLMetric wholestagecodegen_numOutputRows;
/* 031 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue wholestagecodegen_metricValue;
/* 032 */
/* 033 */   public GeneratedIterator(Object[] references) {
/* 034 */     this.references = references;
/* 035 */   }
/* 036 */
/* 037 */   public void init(scala.collection.Iterator inputs[]) {
/* 038 */     agg_initAgg = false;
/* 039 */     this.agg_plan = (org.apache.spark.sql.execution.aggregate.TungstenAggregate) references[0];
/* 040 */     agg_hashMap = agg_plan.createHashMap();
/* 041 */
/* 042 */     this.range_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1];
/* 043 */     range_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) range_numOutputRows.localValue();
/* 044 */     range_initRange = false;
/* 045 */     range_partitionEnd = 0L;
/* 046 */     range_number = 0L;
/* 047 */     range_overflow = false;
/* 048 */     range_input = inputs[0];
/* 049 */     range_result = new UnsafeRow(1);
/* 050 */     this.range_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(range_result, 0);
/* 051 */     this.range_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(range_holder, 1);
/* 052 */     agg_result = new UnsafeRow(1);
/* 053 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 054 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 055 */     agg_unsafeRowJoiner = agg_plan.createUnsafeJoiner();
/* 056 */     this.wholestagecodegen_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[2];
/* 057 */     wholestagecodegen_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) wholestagecodegen_numOutputRows.localValue();
/* 058 */   }
/* 059 */
/* 060 */   private void agg_doAggregateWithKeys() throws java.io.IOException {
/* 061 */     /*** PRODUCE: Range 0, 1, 1, 100, [id#0L] */
/* 062 */
/* 063 */     // initialize Range
/* 064 */     if (!range_initRange) {
/* 065 */       range_initRange = true;
/* 066 */       if (range_input.hasNext()) {
/* 067 */         initRange(((InternalRow) range_input.next()).getInt(0));
/* 068 */       } else {
/* 069 */         return;
/* 070 */       }
/* 071 */     }
/* 072 */
/* 073 */     while (!range_overflow && range_number < range_partitionEnd) {
/* 074 */       long range_value = range_number;
/* 075 */       range_number += 1L;
/* 076 */       if (range_number < range_value ^ 1L < 0) {
/* 077 */         range_overflow = true;
/* 078 */       }
/* 079 */
/* 080 */       /*** CONSUME: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) */
/* 081 */
/* 082 */       // generate grouping key
/* 083 */       agg_rowWriter.write(0, range_value);
/* 084 */       /* hash(input[0, bigint], 42) */
/* 085 */       int agg_value1 = 42;
/* 086 */
/* 087 */       agg_value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(range_value, agg_value1);
/* 088 */       UnsafeRow agg_aggBuffer = null;
/* 089 */       if (true) {
/* 090 */         // try to get the buffer from hash map
/* 091 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1);
/* 092 */       }
/* 093 */       if (agg_aggBuffer == null) {
/* 094 */         if (agg_sorter == null) {
/* 095 */           agg_sorter = agg_hashMap.destructAndCreateExternalSorter();
/* 096 */         } else {
/* 097 */           agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter());
/* 098 */         }
/* 099 */
/* 100 */         // the hash map had be spilled, it should have enough memory now,
/* 101 */         // try  to allocate buffer again.
/* 102 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1);
/* 103 */         if (agg_aggBuffer == null) {
/* 104 */           // failed to allocate the first page
/* 105 */           throw new OutOfMemoryError("No enough memory for aggregation");
/* 106 */         }
/* 107 */       }
/* 108 */
/* 109 */       // evaluate aggregate function
/* 110 */       /* (input[0, bigint] + 1) */
/* 111 */       /* input[0, bigint] */
/* 112 */       long agg_value4 = agg_aggBuffer.getLong(0);
/* 113 */
/* 114 */       long agg_value3 = -1L;
/* 115 */       agg_value3 = agg_value4 + 1L;
/* 116 */       // update aggregate buffer
/* 117 */       agg_aggBuffer.setLong(0, agg_value3);
/* 118 */
/* 119 */       if (shouldStop()) return;
/* 120 */     }
/* 121 */
/* 122 */     agg_mapIter = agg_plan.finishAggregate(agg_hashMap, agg_sorter);
/* 123 */   }
/* 124 */
/* 125 */   private void initRange(int idx) {
/* 126 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
/* 127 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(1L);
/* 128 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(100L);
/* 129 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
/* 130 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
/* 131 */
/* 132 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
/* 133 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 134 */       range_number = Long.MAX_VALUE;
/* 135 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 136 */       range_number = Long.MIN_VALUE;
/* 137 */     } else {
/* 138 */       range_number = st.longValue();
/* 139 */     }
/* 140 */
/* 141 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
/* 142 */     .multiply(step).add(start);
/* 143 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
/* 144 */       range_partitionEnd = Long.MAX_VALUE;
/* 145 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
/* 146 */       range_partitionEnd = Long.MIN_VALUE;
/* 147 */     } else {
/* 148 */       range_partitionEnd = end.longValue();
/* 149 */     }
/* 150 */
/* 151 */     range_metricValue.add((range_partitionEnd - range_number) / 1L);
/* 152 */   }
/* 153 */
/* 154 */   protected void processNext() throws java.io.IOException {
/* 155 */     /*** PRODUCE: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L]) */
/* 156 */
/* 157 */     if (!agg_initAgg) {
/* 158 */       agg_initAgg = true;
/* 159 */       agg_doAggregateWithKeys();
/* 160 */     }
/* 161 */
/* 162 */     // output the result
/* 163 */     while (agg_mapIter.next()) {
/* 164 */       wholestagecodegen_metricValue.add(1);
/* 165 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 166 */       UnsafeRow agg_aggBuffer1 = (UnsafeRow) agg_mapIter.getValue();
/* 167 */
/* 168 */       UnsafeRow agg_resultRow = agg_unsafeRowJoiner.join(agg_aggKey, agg_aggBuffer1);
/* 169 */
/* 170 */       /*** CONSUME: WholeStageCodegen */
/* 171 */
/* 172 */       append(agg_resultRow);
/* 173 */
/* 174 */       if (shouldStop()) return;
/* 175 */     }
/* 176 */
/* 177 */     agg_mapIter.close();
/* 178 */     if (agg_sorter == null) {
/* 179 */       agg_hashMap.free();
/* 180 */     }
/* 181 */   }
/* 182 */ }

== Subtree 2 / 3 ==
WholeStageCodegen
:  +- Sort [id#0L ASC], true, 0
:     +- INPUT
+- Exchange rangepartitioning(id#0L ASC, 200), None
   +- WholeStageCodegen
      :  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L])
      :     +- INPUT
      +- Exchange hashpartitioning(id#0L, 200), None
         +- WholeStageCodegen
            :  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L])
            :     +- Range 0, 1, 1, 100, [id#0L]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ /** Codegened pipeline for:
/* 006 */ * Sort [id#0L ASC], true, 0
/* 007 */ +- INPUT
/* 008 */ */
/* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 010 */   private Object[] references;
/* 011 */   private boolean sort_needToSort;
/* 012 */   private org.apache.spark.sql.execution.Sort sort_plan;
/* 013 */   private org.apache.spark.sql.execution.UnsafeExternalRowSorter sort_sorter;
/* 014 */   private org.apache.spark.executor.TaskMetrics sort_metrics;
/* 015 */   private scala.collection.Iterator<UnsafeRow> sort_sortedIter;
/* 016 */   private scala.collection.Iterator inputadapter_input;
/* 017 */   private org.apache.spark.sql.execution.metric.LongSQLMetric sort_dataSize;
/* 018 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue sort_metricValue;
/* 019 */   private org.apache.spark.sql.execution.metric.LongSQLMetric sort_spillSize;
/* 020 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue sort_metricValue1;
/* 021 */
/* 022 */   public GeneratedIterator(Object[] references) {
/* 023 */     this.references = references;
/* 024 */   }
/* 025 */
/* 026 */   public void init(scala.collection.Iterator inputs[]) {
/* 027 */     sort_needToSort = true;
/* 028 */     this.sort_plan = (org.apache.spark.sql.execution.Sort) references[0];
/* 029 */     sort_sorter = sort_plan.createSorter();
/* 030 */     sort_metrics = org.apache.spark.TaskContext.get().taskMetrics();
/* 031 */
/* 032 */     inputadapter_input = inputs[0];
/* 033 */     this.sort_dataSize = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1];
/* 034 */     sort_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) sort_dataSize.localValue();
/* 035 */     this.sort_spillSize = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[2];
/* 036 */     sort_metricValue1 = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) sort_spillSize.localValue();
/* 037 */   }
/* 038 */
/* 039 */   private void sort_addToSorter() throws java.io.IOException {
/* 040 */     /*** PRODUCE: INPUT */
/* 041 */
/* 042 */     while (inputadapter_input.hasNext()) {
/* 043 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 044 */       /*** CONSUME: Sort [id#0L ASC], true, 0 */
/* 045 */
/* 046 */       sort_sorter.insertRow((UnsafeRow)inputadapter_row);
/* 047 */       if (shouldStop()) return;
/* 048 */     }
/* 049 */
/* 050 */   }
/* 051 */
/* 052 */   protected void processNext() throws java.io.IOException {
/* 053 */     /*** PRODUCE: Sort [id#0L ASC], true, 0 */
/* 054 */     if (sort_needToSort) {
/* 055 */       sort_addToSorter();
/* 056 */       Long sort_spillSizeBefore = sort_metrics.memoryBytesSpilled();
/* 057 */       sort_sortedIter = sort_sorter.sort();
/* 058 */       sort_metricValue.add(sort_sorter.getPeakMemoryUsage());
/* 059 */       sort_metricValue1.add(sort_metrics.memoryBytesSpilled() - sort_spillSizeBefore);
/* 060 */       sort_metrics.incPeakExecutionMemory(sort_sorter.getPeakMemoryUsage());
/* 061 */       sort_needToSort = false;
/* 062 */     }
/* 063 */
/* 064 */     while (sort_sortedIter.hasNext()) {
/* 065 */       UnsafeRow sort_outputRow = (UnsafeRow)sort_sortedIter.next();
/* 066 */
/* 067 */       /*** CONSUME: WholeStageCodegen */
/* 068 */
/* 069 */       append(sort_outputRow);
/* 070 */
/* 071 */       if (shouldStop()) return;
/* 072 */     }
/* 073 */   }
/* 074 */ }

== Subtree 3 / 3 ==
WholeStageCodegen
:  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L])
:     +- INPUT
+- Exchange hashpartitioning(id#0L, 200), None
   +- WholeStageCodegen
      :  +- TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Partial,isDistinct=false)], output=[id#0L,count#9L])
      :     +- Range 0, 1, 1, 100, [id#0L]

Generated code:
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ /** Codegened pipeline for:
/* 006 */ * TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L])
/* 007 */ +- INPUT
/* 008 */ */
/* 009 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 010 */   private Object[] references;
/* 011 */   private boolean agg_initAgg;
/* 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 scala.collection.Iterator inputadapter_input;
/* 017 */   private UnsafeRow agg_result;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder;
/* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter;
/* 020 */   private UnsafeRow agg_result1;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder1;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter1;
/* 023 */   private org.apache.spark.sql.execution.metric.LongSQLMetric wholestagecodegen_numOutputRows;
/* 024 */   private org.apache.spark.sql.execution.metric.LongSQLMetricValue wholestagecodegen_metricValue;
/* 025 */
/* 026 */   public GeneratedIterator(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(scala.collection.Iterator inputs[]) {
/* 031 */     agg_initAgg = false;
/* 032 */     this.agg_plan = (org.apache.spark.sql.execution.aggregate.TungstenAggregate) references[0];
/* 033 */     agg_hashMap = agg_plan.createHashMap();
/* 034 */
/* 035 */     inputadapter_input = inputs[0];
/* 036 */     agg_result = new UnsafeRow(1);
/* 037 */     this.agg_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result, 0);
/* 038 */     this.agg_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder, 1);
/* 039 */     agg_result1 = new UnsafeRow(2);
/* 040 */     this.agg_holder1 = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result1, 0);
/* 041 */     this.agg_rowWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_holder1, 2);
/* 042 */     this.wholestagecodegen_numOutputRows = (org.apache.spark.sql.execution.metric.LongSQLMetric) references[1];
/* 043 */     wholestagecodegen_metricValue = (org.apache.spark.sql.execution.metric.LongSQLMetricValue) wholestagecodegen_numOutputRows.localValue();
/* 044 */   }
/* 045 */
/* 046 */   private void agg_doAggregateWithKeys() throws java.io.IOException {
/* 047 */     /*** PRODUCE: INPUT */
/* 048 */
/* 049 */     while (inputadapter_input.hasNext()) {
/* 050 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 051 */       /*** CONSUME: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) */
/* 052 */       /* input[0, bigint] */
/* 053 */       long inputadapter_value = inputadapter_row.getLong(0);
/* 054 */       /* input[1, bigint] */
/* 055 */       long inputadapter_value1 = inputadapter_row.getLong(1);
/* 056 */
/* 057 */       // generate grouping key
/* 058 */       agg_rowWriter.write(0, inputadapter_value);
/* 059 */       /* hash(input[0, bigint], 42) */
/* 060 */       int agg_value1 = 42;
/* 061 */
/* 062 */       agg_value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(inputadapter_value, agg_value1);
/* 063 */       UnsafeRow agg_aggBuffer = null;
/* 064 */       if (true) {
/* 065 */         // try to get the buffer from hash map
/* 066 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1);
/* 067 */       }
/* 068 */       if (agg_aggBuffer == null) {
/* 069 */         if (agg_sorter == null) {
/* 070 */           agg_sorter = agg_hashMap.destructAndCreateExternalSorter();
/* 071 */         } else {
/* 072 */           agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter());
/* 073 */         }
/* 074 */
/* 075 */         // the hash map had be spilled, it should have enough memory now,
/* 076 */         // try  to allocate buffer again.
/* 077 */         agg_aggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(agg_result, agg_value1);
/* 078 */         if (agg_aggBuffer == null) {
/* 079 */           // failed to allocate the first page
/* 080 */           throw new OutOfMemoryError("No enough memory for aggregation");
/* 081 */         }
/* 082 */       }
/* 083 */
/* 084 */       // evaluate aggregate function
/* 085 */       /* (input[0, bigint] + input[2, bigint]) */
/* 086 */       /* input[0, bigint] */
/* 087 */       long agg_value4 = agg_aggBuffer.getLong(0);
/* 088 */
/* 089 */       long agg_value3 = -1L;
/* 090 */       agg_value3 = agg_value4 + inputadapter_value1;
/* 091 */       // update aggregate buffer
/* 092 */       agg_aggBuffer.setLong(0, agg_value3);
/* 093 */       if (shouldStop()) return;
/* 094 */     }
/* 095 */
/* 096 */     agg_mapIter = agg_plan.finishAggregate(agg_hashMap, agg_sorter);
/* 097 */   }
/* 098 */
/* 099 */   protected void processNext() throws java.io.IOException {
/* 100 */     /*** PRODUCE: TungstenAggregate(key=[id#0L], functions=[(count(1),mode=Final,isDistinct=false)], output=[id#0L,count#4L]) */
/* 101 */
/* 102 */     if (!agg_initAgg) {
/* 103 */       agg_initAgg = true;
/* 104 */       agg_doAggregateWithKeys();
/* 105 */     }
/* 106 */
/* 107 */     // output the result
/* 108 */     while (agg_mapIter.next()) {
/* 109 */       wholestagecodegen_metricValue.add(1);
/* 110 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 111 */       UnsafeRow agg_aggBuffer1 = (UnsafeRow) agg_mapIter.getValue();
/* 112 */
/* 113 */       /* input[0, bigint] */
/* 114 */       long agg_value6 = agg_aggKey.getLong(0);
/* 115 */       /* input[0, bigint] */
/* 116 */       long agg_value7 = agg_aggBuffer1.getLong(0);
/* 117 */
/* 118 */       /*** CONSUME: WholeStageCodegen */
/* 119 */
/* 120 */       agg_rowWriter1.write(0, agg_value6);
/* 121 */
/* 122 */       agg_rowWriter1.write(1, agg_value7);
/* 123 */       append(agg_result1);
/* 124 */
/* 125 */       if (shouldStop()) return;
/* 126 */     }
/* 127 */
/* 128 */     agg_mapIter.close();
/* 129 */     if (agg_sorter == null) {
/* 130 */       agg_hashMap.free();
/* 131 */     }
/* 132 */   }
/* 133 */ }
```

rxin

Author: Eric Liang <ekl@databricks.com>

Closes #12025 from ericl/spark-14227.
2016-03-29 13:31:51 -07:00
Dongjoon Hyun 838cb4583d [MINOR][SQL] Fix exception message to print string-array correctly.
## What changes were proposed in this pull request?

This PR is a simple fix for an exception message to print `string[]` content correctly.
```java
String[] colPath = requestedSchema.getPaths().get(i);
...
-          throw new IOException("Required column is missing in data file. Col: " + colPath);
+          throw new IOException("Required column is missing in data file. Col: " + Arrays.toString(colPath));
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12041 from dongjoon-hyun/fix_exception_message_with_string_array.
2016-03-29 12:47:30 -07:00
Dongjoon Hyun d612228eff [MINOR][SQL] Fix typos by replacing 'much' with 'match'.
## What changes were proposed in this pull request?

This PR fixes two trivial typos: 'does not **much**' --> 'does not **match**'.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12042 from dongjoon-hyun/fix_typo_by_replacing_much_with_match.
2016-03-29 12:45:43 -07:00
Cheng Lian a632bb56f8 [SPARK-14208][SQL] Renames spark.sql.parquet.fileScan
## What changes were proposed in this pull request?

Renames SQL option `spark.sql.parquet.fileScan` since now all `HadoopFsRelation` based data sources are being migrated to `FileScanRDD` code path.

## How was this patch tested?

None.

Author: Cheng Lian <lian@databricks.com>

Closes #12003 from liancheng/spark-14208-option-renaming.
2016-03-29 20:56:01 +08:00
Wenchen Fan 83775bc78e [SPARK-14158][SQL] implement buildReader for json data source
## What changes were proposed in this pull request?

This PR implements buildReader for json data source and enable it in the new data source code path.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11960 from cloud-fan/json.
2016-03-29 14:34:12 +08:00
Nong Li a180286b79 [SPARK-14210] [SQL] Add a metric for time spent in scans.
## What changes were proposed in this pull request?

This adds a metric to parquet scans that measures the time in just the scan phase. This is
only possible when the scan returns ColumnarBatches, otherwise the overhead is too high.

This combined with the pipeline metric lets us easily see what percent of the time was
in the scan.

Author: Nong Li <nong@databricks.com>

Closes #12007 from nongli/spark-14210.
2016-03-28 21:37:46 -07:00
Nong Li 4a55c33639 [SPARK-13981][SQL] Defer evaluating variables within Filter operator.
## What changes were proposed in this pull request?

This improves the Filter codegen for NULLs by deferring loading the values for IsNotNull.
Instead of generating code like:

boolean isNull = ...
int value = ...
if (isNull) continue;

we will generate:
boolean isNull = ...
if (isNull) continue;
int value = ...

This is useful since retrieving the values can be non-trivial (they can be dictionary encoded
among other things). This currently only works when the attribute comes from the column batch
but could be extended to other cases in the future.

## How was this patch tested?

On tpcds q55, this fixes the regression from introducing the IsNotNull predicates.

```
TPCDS Snappy:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
--------------------------------------------------------------------------------
q55                                      4564 / 5036         25.2          39.6
q55                                      4064 / 4340         28.3          35.3
```

Author: Nong Li <nong@databricks.com>

Closes #11792 from nongli/spark-13981.
2016-03-28 20:32:58 -07:00
Herman van Hovell 27d4ef0c61 [SPARK-14213][SQL] Migrate HiveQl parsing to ANTLR4 parser
### What changes were proposed in this pull request?

This PR migrates all HiveQl parsing to the new ANTLR4 parser. This PR is build on top of https://github.com/apache/spark/pull/12011, and we should wait with merging until that one is in (hence the WIP tag).

As soon as this PR is merged we can start removing much of the old parser infrastructure.

### How was this patch tested?

Exisiting Hive unit tests.

cc rxin andrewor14 yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #12015 from hvanhovell/SPARK-14213.
2016-03-28 20:19:21 -07:00
Wenchen Fan 38326cad87 [SPARK-14205][SQL] remove trait Queryable
## What changes were proposed in this pull request?

After DataFrame and Dataset are merged, the trait `Queryable` becomes unnecessary as it has only one implementation. We should remove it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12001 from cloud-fan/df-ds.
2016-03-28 18:53:47 -07:00
Andrew Or a916d2a454 [SPARK-14119][SPARK-14120][SPARK-14122][SQL] Throw exception on unsupported DDL commands
## What changes were proposed in this pull request?

Before: We just pass all role commands to Hive even though it doesn't work.
After: We throw an `AnalysisException` that looks like this:

```
scala> sql("CREATE ROLE x")
org.apache.spark.sql.AnalysisException: Unsupported Hive operation: CREATE ROLE;
  at org.apache.spark.sql.hive.HiveQl$$anonfun$parsePlan$1.apply(HiveQl.scala:213)
  at org.apache.spark.sql.hive.HiveQl$$anonfun$parsePlan$1.apply(HiveQl.scala:208)
  at org.apache.spark.sql.catalyst.parser.CatalystQl.safeParse(CatalystQl.scala:49)
  at org.apache.spark.sql.hive.HiveQl.parsePlan(HiveQl.scala:208)
  at org.apache.spark.sql.SQLContext.parseSql(SQLContext.scala:198)
```

## How was this patch tested?

`HiveQuerySuite`

Author: Andrew Or <andrew@databricks.com>

Closes #11948 from andrewor14/ddl-role-management.
2016-03-28 16:45:31 -07:00
Andrew Or 27aab80695 [SPARK-14013][SQL] Proper temp function support in catalog
## What changes were proposed in this pull request?

Session catalog was added in #11750. However, it doesn't really support temporary functions properly; right now we only store the metadata in the form of `CatalogFunction`, but this doesn't make sense for temporary functions because there is no class name.

This patch moves the `FunctionRegistry` into the `SessionCatalog`. With this, the user can call `catalog.createTempFunction` and `catalog.lookupFunction` to use the function they registered previously. This is currently still dead code, however.

## How was this patch tested?

`SessionCatalogSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #11972 from andrewor14/temp-functions.
2016-03-28 16:45:02 -07:00
Shixiong Zhu 2f98ee67df [SPARK-14169][CORE] Add UninterruptibleThread
## What changes were proposed in this pull request?

Extract the workaround for HADOOP-10622 introduced by #11940 into UninterruptibleThread so that we can test and reuse it.

## How was this patch tested?

Unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11971 from zsxwing/uninterrupt.
2016-03-28 16:29:11 -07:00
Reynold Xin b7836492bb [SPARK-14155][SQL] Hide UserDefinedType interface in Spark 2.0
## What changes were proposed in this pull request?
UserDefinedType is a developer API in Spark 1.x. With very high probability we will create a new API for user-defined type that also works well with column batches as well as encoders (datasets). In Spark 2.0, let's make `UserDefinedType` `private[spark]` first.

## How was this patch tested?
Existing unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11955 from rxin/SPARK-14155.
2016-03-28 16:26:32 -07:00
Andrew Or eebc8c1c95 [SPARK-13923][SPARK-14014][SQL] Session catalog follow-ups
## What changes were proposed in this pull request?

This patch addresses the remaining comments left in #11750 and #11918 after they are merged. For a full list of changes in this patch, just trace the commits.

## How was this patch tested?

`SessionCatalogSuite` and `CatalogTestCases`

Author: Andrew Or <andrew@databricks.com>

Closes #12006 from andrewor14/session-catalog-followup.
2016-03-28 16:25:15 -07:00
Herman van Hovell 328c71161b [SPARK-14086][SQL] Add DDL commands to ANTLR4 parser
#### What changes were proposed in this pull request?

This PR adds all the current Spark SQL DDL commands to the new ANTLR 4 based SQL parser.

I have found a few inconsistencies in the current commands:
- Function has an alias field. This is actually the class name of the function.
- Partition specifications should contain nulls in some commands, and contain `None`s in others.
- `AlterTableSkewedLocation`: Should defines which columns have skewed values, and should allow us to define storage for each skewed combination of values. We currently only allow one value per field.
- `AlterTableSetFileFormat`: Should only have one file format, it currently supports both.

I have implemented all these comments like they were, and I propose to improve them in follow-up PRs.

#### How was this patch tested?

The existing DDLCommandSuite.

cc rxin andrewor14 yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #12011 from hvanhovell/SPARK-14086.
2016-03-28 16:22:02 -07:00
Yin Huai 7007f72ba7 [SPARK-13713][SQL][TEST-MAVEN] Add Antlr4 maven plugin.
Seems 600c0b69ca is missing the antlr4 maven plugin. This pr adds it.

Author: Yin Huai <yhuai@databricks.com>

Closes #12010 from yhuai/mavenAntlr4.
2016-03-28 13:50:42 -07:00
Davies Liu d7b58f1461 [SPARK-14052] [SQL] build a BytesToBytesMap directly in HashedRelation
## What changes were proposed in this pull request?

Currently, for the key that can not fit within a long,  we build a hash map for UnsafeHashedRelation, it's converted to BytesToBytesMap after serialization and deserialization. We should build a BytesToBytesMap directly to have better memory efficiency.

In order to do that, BytesToBytesMap should support multiple (K,V) pair with the same K,  Location.putNewKey() is renamed to Location.append(), which could append multiple values for the same key (same Location). `Location.newValue()` is added to find the next value for the same key.

## How was this patch tested?

Existing tests. Added benchmark for broadcast hash join with duplicated keys.

Author: Davies Liu <davies@databricks.com>

Closes #11870 from davies/map2.
2016-03-28 13:07:32 -07:00
Herman van Hovell 600c0b69ca [SPARK-13713][SQL] Migrate parser from ANTLR3 to ANTLR4
### What changes were proposed in this pull request?
The current ANTLR3 parser is quite complex to maintain and suffers from code blow-ups. This PR introduces a new parser that is based on ANTLR4.

This parser is based on the [Presto's SQL parser](https://github.com/facebook/presto/blob/master/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4). The current implementation can parse and create Catalyst and SQL plans. Large parts of the HiveQl DDL and some of the DML functionality is currently missing, the plan is to add this in follow-up PRs.

This PR is a work in progress, and work needs to be done in the following area's:

- [x] Error handling should be improved.
- [x] Documentation should be improved.
- [x] Multi-Insert needs to be tested.
- [ ] Naming and package locations.

### How was this patch tested?

Catalyst and SQL unit tests.

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #11557 from hvanhovell/ngParser.
2016-03-28 12:31:12 -07:00
Liang-Chi Hsieh 1528ff4c9a [SPARK-14156][SQL] Use executedPlan in HiveComparisonTest for the messages of computed tables
## What changes were proposed in this pull request?
JIRA: https://issues.apache.org/jira/browse/SPARK-14156

In HiveComparisonTest, when catalyst results are different to hive results, we will collect the messages for computed tables during the test. During creating the message, we use sparkPlan. But we actually run the query with executedPlan. So the error message is sometimes confusing.

For example, as wholestage codegen is enabled by default now. The shown spark plan for computed tables is the plan before wholestage codegen.

A concrete is the following error message shown before this patch. It is the error shown when running `HiveCompatibilityTest` `auto_join26`.

auto_join26 has one SQL to create table:

    INSERT OVERWRITE TABLE dest_j1
    SELECT  x.key, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key;   (1)

Then a SQL to retrieve the result:

    select * from dest_j1 x order by x.key;   (2)

When the above SQL (2) to retrieve the result fails, In `HiveComparisonTest` we will try to collect and show the generated data from table `dest_j1` using the SQL (1)'s spark plan. The you will see this error:

    TungstenAggregate(key=[key#8804], functions=[(count(1),mode=Partial,isDistinct=false)], output=[key#8804,count#8834L])
    +- Project [key#8804]
       +- BroadcastHashJoin [key#8804], [key#8806], Inner, BuildRight, None
          :- Filter isnotnull(key#8804)
          :  +- InMemoryColumnarTableScan [key#8804], [isnotnull(key#8804)], InMemoryRelation [key#8804,value#8805], true, 5, StorageLevel(true, true, false, true, 1), HiveTableScan [key#8717,value#8718], MetastoreRelation default, src1, None, Some(src1)
          +- Filter isnotnull(key#8806)
             +- InMemoryColumnarTableScan [key#8806], [isnotnull(key#8806)], InMemoryRelation [key#8806,value#8807], true, 5, StorageLevel(true, true, false, true, 1), HiveTableScan [key#8760,value#8761], MetastoreRelation default, src, None, Some(src)

	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:47)
	at org.apache.spark.sql.execution.aggregate.TungstenAggregate.doExecute(TungstenAggregate.scala:82)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:121)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:121)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:140)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:137)
	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:120)
	at org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1.apply(TungstenAggregate.scala:87)
	at org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1.apply(TungstenAggregate.scala:82)
	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:46)
	... 70 more
    Caused by: java.lang.UnsupportedOperationException: Filter does not implement doExecuteBroadcast
	at org.apache.spark.sql.execution.SparkPlan.doExecuteBroadcast(SparkPlan.scala:221)

The message is confusing because it is not the plan actually run by SparkSQL engine to create the generated table. The plan actually run is no problem. But as before this patch, we run `e.sparkPlan.collect` to retrieve and show the generated data, spark plan is not the plan we can run. So the above error will be shown.

After this patch, we won't see the error because the executed plan is no problem and works.

## How was this patch tested?
Existing tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #11957 from viirya/use-executedplan.
2016-03-28 10:43:54 -07:00
Kazuaki Ishizaki 4a7636f2da [SPARK-13844] [SQL] Generate better code for filters with a non-nullable column
## What changes were proposed in this pull request?

This PR simplifies generated code with a non-nullable column. This PR addresses three items:
1. Generate simplified code for and / or
2. Generate better code for divide and remainder with non-zero dividend
3. Pass nullable information into BoundReference at WholeStageCodegen

I have attached the generated code with and without this PR

## How was this patch tested?

Tested by existing test suites in sql/core

Here is a motivating example
````
(0 to 6).map(i => (i.toString, i.toInt)).toDF("k", "v")
  .filter("v % 2 == 0").filter("v <= 4").filter("v > 1").show()
````

Generated code without this PR
````java
/* 032 */   protected void processNext() throws java.io.IOException {
/* 033 */     /*** PRODUCE: Project [_1#0 AS k#3,_2#1 AS v#4] */
/* 034 */
/* 035 */     /*** PRODUCE: Filter ((isnotnull((_2#1 % 2)) && ((_2#1 % 2) = 0)) && ((_2#1 <= 4) && (_2#1 > 1))) */
/* 036 */
/* 037 */     /*** PRODUCE: INPUT */
/* 038 */
/* 039 */     while (!shouldStop() && inputadapter_input.hasNext()) {
/* 040 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 041 */       /*** CONSUME: Filter ((isnotnull((_2#1 % 2)) && ((_2#1 % 2) = 0)) && ((_2#1 <= 4) && (_2#1 > 1))) */
/* 042 */       /* input[1, int] */
/* 043 */       int filter_value1 = inputadapter_row.getInt(1);
/* 044 */
/* 045 */       /* isnotnull((input[1, int] % 2)) */
/* 046 */       /* (input[1, int] % 2) */
/* 047 */       boolean filter_isNull3 = false;
/* 048 */       int filter_value3 = -1;
/* 049 */       if (false || 2 == 0) {
/* 050 */         filter_isNull3 = true;
/* 051 */       } else {
/* 052 */         if (false) {
/* 053 */           filter_isNull3 = true;
/* 054 */         } else {
/* 055 */           filter_value3 = (int)(filter_value1 % 2);
/* 056 */         }
/* 057 */       }
/* 058 */       if (!(!(filter_isNull3))) continue;
/* 059 */
/* 060 */       /* ((input[1, int] % 2) = 0) */
/* 061 */       boolean filter_isNull6 = true;
/* 062 */       boolean filter_value6 = false;
/* 063 */       /* (input[1, int] % 2) */
/* 064 */       boolean filter_isNull7 = false;
/* 065 */       int filter_value7 = -1;
/* 066 */       if (false || 2 == 0) {
/* 067 */         filter_isNull7 = true;
/* 068 */       } else {
/* 069 */         if (false) {
/* 070 */           filter_isNull7 = true;
/* 071 */         } else {
/* 072 */           filter_value7 = (int)(filter_value1 % 2);
/* 073 */         }
/* 074 */       }
/* 075 */       if (!filter_isNull7) {
/* 076 */         filter_isNull6 = false; // resultCode could change nullability.
/* 077 */         filter_value6 = filter_value7 == 0;
/* 078 */
/* 079 */       }
/* 080 */       if (filter_isNull6 || !filter_value6) continue;
/* 081 */
/* 082 */       /* (input[1, int] <= 4) */
/* 083 */       boolean filter_value11 = false;
/* 084 */       filter_value11 = filter_value1 <= 4;
/* 085 */       if (!filter_value11) continue;
/* 086 */
/* 087 */       /* (input[1, int] > 1) */
/* 088 */       boolean filter_value14 = false;
/* 089 */       filter_value14 = filter_value1 > 1;
/* 090 */       if (!filter_value14) continue;
/* 091 */
/* 092 */       filter_metricValue.add(1);
/* 093 */
/* 094 */       /*** CONSUME: Project [_1#0 AS k#3,_2#1 AS v#4] */
/* 095 */
/* 096 */       /* input[0, string] */
/* 097 */       /* input[0, string] */
/* 098 */       boolean filter_isNull = inputadapter_row.isNullAt(0);
/* 099 */       UTF8String filter_value = filter_isNull ? null : (inputadapter_row.getUTF8String(0));
/* 100 */       project_holder.reset();
/* 101 */
/* 102 */       project_rowWriter.zeroOutNullBytes();
/* 103 */
/* 104 */       if (filter_isNull) {
/* 105 */         project_rowWriter.setNullAt(0);
/* 106 */       } else {
/* 107 */         project_rowWriter.write(0, filter_value);
/* 108 */       }
/* 109 */
/* 110 */       project_rowWriter.write(1, filter_value1);
/* 111 */       project_result.setTotalSize(project_holder.totalSize());
/* 112 */       append(project_result.copy());
/* 113 */     }
/* 114 */   }
/* 115 */ }
````

Generated code with this PR
````java
/* 032 */   protected void processNext() throws java.io.IOException {
/* 033 */     /*** PRODUCE: Project [_1#0 AS k#3,_2#1 AS v#4] */
/* 034 */
/* 035 */     /*** PRODUCE: Filter (((_2#1 % 2) = 0) && ((_2#1 <= 5) && (_2#1 > 1))) */
/* 036 */
/* 037 */     /*** PRODUCE: INPUT */
/* 038 */
/* 039 */     while (!shouldStop() && inputadapter_input.hasNext()) {
/* 040 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 041 */       /*** CONSUME: Filter (((_2#1 % 2) = 0) && ((_2#1 <= 5) && (_2#1 > 1))) */
/* 042 */       /* input[1, int] */
/* 043 */       int filter_value1 = inputadapter_row.getInt(1);
/* 044 */
/* 045 */       /* ((input[1, int] % 2) = 0) */
/* 046 */       /* (input[1, int] % 2) */
/* 047 */       int filter_value3 = (int)(filter_value1 % 2);
/* 048 */
/* 049 */       boolean filter_value2 = false;
/* 050 */       filter_value2 = filter_value3 == 0;
/* 051 */       if (!filter_value2) continue;
/* 052 */
/* 053 */       /* (input[1, int] <= 5) */
/* 054 */       boolean filter_value7 = false;
/* 055 */       filter_value7 = filter_value1 <= 5;
/* 056 */       if (!filter_value7) continue;
/* 057 */
/* 058 */       /* (input[1, int] > 1) */
/* 059 */       boolean filter_value10 = false;
/* 060 */       filter_value10 = filter_value1 > 1;
/* 061 */       if (!filter_value10) continue;
/* 062 */
/* 063 */       filter_metricValue.add(1);
/* 064 */
/* 065 */       /*** CONSUME: Project [_1#0 AS k#3,_2#1 AS v#4] */
/* 066 */
/* 067 */       /* input[0, string] */
/* 068 */       /* input[0, string] */
/* 069 */       boolean filter_isNull = inputadapter_row.isNullAt(0);
/* 070 */       UTF8String filter_value = filter_isNull ? null : (inputadapter_row.getUTF8String(0));
/* 071 */       project_holder.reset();
/* 072 */
/* 073 */       project_rowWriter.zeroOutNullBytes();
/* 074 */
/* 075 */       if (filter_isNull) {
/* 076 */         project_rowWriter.setNullAt(0);
/* 077 */       } else {
/* 078 */         project_rowWriter.write(0, filter_value);
/* 079 */       }
/* 080 */
/* 081 */       project_rowWriter.write(1, filter_value1);
/* 082 */       project_result.setTotalSize(project_holder.totalSize());
/* 083 */       append(project_result.copy());
/* 084 */     }
/* 085 */   }
/* 086 */ }
````

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #11684 from kiszk/SPARK-13844.
2016-03-28 10:35:48 -07:00
Kousuke Saruta aac13fb48c [SPARK-14185][SQL][MINOR] Make indentation of debug log for generated code proper
## What changes were proposed in this pull request?

The indentation of debug log output by `CodeGenerator` is weird.
The first line of the generated code should be put on the next line of the first line of the log message.

```
16/03/28 11:10:24 DEBUG CodeGenerator: /* 001 */
/* 002 */ public java.lang.Object generate(Object[] references) {
/* 003 */   return new SpecificSafeProjection(references);
...
```

After this patch is applied, we get debug log like as follows.

```
16/03/28 10:45:50 DEBUG CodeGenerator:
/* 001 */
/* 002 */ public java.lang.Object generate(Object[] references) {
/* 003 */   return new SpecificSafeProjection(references);
...
```
## How was this patch tested?

Ran some jobs and checked debug logs.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #11990 from sarutak/fix-debuglog-indentation.
2016-03-27 23:50:23 -07:00
Dongjoon Hyun cfcca732b4 [MINOR][SQL] Fix substr/substring testcases.
## What changes were proposed in this pull request?

This PR fixes the following two testcases in order to test the correct usages.
```
checkSqlGeneration("SELECT substr('This is a test', 'is')")
checkSqlGeneration("SELECT substring('This is a test', 'is')")
```

Actually, the testcases works but tests on exceptional cases.
```
scala> sql("SELECT substr('This is a test', 'is')")
res0: org.apache.spark.sql.DataFrame = [substring(This is a test, CAST(is AS INT), 2147483647): string]

scala> sql("SELECT substr('This is a test', 'is')").collect()
res1: Array[org.apache.spark.sql.Row] = Array([null])
```

## How was this patch tested?

Pass the modified unit tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11963 from dongjoon-hyun/fix_substr_testcase.
2016-03-27 20:06:02 +01:00
gatorsmile a01b6a92b5 [SPARK-14177][SQL] Native Parsing for DDL Command "Describe Database" and "Alter Database"
#### What changes were proposed in this pull request?

This PR is to provide native parsing support for two DDL commands:  ```Describe Database``` and ```Alter Database Set Properties```

Based on the Hive DDL document:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL

##### 1. ALTER DATABASE
**Syntax:**
```SQL
ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...)
```
 - `ALTER DATABASE` is to add new (key, value) pairs into `DBPROPERTIES`

##### 2. DESCRIBE DATABASE
**Syntax:**
```SQL
DESCRIBE DATABASE [EXTENDED] db_name
```
 - `DESCRIBE DATABASE` shows the name of the database, its comment (if one has been set), and its root location on the filesystem. When `extended` is true, it also shows the database's properties

#### How was this patch tested?
Added the related test cases to `DDLCommandSuite`

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

This patch had conflicts when merged, resolved by
Committer: Yin Huai <yhuai@databricks.com>

Closes #11977 from gatorsmile/parseAlterDatabase.
2016-03-26 20:12:30 -07:00
Liang-Chi Hsieh bc925b73a6 [SPARK-14157][SQL] Parse Drop Function DDL command
## What changes were proposed in this pull request?
JIRA: https://issues.apache.org/jira/browse/SPARK-14157

We only parse create function command. In order to support native drop function command, we need to parse it too.

From Hive [manual](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Create/Drop/ReloadFunction), the drop function command has syntax as:

DROP [TEMPORARY] FUNCTION [IF EXISTS] function_name;

## How was this patch tested?

Added test into `DDLCommandSuite`.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #11959 from viirya/parse-drop-func.
2016-03-26 20:09:01 -07:00
Cheng Lian b547de8a60 [SPARK-14116][SQL] Implements buildReader() for ORC data source
## What changes were proposed in this pull request?

This PR implements `FileFormat.buildReader()` for our ORC data source. It also fixed several minor styling issues related to `HadoopFsRelation` planning code path.

Note that `OrcNewInputFormat` doesn't rely on `OrcNewSplit` for creating `OrcRecordReader`s, plain `FileSplit` is just fine. That's why we can simply create the record reader with the help of `OrcNewInputFormat` and `FileSplit`.

## How was this patch tested?

Existing test cases should do the work

Author: Cheng Lian <lian@databricks.com>

Closes #11936 from liancheng/spark-14116-build-reader-for-orc.
2016-03-26 16:10:35 -07:00
gatorsmile 8989d3a396 [SPARK-14161][SQL] Native Parsing for DDL Command Drop Database
### What changes were proposed in this pull request?
Based on the Hive DDL document https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL

The syntax of DDL command for Drop Database is
```SQL
DROP (DATABASE|SCHEMA) [IF EXISTS] database_name [RESTRICT|CASCADE];
```
 - If `IF EXISTS` is not specified, the default behavior is to issue a warning message if `database_name` does't exist
 - `RESTRICT` is the default behavior.

This PR is to provide a native parsing support for `DROP DATABASE`.

#### How was this patch tested?

Added a test case `DDLCommandSuite`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11962 from gatorsmile/parseDropDatabase.
2016-03-26 14:11:13 -07:00
Davies Liu bd94ea4c80 [SPARK-14175][SQL] whole stage codegen interface refactor
## What changes were proposed in this pull request?

1. merge consumeChild into consume()
2. always generate code for input variables and UnsafeRow, a plan can use eight of them.

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #11975 from davies/gen_refactor.
2016-03-26 11:03:05 -07:00
Dongjoon Hyun 1808465855 [MINOR] Fix newly added java-lint errors
## What changes were proposed in this pull request?

This PR fixes some newly added java-lint errors(unused-imports, line-lengsth).

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11968 from dongjoon-hyun/SPARK-14167.
2016-03-26 11:55:49 +00:00
Tathagata Das 13945dd83b [SPARK-14109][SQL] Fix HDFSMetadataLog to fallback from FileContext to FileSystem API
## What changes were proposed in this pull request?

HDFSMetadataLog uses newer FileContext API to achieve atomic renaming. However, FileContext implementations may not exist for many scheme for which there may be FileSystem implementations. In those cases, rather than failing completely, we should fallback to the FileSystem based implementation, and log warning that there may be file consistency issues in case the log directory is concurrently modified.

In addition I have also added more tests to increase the code coverage.

## How was this patch tested?

Unit test.
Tested on cluster with custom file system.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #11925 from tdas/SPARK-14109.
2016-03-25 20:07:54 -07:00
Shixiong Zhu 24587ce433 [SPARK-14073][STREAMING][TEST-MAVEN] Move flume back to Spark
## What changes were proposed in this pull request?

This PR moves flume back to Spark as per the discussion in the dev mail-list.

## How was this patch tested?

Existing Jenkins tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11895 from zsxwing/move-flume-back.
2016-03-25 17:37:16 -07:00
Shixiong Zhu b554b3c46b [SPARK-14131][SQL] Add a workaround for HADOOP-10622 to fix DataFrameReaderWriterSuite
## What changes were proposed in this pull request?

There is a potential dead-lock in Hadoop Shell.runCommand before 2.5.0 ([HADOOP-10622](https://issues.apache.org/jira/browse/HADOOP-10622)). If we interrupt some thread running Shell.runCommand, we may hit this issue.

This PR adds some protecion to prevent from interrupting the microBatchThread when we may run into Shell.runCommand. There are two places will call Shell.runCommand now:

- offsetLog.add
- FileStreamSource.getOffset

They will create a file using HDFS API and call Shell.runCommand to set the file permission.

## How was this patch tested?

Existing unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11940 from zsxwing/workaround-for-HADOOP-10622.
2016-03-25 13:28:26 -07:00
Sameer Agarwal afd0debe07 [SPARK-14137] [SPARK-14150] [SQL] Infer IsNotNull constraints from non-nullable attributes
## What changes were proposed in this pull request?

This PR adds support for automatically inferring `IsNotNull` constraints from any non-nullable attributes that are part of an operator's output. This also fixes the issue that causes the optimizer to hit the maximum number of iterations for certain queries in https://github.com/apache/spark/pull/11828.

## How was this patch tested?

Unit test in `ConstraintPropagationSuite`

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11953 from sameeragarwal/infer-isnotnull.
2016-03-25 12:57:26 -07:00
Liang-Chi Hsieh ca003354da [SPARK-12443][SQL] encoderFor should support Decimal
## What changes were proposed in this pull request?

JIRA: https://issues.apache.org/jira/browse/SPARK-12443

`constructorFor` will call `dataTypeFor` to determine if a type is `ObjectType` or not. If there is not case for `Decimal`, it will be recognized as `ObjectType` and causes the bug.

## How was this patch tested?

Test is added into `ExpressionEncoderSuite`.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #10399 from viirya/fix-encoder-decimal.
2016-03-25 12:07:56 -07:00
Tathagata Das 11fa8741ca [SQL][HOTFIX] Fix flakiness in StateStoreRDDSuite
## What changes were proposed in this pull request?
StateStoreCoordinator.reportActiveInstance is async, so subsequence state checks must be in eventually.
## How was this patch tested?
Jenkins tests

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #11924 from tdas/state-store-flaky-fix.
2016-03-25 12:04:47 -07:00
Sameer Agarwal b5f8c36e3c [SPARK-14144][SQL] Explicitly identify/catch UnsupportedOperationException during parquet reader initialization
## What changes were proposed in this pull request?

This PR is a minor cleanup task as part of https://issues.apache.org/jira/browse/SPARK-14008 to explicitly identify/catch the `UnsupportedOperationException` while initializing the vectorized parquet reader. Other exceptions will simply be thrown back to `SqlNewHadoopPartition`.

## How was this patch tested?

N/A (cleanup only; no new functionality added)

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11950 from sameeragarwal/parquet-cleanup.
2016-03-25 11:48:05 -07:00
Wenchen Fan 43b15e01c4 [SPARK-14061][SQL] implement CreateMap
## What changes were proposed in this pull request?

As we have `CreateArray` and `CreateStruct`, we should also have `CreateMap`.  This PR adds the `CreateMap` expression, and the DataFrame API, and python API.

## How was this patch tested?

various new tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11879 from cloud-fan/create_map.
2016-03-25 09:50:06 -07:00
Davies Liu 6603d9f7e2 [SPARK-13919] [SQL] fix column pruning through filter
## What changes were proposed in this pull request?

This PR fix the conflict between ColumnPruning and PushPredicatesThroughProject, because ColumnPruning will try to insert a Project before Filter, but PushPredicatesThroughProject will move the Filter before Project.This is fixed by remove the Project before Filter, if the Project only do column pruning.

The RuleExecutor will fail the test if reached max iterations.

Closes #11745

## How was this patch tested?

Existing tests.

This is a test case still failing, disabled for now, will be fixed by https://issues.apache.org/jira/browse/SPARK-14137

Author: Davies Liu <davies@databricks.com>

Closes #11828 from davies/fail_rule.
2016-03-25 09:05:23 -07:00
Wenchen Fan e9b6e7d857 [SPARK-13456][SQL][FOLLOW-UP] lazily generate the outer pointer for case class defined in REPL
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/11410, we missed a corner case: define the inner class and use it in `Dataset` at the same time by using paste mode. For this case, the inner class and the `Dataset` are inside same line object, when we build the `Dataset`, we try to get outer pointer from line object, and it will fail because the line object is not initialized yet.

https://issues.apache.org/jira/browse/SPARK-13456?focusedCommentId=15209174&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15209174 is an example for this corner case.

This PR make the process of getting outer pointer from line object lazy, so that we can successfully build the `Dataset` and finish initializing the line object.

## How was this patch tested?

new test in repl suite.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11931 from cloud-fan/repl.
2016-03-25 20:19:04 +08:00
Reynold Xin 70a6f0bb57 [SPARK-14149] Log exceptions in tryOrIOException
## What changes were proposed in this pull request?
We ran into a problem today debugging some class loading problem during deserialization, and JVM was masking the underlying exception which made it very difficult to debug. We can however log the exceptions using try/catch ourselves in serialization/deserialization. The good thing is that all these methods are already using Utils.tryOrIOException, so we can just put the try catch and logging in a single place.

## How was this patch tested?
A logging change with a manual test.

Author: Reynold Xin <rxin@databricks.com>

Closes #11951 from rxin/SPARK-14149.
2016-03-25 01:17:23 -07:00
Andrew Or 20ddf5fddf [SPARK-14014][SQL] Integrate session catalog (attempt #2)
## What changes were proposed in this pull request?

This reopens #11836, which was merged but promptly reverted because it introduced flaky Hive tests.

## How was this patch tested?

See `CatalogTestCases`, `SessionCatalogSuite` and `HiveContextSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #11938 from andrewor14/session-catalog-again.
2016-03-24 22:59:35 -07:00
Reynold Xin 1c70b7650f [SPARK-14145][SQL] Remove the untyped version of Dataset.groupByKey
## What changes were proposed in this pull request?
Dataset has two variants of groupByKey, one for untyped and the other for typed. It actually doesn't make as much sense to have an untyped API here, since apps that want to use untyped APIs should just use the groupBy "DataFrame" API.

## How was this patch tested?
This patch removes a method, and removes the associated tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11949 from rxin/SPARK-14145.
2016-03-24 22:56:34 -07:00
Reynold Xin 3619fec1ec [SPARK-14142][SQL] Replace internal use of unionAll with union
## What changes were proposed in this pull request?
unionAll has been deprecated in SPARK-14088.

## How was this patch tested?
Should be covered by all existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11946 from rxin/SPARK-14142.
2016-03-24 22:34:55 -07:00
gatorsmile 05f652d6c2 [SPARK-13957][SQL] Support Group By Ordinal in SQL
#### What changes were proposed in this pull request?
This PR is to support group by position in SQL. For example, when users input the following query
```SQL
select c1 as a, c2, c3, sum(*) from tbl group by 1, 3, c4
```
The ordinals are recognized as the positions in the select list. Thus, `Analyzer` converts it to
```SQL
select c1, c2, c3, sum(*) from tbl group by c1, c3, c4
```

This is controlled by the config option `spark.sql.groupByOrdinal`.
- When true, the ordinal numbers in group by clauses are treated as the position in the select list.
- When false, the ordinal numbers are ignored.
- Only convert integer literals (not foldable expressions). If found foldable expressions, ignore them.
- When the positions specified in the group by clauses correspond to the aggregate functions in select list, output an exception message.
- star is not allowed to use in the select list when users specify ordinals in group by

Note: This PR is taken from https://github.com/apache/spark/pull/10731. When merging this PR, please give the credit to zhichao-li

Also cc all the people who are involved in the previous discussion:  rxin cloud-fan marmbrus yhuai hvanhovell adrian-wang chenghao-intel tejasapatil

#### How was this patch tested?

Added a few test cases for both positive and negative test cases.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #11846 from gatorsmile/groupByOrdinal.
2016-03-25 12:55:58 +08:00
Andrew Or c44d140cae Revert "[SPARK-14014][SQL] Replace existing catalog with SessionCatalog"
This reverts commit 5dfc01976b.
2016-03-23 22:21:15 -07:00
gatorsmile f42eaf42bd [SPARK-14085][SQL] Star Expansion for Hash
#### What changes were proposed in this pull request?

This PR is to support star expansion in hash. For example,
```SQL
val structDf = testData2.select("a", "b").as("record")
structDf.select(hash($"*")
```

In addition, it refactors the codes for the rule `ResolveStar` and fixes a regression for star expansion in group by when using SQL API. For example,
```SQL
SELECT * FROM testData2 group by a, b
```

cc cloud-fan Now, the code for star resolution is much cleaner. The coverage is better. Could you check if this refactoring is good? Thanks!

#### How was this patch tested?
Added a few test cases to cover it.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11904 from gatorsmile/starResolution.
2016-03-24 11:13:36 +08:00
Andrew Or 5dfc01976b [SPARK-14014][SQL] Replace existing catalog with SessionCatalog
## What changes were proposed in this pull request?

`SessionCatalog`, introduced in #11750, is a catalog that keeps track of temporary functions and tables, and delegates metastore operations to `ExternalCatalog`. This functionality overlaps a lot with the existing `analysis.Catalog`.

As of this commit, `SessionCatalog` and `ExternalCatalog` will no longer be dead code. There are still things that need to be done after this patch, namely:
- SPARK-14013: Properly implement temporary functions in `SessionCatalog`
- SPARK-13879: Decide which DDL/DML commands to support natively in Spark
- SPARK-?????: Implement the ones we do want to support through `SessionCatalog`.
- SPARK-?????: Merge SQL/HiveContext

## How was this patch tested?

This is largely a refactoring task so there are no new tests introduced. The particularly relevant tests are `SessionCatalogSuite` and `ExternalCatalogSuite`.

Author: Andrew Or <andrew@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #11836 from andrewor14/use-session-catalog.
2016-03-23 13:34:22 -07:00
Michael Armbrust 6bc4be64f8 [SPARK-14078] Streaming Parquet Based FileSink
This PR adds a new `Sink` implementation that writes out Parquet files.  In order to correctly handle partial failures while maintaining exactly once semantics, the files for each batch are written out to a unique directory and then atomically appended to a metadata log.  When a parquet based `DataSource` is initialized for reading, we first check for this log directory and use it instead of file listing when present.

Unit tests are added, as well as a stress test that checks the answer after non-deterministic injected failures.

Author: Michael Armbrust <michael@databricks.com>

Closes #11897 from marmbrus/fileSink.
2016-03-23 13:03:25 -07:00
Herman van Hovell 919bf32198 [SPARK-13325][SQL] Create a 64-bit hashcode expression
This PR introduces a 64-bit hashcode expression. Such an expression is especially usefull for HyperLogLog++ and other probabilistic datastructures.

I have implemented xxHash64 which is a 64-bit hashing algorithm created by Yann Colet and Mathias Westerdahl. This is a high speed (C implementation runs at memory bandwidth) and high quality hashcode. It exploits both Instruction Level Parralellism (for speed) and the multiplication and rotation techniques (for quality) like MurMurHash does.

The initial results are promising. I have added a CG'ed test to the `HashBenchmark`, and this results in the following results (running from SBT):

    Running benchmark: Hash For simple
      Running case: interpreted version
      Running case: codegen version
      Running case: codegen version 64-bit

    Intel(R) Core(TM) i7-4750HQ CPU  2.00GHz
    Hash For simple:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    interpreted version                      1011 / 1016        132.8           7.5       1.0X
    codegen version                          1864 / 1869         72.0          13.9       0.5X
    codegen version 64-bit                   1614 / 1644         83.2          12.0       0.6X

    Running benchmark: Hash For normal
      Running case: interpreted version
      Running case: codegen version
      Running case: codegen version 64-bit

    Intel(R) Core(TM) i7-4750HQ CPU  2.00GHz
    Hash For normal:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    interpreted version                      2467 / 2475          0.9        1176.1       1.0X
    codegen version                          2008 / 2115          1.0         957.5       1.2X
    codegen version 64-bit                    728 /  758          2.9         347.0       3.4X

    Running benchmark: Hash For array
      Running case: interpreted version
      Running case: codegen version
      Running case: codegen version 64-bit

    Intel(R) Core(TM) i7-4750HQ CPU  2.00GHz
    Hash For array:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    interpreted version                      1544 / 1707          0.1       11779.6       1.0X
    codegen version                          2728 / 2745          0.0       20815.5       0.6X
    codegen version 64-bit                   2508 / 2549          0.1       19132.8       0.6X

    Running benchmark: Hash For map
      Running case: interpreted version
      Running case: codegen version
      Running case: codegen version 64-bit

    Intel(R) Core(TM) i7-4750HQ CPU  2.00GHz
    Hash For map:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    -------------------------------------------------------------------------------------------
    interpreted version                      1819 / 1826          0.0      444014.3       1.0X
    codegen version                           183 /  194          0.0       44642.9       9.9X
    codegen version 64-bit                    173 /  174          0.0       42120.9      10.5X

This shows that algorithm is consistently faster than MurMurHash32 in all cases and up to 3x (!) in the normal case.

I have also added this to HyperLogLog++ and it cuts the processing time of the following code in half:

    val df = sqlContext.range(1<<25).agg(approxCountDistinct("id"))
    df.explain()
    val t = System.nanoTime()
    df.show()
    val ns = System.nanoTime() - t

    // Before
    ns: Long = 5821524302

    // After
    ns: Long = 2836418963

cc cloud-fan (you have been working on hashcodes) / rxin

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #11209 from hvanhovell/xxHash.
2016-03-23 20:51:01 +01:00
Tathagata Das 8c826880f5 [SPARK-13809][SQL] State store for streaming aggregations
## What changes were proposed in this pull request?

In this PR, I am implementing a new abstraction for management of streaming state data - State Store. It is a key-value store for persisting running aggregates for aggregate operations in streaming dataframes. The motivation and design is discussed here.

https://docs.google.com/document/d/1-ncawFx8JS5Zyfq1HAEGBx56RDet9wfVp_hDM8ZL254/edit#

## How was this patch tested?
- [x] Unit tests
- [x] Cluster tests

**Coverage from unit tests**

<img width="952" alt="screen shot 2016-03-21 at 3 09 40 pm" src="https://cloud.githubusercontent.com/assets/663212/13935872/fdc8ba86-ef76-11e5-93e8-9fa310472c7b.png">

## TODO
- [x] Fix updates() iterator to avoid duplicate updates for same key
- [x] Use Coordinator in ContinuousQueryManager
- [x] Plugging in hadoop conf and other confs
- [x] Unit tests
  - [x] StateStore object lifecycle and methods
  - [x] StateStoreCoordinator communication and logic
  - [x] StateStoreRDD fault-tolerance
  - [x] StateStoreRDD preferred location using StateStoreCoordinator
- [ ] Cluster tests
  - [ ] Whether preferred locations are set correctly
  - [ ] Whether recovery works correctly with distributed storage
  - [x] Basic performance tests
- [x] Docs

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #11645 from tdas/state-store.
2016-03-23 12:48:05 -07:00
Sameer Agarwal 0a64294fcb [SPARK-14015][SQL] Support TimestampType in vectorized parquet reader
## What changes were proposed in this pull request?

This PR adds support for TimestampType in the vectorized parquet reader

## How was this patch tested?

1. `VectorizedColumnReader` initially had a gating condition on `primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT96)` that made us fall back on parquet-mr for handling timestamps. This condition is now removed.
2. The `ParquetHadoopFsRelationSuite` (that tests for all supported hive types -- including `TimestampType`) fails when the gating condition is removed (https://github.com/apache/spark/pull/11808) and should now pass with this change. Similarly, the `ParquetHiveCompatibilitySuite.SPARK-10177 timestamp` test that fails when the gating condition is removed, should now pass as well.
3.  Added tests in `HadoopFsRelationTest` that test both the dictionary encoded and non-encoded versions across all supported datatypes.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11882 from sameeragarwal/timestamp-parquet.
2016-03-23 12:13:32 -07:00
Davies Liu 02d9c352c7 [SPARK-14092] [SQL] move shouldStop() to end of while loop
## What changes were proposed in this pull request?

This PR rollback some changes in #11274 , which introduced some performance regression when do a simple aggregation on parquet scan with one integer column.

Does not really understand how this change introduce this huge impact, maybe related show JIT compiler inline functions. (saw very different stats from profiling).

## How was this patch tested?

Manually run the parquet reader benchmark, before this change:
```
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
Int and String Scan:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
SQL Parquet Vectorized                   2391 / 3107         43.9          22.8       1.0X
```
After this change
```
Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5
Intel(R) Core(TM) i7-4558U CPU  2.80GHz
Int and String Scan:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
SQL Parquet Vectorized                   2032 / 2626         51.6          19.4       1.0X```

Author: Davies Liu <davies@databricks.com>

Closes #11912 from davies/fix_regression.
2016-03-23 11:58:43 -07:00
Josh Rosen 3de24ae2ed [SPARK-14075] Refactor MemoryStore to be testable independent of BlockManager
This patch refactors the `MemoryStore` so that it can be tested without needing to construct / mock an entire `BlockManager`.

- The block manager's serialization- and compression-related methods have been moved from `BlockManager` to `SerializerManager`.
- `BlockInfoManager `is now passed directly to classes that need it, rather than being passed via the `BlockManager`.
- The `MemoryStore` now calls `dropFromMemory` via a new `BlockEvictionHandler` interface rather than directly calling the `BlockManager`. This change helps to enforce a narrow interface between the `MemoryStore` and `BlockManager` functionality and makes this interface easier to mock in tests.
- Several of the block unrolling tests have been moved from `BlockManagerSuite` into a new `MemoryStoreSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11899 from JoshRosen/reduce-memorystore-blockmanager-coupling.
2016-03-23 10:15:23 -07:00
gatorsmile 6ce008ba46 [SPARK-13549][SQL] Refactor the Optimizer Rule CollapseProject
#### What changes were proposed in this pull request?

The PR https://github.com/apache/spark/pull/10541 changed the rule `CollapseProject` by enabling collapsing `Project` into `Aggregate`. It leaves a to-do item to remove the duplicate code. This PR is to finish this to-do item. Also added a test case for covering this change.

#### How was this patch tested?

Added a new test case.

liancheng Could you check if the code refactoring is fine? Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11427 from gatorsmile/collapseProjectRefactor.
2016-03-24 00:51:31 +08:00
Cheng Lian cde086cb2a [SPARK-13817][SQL][MINOR] Renames Dataset.newDataFrame to Dataset.ofRows
## What changes were proposed in this pull request?

This PR does the renaming as suggested by marmbrus in [this comment][1].

## How was this patch tested?

Existing tests.

[1]: 6d37e1eb90 (commitcomment-16654694)

Author: Cheng Lian <lian@databricks.com>

Closes #11889 from liancheng/spark-13817-follow-up.
2016-03-24 00:42:13 +08:00
Shixiong Zhu abacf5f258 [HOTFIX][SQL] Don't stop ContinuousQuery in quietly
## What changes were proposed in this pull request?

Try to fix a flaky hang

## How was this patch tested?

Existing Jenkins test

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11909 from zsxwing/hotfix2.
2016-03-23 00:00:35 -07:00
Reynold Xin 926a93e54b [SPARK-14088][SQL] Some Dataset API touch-up
## What changes were proposed in this pull request?
1. Deprecated unionAll. It is pretty confusing to have both "union" and "unionAll" when the two do the same thing in Spark but are different in SQL.
2. Rename reduce in KeyValueGroupedDataset to reduceGroups so it is more consistent with rest of the functions in KeyValueGroupedDataset. Also makes it more obvious what "reduce" and "reduceGroups" mean. Previously it was confusing because it could be reducing a Dataset, or just reducing groups.
3. Added a "name" function, which is more natural to name columns than "as" for non-SQL users.
4. Remove "subtract" function since it is just an alias for "except".

## How was this patch tested?
All changes should be covered by existing tests. Also added couple test cases to cover "name".

Author: Reynold Xin <rxin@databricks.com>

Closes #11908 from rxin/SPARK-14088.
2016-03-22 23:43:09 -07:00
Dongjoon Hyun 1a22cf1e9b [MINOR][SQL][DOCS] Update sql/README.md and remove some unused imports in sql module.
## What changes were proposed in this pull request?

This PR updates `sql/README.md` according to the latest console output and removes some unused imports in `sql` module. This is done by manually, so there is no guarantee to remove all unused imports.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11907 from dongjoon-hyun/update_sql_module.
2016-03-22 23:07:49 -07:00
Yong Tang 75dc29620e [SPARK-13401][SQL][TESTS] Fix SQL test warnings.
## What changes were proposed in this pull request?

This fix tries to fix several SQL test warnings under the sql/core/src/test directory. The fixed warnings includes "[unchecked]", "[rawtypes]", and "[varargs]".

## How was this patch tested?

All existing tests passed.

Author: Yong Tang <yong.tang.github@outlook.com>

Closes #11857 from yongtang/SPARK-13401.
2016-03-22 21:08:11 -07:00
Davies Liu 4700adb98e [SPARK-13806] [SQL] fix rounding mode of negative float/double
## What changes were proposed in this pull request?

Round() in database usually round the number up (away from zero), it's different than Math.round() in Java.

For example:
```
scala> java.lang.Math.round(-3.5)
res3: Long = -3
```
In Database, we should return -4.0 in this cases.

This PR remove the buggy special case for scale=0.

## How was this patch tested?

Add tests for negative values with tie.

Author: Davies Liu <davies@databricks.com>

Closes #11894 from davies/fix_round.
2016-03-22 16:45:20 -07:00
Shixiong Zhu d16710b4c9 [HOTFIX][SQL] Add a timeout for 'cq.stop'
## What changes were proposed in this pull request?

Fix an issue that DataFrameReaderWriterSuite may hang forever.

## How was this patch tested?

Existing tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11902 from zsxwing/hotfix.
2016-03-22 16:41:55 -07:00
Reynold Xin b2b1ad7d4c [SPARK-14060][SQL] Move StringToColumn implicit class into SQLImplicits
## What changes were proposed in this pull request?
This patch moves StringToColumn implicit class into SQLImplicits. This was kept in SQLContext.implicits object for binary backward compatibility, in the Spark 1.x series. It makes more sense for this API to be in SQLImplicits since that's the single class that defines all the SQL implicits.

## How was this patch tested?
Should be covered by existing unit tests.

Author: Reynold Xin <rxin@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #11878 from rxin/SPARK-14060.
2016-03-22 13:48:03 -07:00
Reynold Xin 297c20226d [SPARK-14063][SQL] SQLContext.range should return Dataset[java.lang.Long]
## What changes were proposed in this pull request?
This patch changed the return type for SQLContext.range from `Dataset[Long]` (Scala primitive) to `Dataset[java.lang.Long]` (Java boxed long).

Previously, SPARK-13894 changed the return type of range from `Dataset[Row]` to `Dataset[Long]`. The problem is that due to https://issues.scala-lang.org/browse/SI-4388, Scala compiles primitive types in generics into just Object, i.e. range at bytecode level now just returns `Dataset[Object]`. This is really bad for Java users because they are losing type safety and also need to add a type cast every time they use range.

Talked to Jason Zaugg from Lightbend (Typesafe) who suggested the best approach is to return `Dataset[java.lang.Long]`. The downside is that when Scala users want to explicitly type a closure used on the dataset returned by range, they would need to use `java.lang.Long` instead of the Scala `Long`.

## How was this patch tested?
The signature change should be covered by existing unit tests and API tests. I also added a new test case in DatasetSuite for range.

Author: Reynold Xin <rxin@databricks.com>

Closes #11880 from rxin/SPARK-14063.
2016-03-22 11:37:37 -07:00
Michael Armbrust caea152145 [SPARK-13985][SQL] Deterministic batches with ids
This PR relaxes the requirements of a `Sink` for structured streaming to only require idempotent appending of data.  Previously the `Sink` needed to be able to transactionally append data while recording an opaque offset indicated how far in a stream we have processed.

In order to do this, a new write-ahead-log has been added to stream execution, which records the offsets that will are present in each batch.  The log is created in the newly added `checkpointLocation`, which defaults to `${spark.sql.streaming.checkpointLocation}/${queryName}` but can be overriden by setting `checkpointLocation` in `DataFrameWriter`.

In addition to making sinks easier to write the addition of batchIds and a checkpoint location is done in anticipation of integration with the the `StateStore` (#11645).

Author: Michael Armbrust <michael@databricks.com>

Closes #11804 from marmbrus/batchIds.
2016-03-22 10:18:42 -07:00
Dongjoon Hyun c632bdc01f [SPARK-14029][SQL] Improve BooleanSimplification optimization by implementing Not canonicalization.
## What changes were proposed in this pull request?

Currently, **BooleanSimplification** optimization can handle the following cases.
* a && (!a || b ) ==> a && b
* a && (b || !a ) ==> a && b

However, it can not handle the followings cases since those equations fail at the comparisons between their canonicalized forms.
* a < 1 && (!(a < 1) || b)     ==> (a < 1) && b
* a <= 1 && (!(a <= 1) || b) ==> (a <= 1) && b
* a > 1 && (!(a > 1) || b)     ==> (a > 1) && b
* a >= 1 && (!(a >= 1) || b) ==> (a >= 1) && b

This PR implements the above cases and also the followings, too.
* a < 1 && ((a >= 1) || b )   ==> (a < 1) && b
* a <= 1 && ((a > 1) || b )   ==> (a <= 1) && b
* a > 1 && ((a <= 1) || b)  ==> (a > 1) && b
* a >= 1 && ((a < 1) || b)  ==> (a >= 1) && b

## How was this patch tested?

Pass the Jenkins tests including new test cases in BooleanSimplicationSuite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11851 from dongjoon-hyun/SPARK-14029.
2016-03-22 10:17:08 -07:00
Sunitha Kambhampati 0ce01635cc [SPARK-13774][SQL] - Improve error message for non-existent paths and add tests
SPARK-13774: IllegalArgumentException: Can not create a Path from an empty string for incorrect file path

**Overview:**
-	If a non-existent path is given in this call
``
scala> sqlContext.read.format("csv").load("file-path-is-incorrect.csv")
``
it throws the following error:
`java.lang.IllegalArgumentException: Can not create a Path from an empty string` …..
`It gets called from inferSchema call in org.apache.spark.sql.execution.datasources.DataSource.resolveRelation`

-	The purpose of this JIRA is to throw a better error message.
-	With the fix, you will now get a _Path does not exist_ error message.
```
scala> sqlContext.read.format("csv").load("file-path-is-incorrect.csv")
org.apache.spark.sql.AnalysisException: Path does not exist: file:/Users/ksunitha/trunk/spark/file-path-is-incorrect.csv;
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:215)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:204)
  ...
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:204)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:131)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:141)
  ... 49 elided
```

**Details**
_Changes include:_
-	Check if path exists or not in resolveRelation in DataSource, and throw an AnalysisException with message like “Path does not exist: $path”
-	AnalysisException is thrown similar to the exceptions thrown in resolveRelation.
-	The glob path and the non glob path is checked with minimal calls to path exists. If the globPath is empty, then it is a nonexistent glob pattern and an error will be thrown. In the scenario that it is not globPath, it is necessary to only check if the first element in the Seq is valid or not.

_Test modifications:_
-	Changes went in for 3 tests to account for this error checking.
-	SQLQuerySuite:test("run sql directly on files") – Error message needed to be updated.
-	2 tests failed in MetastoreDataSourcesSuite because they had a dummy path and so test is modified to give a tempdir and allow it to move past so it can continue to test the codepath it meant to test

_New Tests:_
2 new tests are added to DataFrameSuite to validate that glob and non-glob path will throw the new error message.

_Testing:_
Unit tests were run with the fix.

**Notes/Questions to reviewers:**
-	There is some code duplication in DataSource.scala in resolveRelation method and also createSource with respect to getting the paths.  I have not made any changes to the createSource codepath.  Should we make the change there as well ?

-	From other JIRAs, I know there is restructuring and changes going on in this area, not sure how that will affect these changes, but since this seemed like a starter issue, I looked into it.  If we prefer not to add the overhead of the checks, or if there is a better place to do so, let me know.

I would appreciate your review. Thanks for your time and comments.

Author: Sunitha Kambhampati <skambha@us.ibm.com>

Closes #11775 from skambha/improve_errmsg.
2016-03-22 20:47:57 +08:00
hyukjinkwon 4e09a0d5ea [SPARK-13953][SQL] Specifying the field name for corrupted record via option at JSON datasource
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-13953

Currently, JSON data source creates a new field in `PERMISSIVE` mode for storing malformed string.
This field can be renamed via `spark.sql.columnNameOfCorruptRecord` option but it is a global configuration.

This PR make that option can be applied per read and can be specified via `option()`. This will overwrites `spark.sql.columnNameOfCorruptRecord` if it is set.

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for coding style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11881 from HyukjinKwon/SPARK-13953.
2016-03-22 20:30:48 +08:00
Cheng Lian f2e855fba8 [SPARK-13473][SQL] Simplifies PushPredicateThroughProject
## What changes were proposed in this pull request?

This is a follow-up of PR #11348.

After PR #11348, a predicate is never pushed through a project as long as the project contains any non-deterministic fields. Thus, it's impossible that the candidate filter condition can reference any non-deterministic projected fields, and related logic can be safely cleaned up.

To be more specific, the following optimization is allowed:

```scala
// From:
df.select('a, 'b).filter('c > rand(42))
// To:
df.filter('c > rand(42)).select('a, 'b)
```

while this isn't:

```scala
// From:
df.select('a, rand('b) as 'rb, 'c).filter('c > 'rb)
// To:
df.filter('c > rand('b)).select('a, rand('b) as 'rb, 'c)
```

## How was this patch tested?

Existing test cases should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #11864 from liancheng/spark-13473-cleanup.
2016-03-22 19:20:56 +08:00
Wenchen Fan 14464cadb9 [SPARK-14038][SQL] enable native view by default
## What changes were proposed in this pull request?

As we have completed the `SQLBuilder`, we can safely turn on native view by default.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11872 from cloud-fan/native-view.
2016-03-22 00:07:57 -07:00
Michael Armbrust 8014a516d1 [SPARK-13883][SQL] Parquet Implementation of FileFormat.buildReader
This PR add implements the new `buildReader` interface for the Parquet `FileFormat`.  An simple implementation of `FileScanRDD` is also included.

This code should be tested by the many existing tests for parquet.

Author: Michael Armbrust <michael@databricks.com>
Author: Sameer Agarwal <sameer@databricks.com>
Author: Nong Li <nong@databricks.com>

Closes #11709 from marmbrus/parquetReader.
2016-03-21 20:16:01 -07:00
Sameer Agarwal 7299961657 [SPARK-14016][SQL] Support high-precision decimals in vectorized parquet reader
## What changes were proposed in this pull request?

This patch adds support for reading `DecimalTypes` with high (> 18) precision in `VectorizedColumnReader`

## How was this patch tested?

1. `VectorizedColumnReader` initially had a gating condition on `primitiveType.getDecimalMetadata().getPrecision() > Decimal.MAX_LONG_DIGITS()` that made us fall back on parquet-mr for handling high-precision decimals. This condition is now removed.
2. In particular, the `ParquetHadoopFsRelationSuite` (that tests for all supported hive types -- including `DecimalType(25, 5)`) fails when the gating condition is removed (https://github.com/apache/spark/pull/11808) and should now pass with this change.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11869 from sameeragarwal/bigdecimal-parquet.
2016-03-21 18:19:54 -07:00
gatorsmile 3f49e0766f [SPARK-13320][SQL] Support Star in CreateStruct/CreateArray and Error Handling when DataFrame/DataSet Functions using Star
This PR resolves two issues:

First, expanding * inside aggregate functions of structs when using Dataframe/Dataset APIs. For example,
```scala
structDf.groupBy($"a").agg(min(struct($"record.*")))
```

Second, it improves the error messages when having invalid star usage when using Dataframe/Dataset APIs. For example,
```scala
pagecounts4PartitionsDS
  .map(line => (line._1, line._3))
  .toDF()
  .groupBy($"_1")
  .agg(sum("*") as "sumOccurances")
```
Before the fix, the invalid usage will issue a confusing error message, like:
```
org.apache.spark.sql.AnalysisException: cannot resolve '_1' given input columns _1, _2;
```
After the fix, the message is like:
```
org.apache.spark.sql.AnalysisException: Invalid usage of '*' in function 'sum'
```
cc: rxin nongli cloud-fan

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11208 from gatorsmile/sumDataSetResolution.
2016-03-22 08:21:02 +08:00
Reynold Xin b3e5af62a1 [SPARK-13898][SQL] Merge DatasetHolder and DataFrameHolder
## What changes were proposed in this pull request?
This patch merges DatasetHolder and DataFrameHolder. This makes more sense because DataFrame/Dataset are now one class.

In addition, fixed some minor issues with pull request #11732.

## How was this patch tested?
Updated existing unit tests that test these implicits.

Author: Reynold Xin <rxin@databricks.com>

Closes #11737 from rxin/SPARK-13898.
2016-03-21 17:17:25 -07:00
Nong Li 5e86e9262f [SPARK-13916][SQL] Add a metric to WholeStageCodegen to measure duration.
## What changes were proposed in this pull request?

WholeStageCodegen naturally breaks the execution into pipelines that are easier to
measure duration. This is more granular than the task timings (a task can be multiple
pipelines) and is integrated with the web ui.

We currently report total time (across all tasks), min/mask/median to get a sense of how long each is taking.

## How was this patch tested?

Manually tested looking at the web ui.

Author: Nong Li <nong@databricks.com>

Closes #11741 from nongli/spark-13916.
2016-03-21 16:56:33 -07:00
Wenchen Fan f3717fc7c9 [SPARK-14004][FOLLOW-UP] Implementations of NonSQLExpression should not override sql method
## What changes were proposed in this pull request?

There is only one exception: `PythonUDF`. However, I don't think the `PythonUDF#` prefix is useful, as we can only create python udf under python context. This PR removes the `PythonUDF#` prefix from `PythonUDF.toString`, so that it doesn't need to overrde `sql`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11859 from cloud-fan/tmp.
2016-03-21 15:24:18 -07:00
Kazuaki Ishizaki f35df7d182 [SPARK-13805] [SQL] Generate code that get a value in each column from ColumnVector when ColumnarBatch is used
## What changes were proposed in this pull request?

This PR generates code that get a value in each column from ```ColumnVector``` instead of creating ```InternalRow``` when ```ColumnarBatch``` is accessed. This PR improves benchmark program by up to 15%.
This PR consists of two parts:

1. Get an ```ColumnVector ``` by using ```ColumnarBatch.column()``` method
2. Get a value of each column by using ```rdd_col${COLIDX}.getInt(ROWIDX)``` instead of ```rdd_row.getInt(COLIDX)```

This is a motivated example.
````
    sqlContext.conf.setConfString(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true")
    sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true")
    val values = 10
    withTempPath { dir =>
      withTempTable("t1", "tempTable") {
        sqlContext.range(values).registerTempTable("t1")
        sqlContext.sql("select id % 2 as p, cast(id as INT) as id from t1")
          .write.partitionBy("p").parquet(dir.getCanonicalPath)
        sqlContext.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable")
        sqlContext.sql("select sum(p) from tempTable").collect
      }
    }
````

The original code
````java
    ...
    /* 072 */       while (!shouldStop() && rdd_batchIdx < numRows) {
    /* 073 */         InternalRow rdd_row = rdd_batch.getRow(rdd_batchIdx++);
    /* 074 */         /*** CONSUME: TungstenAggregate(key=[], functions=[(sum(cast(p#4 as bigint)),mode=Partial,isDistinct=false)], output=[sum#10L]) */
    /* 075 */         /* input[0, int] */
    /* 076 */         boolean rdd_isNull = rdd_row.isNullAt(0);
    /* 077 */         int rdd_value = rdd_isNull ? -1 : (rdd_row.getInt(0));
    ...
````

The code generated by this PR
````java
    /* 072 */       while (!shouldStop() && rdd_batchIdx < numRows) {
    /* 073 */         org.apache.spark.sql.execution.vectorized.ColumnVector rdd_col0 = rdd_batch.column(0);
    /* 074 */         /*** CONSUME: TungstenAggregate(key=[], functions=[(sum(cast(p#4 as bigint)),mode=Partial,isDistinct=false)], output=[sum#10L]) */
    /* 075 */         /* input[0, int] */
    /* 076 */         boolean rdd_isNull = rdd_col0.getIsNull(rdd_batchIdx);
    /* 077 */         int rdd_value = rdd_isNull ? -1 : (rdd_col0.getInt(rdd_batchIdx));
    ...
    /* 128 */         rdd_batchIdx++;
    /* 129 */       }
    /* 130 */       if (shouldStop()) return;

````
Performance
Without this PR
````
model name	: Intel(R) Xeon(R) CPU E5-2667 v2  3.30GHz
Partitioned Table:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
Read data column                          434 /  488         36.3          27.6       1.0X
Read partition column                     302 /  346         52.1          19.2       1.4X
Read both columns                         588 /  643         26.8          37.4       0.7X
````
With this PR
````
model name	: Intel(R) Xeon(R) CPU E5-2667 v2  3.30GHz
Partitioned Table:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
Read data column                          392 /  516         40.1          24.9       1.0X
Read partition column                     256 /  318         61.4          16.3       1.5X
Read both columns                         523 /  539         30.1          33.3       0.7X
````

## How was this patch tested?
Tested by existing test suites and benchmark

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #11636 from kiszk/SPARK-13805.
2016-03-21 14:36:51 -07:00
Davies Liu 9b4e15ba13 [SPARK-14007] [SQL] Manage the memory used by hash map in shuffled hash join
## What changes were proposed in this pull request?

This PR try acquire the memory for hash map in shuffled hash join, fail the task if there is no enough memory (otherwise it could OOM the executor).

It also removed unused HashedRelation.

## How was this patch tested?

Existing unit tests. Manual tests with TPCDS Q78.

Author: Davies Liu <davies@databricks.com>

Closes #11826 from davies/cleanup_hash2.
2016-03-21 11:21:39 -07:00
Cheng Lian 5d8de16e71 [SPARK-14004][SQL] NamedExpressions should have at most one qualifier
## What changes were proposed in this pull request?

This is a more aggressive version of PR #11820, which not only fixes the original problem, but also does the following updates to enforce the at-most-one-qualifier constraint:

- Renames `NamedExpression.qualifiers` to `NamedExpression.qualifier`
- Uses `Option[String]` rather than `Seq[String]` for `NamedExpression.qualifier`

Quoted PR description of #11820 here:

> Current implementations of `AttributeReference.sql` and `Alias.sql` joins all available qualifiers, which is logically wrong. But this implementation mistake doesn't cause any real SQL generation bugs though, since there is always at most one qualifier for any given `AttributeReference` or `Alias`.

## How was this patch tested?

Existing tests should be enough.

Author: Cheng Lian <lian@databricks.com>

Closes #11822 from liancheng/spark-14004-aggressive.
2016-03-21 11:00:09 -07:00
Wenchen Fan 43ebf7a9cb [SPARK-13456][SQL] fix creating encoders for case classes defined in Spark shell
## What changes were proposed in this pull request?

case classes defined in REPL are wrapped by line classes, and we have a trick for scala 2.10 REPL to automatically register the wrapper classes to `OuterScope` so that we can use when create encoders.
However, this trick doesn't work right after we upgrade to scala 2.11, and unfortunately the tests are only in scala 2.10, which makes this bug hidden until now.

This PR moves the encoder tests to scala 2.11  `ReplSuite`, and fixes this bug by another approach(the previous trick can't port to scala 2.11 REPL): make `OuterScope` smarter that can detect classes defined in REPL and load the singleton of line wrapper classes automatically.

## How was this patch tested?

the migrated encoder tests in `ReplSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11410 from cloud-fan/repl.
2016-03-21 10:37:24 -07:00
Cheng Lian 060a28c633 [SPARK-13826][SQL] Ad-hoc Dataset API ScalaDoc fixes
## What changes were proposed in this pull request?

Ad-hoc Dataset API ScalaDoc fixes

## How was this patch tested?

By building and checking ScalaDoc locally.

Author: Cheng Lian <lian@databricks.com>

Closes #11862 from liancheng/ds-doc-fixes.
2016-03-21 10:06:02 -07:00
Wenchen Fan a2a9078028 [SPARK-14039][SQL][MINOR] make SubqueryHolder an inner class
## What changes were proposed in this pull request?

`SubqueryHolder` is only used when generate SQL string in `SQLBuilder`, it's more clear to make it an inner class in `SQLBuilder`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11861 from cloud-fan/gensql.
2016-03-21 10:04:49 -07:00
Wenchen Fan 17a3f00676 [SPARK-14000][SQL] case class with a tuple field can't work in Dataset
## What changes were proposed in this pull request?

When we validate an encoder, we may call `dataType` on unresolved expressions. This PR fix the validation so that we will resolve attributes first.

## How was this patch tested?

a new test in `DatasetSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11816 from cloud-fan/encoder.
2016-03-21 22:22:15 +08:00
gatorsmile 2c5b18fb0f [SPARK-12789][SQL] Support Order By Ordinal in SQL
#### What changes were proposed in this pull request?
This PR is to support order by position in SQL, e.g.
```SQL
select c1, c2, c3 from tbl order by 1 desc, 3
```
should be equivalent to
```SQL
select c1, c2, c3 from tbl order by c1 desc, c3 asc
```

This is controlled by config option `spark.sql.orderByOrdinal`.
- When true, the ordinal numbers are treated as the position in the select list.
- When false, the ordinal number in order/sort By clause are ignored.

- Only convert integer literals (not foldable expressions). If found foldable expressions, ignore them
- This also works with select *.

**Question**: Do we still need sort by columns that contain zero reference? In this case, it will have no impact on the sorting results. IMO, we should not allow users do it. rxin cloud-fan marmbrus yhuai hvanhovell
-- Update: In these cases, they are ignored in this case.

**Note**: This PR is taken from https://github.com/apache/spark/pull/10731. When merging this PR, please give the credit to zhichao-li

Also cc all the people who are involved in the previous discussion: adrian-wang chenghao-intel tejasapatil

#### How was this patch tested?
Added a few test cases for both positive and negative test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11815 from gatorsmile/orderByPosition.
2016-03-21 18:08:41 +08:00
Dongjoon Hyun 761c2d1b6e [MINOR][DOCS] Add proper periods and spaces for CLI help messages and config doc.
## What changes were proposed in this pull request?

This PR adds some proper periods and spaces to Spark CLI help messages and SQL/YARN conf docs for consistency.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11848 from dongjoon-hyun/add_proper_period_and_space.
2016-03-21 08:00:09 +00:00
Dongjoon Hyun 20fd254101 [SPARK-14011][CORE][SQL] Enable LineLength Java checkstyle rule
## What changes were proposed in this pull request?

[Spark Coding Style Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide) has 100-character limit on lines, but it's disabled for Java since 11/09/15. This PR enables **LineLength** checkstyle again. To help that, this also introduces **RedundantImport** and **RedundantModifier**, too. The following is the diff on `checkstyle.xml`.

```xml
-        <!-- TODO: 11/09/15 disabled - the lengths are currently > 100 in many places -->
-        <!--
         <module name="LineLength">
             <property name="max" value="100"/>
             <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://"/>
         </module>
-        -->
         <module name="NoLineWrap"/>
         <module name="EmptyBlock">
             <property name="option" value="TEXT"/>
 -167,5 +164,7
         </module>
         <module name="CommentsIndentation"/>
         <module name="UnusedImports"/>
+        <module name="RedundantImport"/>
+        <module name="RedundantModifier"/>
```

## How was this patch tested?

Currently, `lint-java` is disabled in Jenkins. It needs a manual test.
After passing the Jenkins tests, `dev/lint-java` should passes locally.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11831 from dongjoon-hyun/SPARK-14011.
2016-03-21 07:58:57 +00:00
hyukjinkwon e474088144 [SPARK-13764][SQL] Parse modes in JSON data source
## What changes were proposed in this pull request?

Currently, there is no way to control the behaviour when fails to parse corrupt records in JSON data source .

This PR adds the support for parse modes just like CSV data source. There are three modes below:

- `PERMISSIVE` :  When it fails to parse, this sets `null` to to field. This is a default mode when it has been this mode.
- `DROPMALFORMED`: When it fails to parse, this drops the whole record.
- `FAILFAST`: When it fails to parse, it just throws an exception.

This PR also make JSON data source share the `ParseModes` in CSV data source.

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for code style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11756 from HyukjinKwon/SPARK-13764.
2016-03-21 15:42:35 +08:00
gatorsmile f58319a24f [SPARK-14019][SQL] Remove noop SortOrder in Sort
#### What changes were proposed in this pull request?

This PR is to add a new Optimizer rule for pruning Sort if its SortOrder is no-op. In the phase of **Optimizer**, if a specific `SortOrder` does not have any reference, it has no effect on the sorting results. If `Sort` is empty, remove the whole `Sort`.

For example, in the following SQL query
```SQL
SELECT * FROM t ORDER BY NULL + 5
```

Before the fix, the plan is like
```
== Analyzed Logical Plan ==
a: int, b: int
Sort [(cast(null as int) + 5) ASC], true
+- Project [a#92,b#93]
   +- SubqueryAlias t
      +- Project [_1#89 AS a#92,_2#90 AS b#93]
         +- LocalRelation [_1#89,_2#90], [[1,2],[1,2]]

== Optimized Logical Plan ==
Sort [null ASC], true
+- LocalRelation [a#92,b#93], [[1,2],[1,2]]

== Physical Plan ==
WholeStageCodegen
:  +- Sort [null ASC], true, 0
:     +- INPUT
+- Exchange rangepartitioning(null ASC, 5), None
   +- LocalTableScan [a#92,b#93], [[1,2],[1,2]]
```

After the fix, the plan is like
```
== Analyzed Logical Plan ==
a: int, b: int
Sort [(cast(null as int) + 5) ASC], true
+- Project [a#92,b#93]
   +- SubqueryAlias t
      +- Project [_1#89 AS a#92,_2#90 AS b#93]
         +- LocalRelation [_1#89,_2#90], [[1,2],[1,2]]

== Optimized Logical Plan ==
LocalRelation [a#92,b#93], [[1,2],[1,2]]

== Physical Plan ==
LocalTableScan [a#92,b#93], [[1,2],[1,2]]
```

cc rxin cloud-fan marmbrus Thanks!

#### How was this patch tested?
Added a test suite for covering this rule

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11840 from gatorsmile/sortElimination.
2016-03-21 10:34:54 +08:00
Reynold Xin dcaa016610 [SPARK-13897][SQL] RelationalGroupedDataset and KeyValueGroupedDataset
## What changes were proposed in this pull request?
Previously, Dataset.groupBy returns a GroupedData, and Dataset.groupByKey returns a GroupedDataset. The naming is very similar, and unfortunately does not convey the real differences between the two.

Assume we are grouping by some keys (K). groupByKey is a key-value style group by, in which the schema of the returned dataset is a tuple of just two fields: key and value. groupBy, on the other hand, is a relational style group by, in which the schema of the returned dataset is flattened and contain |K| + |V| fields.

This pull request also removes the experimental tag from RelationalGroupedDataset. It has been with DataFrame since 1.3, and we have enough confidence now to stabilize it.

## How was this patch tested?
This is a rename to improve API understandability. Should be covered by all existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11841 from rxin/SPARK-13897.
2016-03-19 11:23:14 -07:00
Reynold Xin 1970d911d9 [SPARK-14018][SQL] Use 64-bit num records in BenchmarkWholeStageCodegen
## What changes were proposed in this pull request?
500L << 20 is actually pretty close to 32-bit int limit. I was trying to increase this to 500L << 23 and got negative numbers instead.

## How was this patch tested?
I'm only modifying test code.

Author: Reynold Xin <rxin@databricks.com>

Closes #11839 from rxin/SPARK-14018.
2016-03-19 00:27:23 -07:00
Sameer Agarwal b39594472b [SPARK-14012][SQL] Extract VectorizedColumnReader from VectorizedParquetRecordReader
## What changes were proposed in this pull request?

This is a minor followup on https://github.com/apache/spark/pull/11799 that extracts out the `VectorizedColumnReader` from `VectorizedParquetRecordReader` into its own file.

## How was this patch tested?

N/A (refactoring only)

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11834 from sameeragarwal/rename.
2016-03-18 22:33:43 -07:00
Sameer Agarwal 54794113a6 [SPARK-13989] [SQL] Remove non-vectorized/unsafe-row parquet record reader
## What changes were proposed in this pull request?

This PR cleans up the new parquet record reader with the following changes:

1. Removes the non-vectorized parquet reader code from `UnsafeRowParquetRecordReader`.
2. Removes the non-vectorized column reader code from `ColumnReader`.
3. Renames `UnsafeRowParquetRecordReader` to `VectorizedParquetRecordReader` and `ColumnReader` to `VectorizedColumnReader`
4. Deprecate `PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED`

## How was this patch tested?

Refactoring only; Existing tests should reveal any problems.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11799 from sameeragarwal/vectorized-parquet.
2016-03-18 14:04:42 -07:00
Yin Huai 238fb485be [SPARK-13972][SQL][FOLLOW-UP] When creating the query execution for a converted SQL query, we eagerly trigger analysis
## What changes were proposed in this pull request?
As part of testing generating SQL query from a analyzed SQL plan, we run the generated SQL for tests in HiveComparisonTest. This PR makes the generated SQL get eagerly analyzed. So, when a generated SQL has any analysis error, we can see the error message created by
```
                  case NonFatal(e) => fail(
                    s"""Failed to analyze the converted SQL string:
                        |
                        |# Original HiveQL query string:
                        |$queryString
                        |
                        |# Resolved query plan:
                        |${originalQuery.analyzed.treeString}
                        |
                        |# Converted SQL query string:
                        |$convertedSQL
                     """.stripMargin, e)
```

Right now, if we can parse a generated SQL but fail to analyze it, we will see error message generated by the following code (it only mentions that we cannot execute the original query, i.e. `queryString`).
```
            case e: Throwable =>
              val errorMessage =
                s"""
                  |Failed to execute query using catalyst:
                  |Error: ${e.getMessage}
                  |${stackTraceToString(e)}
                  |$queryString
                  |$query
                  |== HIVE - ${hive.size} row(s) ==
                  |${hive.mkString("\n")}
                """.stripMargin
```

## How was this patch tested?
Existing tests.

Author: Yin Huai <yhuai@databricks.com>

Closes #11825 from yhuai/SPARK-13972-follow-up.
2016-03-18 13:40:53 -07:00
Davies Liu 9c23c818ca [SPARK-13977] [SQL] Brings back Shuffled hash join
## What changes were proposed in this pull request?

ShuffledHashJoin (also outer join) is removed in 1.6, in favor of SortMergeJoin, which is more robust and also fast.

ShuffledHashJoin is still useful in this case: 1) one table is much smaller than the other one, then cost to build a hash table on smaller table is smaller than sorting the larger table 2) any partition of the small table could fit in memory.

This PR brings back ShuffledHashJoin, basically revert #9645, and fix the conflict. Also merging outer join and left-semi join into the same class. This PR does not implement full outer join, because it's not implemented efficiently (requiring build hash table on both side).

A simple benchmark (one table is 5x smaller than other one) show that ShuffledHashJoin could be 2X faster than SortMergeJoin.

## How was this patch tested?

Added new unit tests for ShuffledHashJoin.

Author: Davies Liu <davies@databricks.com>

Closes #11788 from davies/shuffle_join.
2016-03-18 10:32:53 -07:00
Cheng Lian 14c7236dc6 [SPARK-14004][SQL][MINOR] AttributeReference and Alias should only use the first qualifier to generate SQL strings
## What changes were proposed in this pull request?

Current implementations of `AttributeReference.sql` and `Alias.sql` joins all available qualifiers, which is logically wrong. But this implementation mistake doesn't cause any real SQL generation bugs though, since there is always at most one qualifier for any given `AttributeReference` or `Alias`.

This PR fixes this issue by only picking the first qualifiers.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

Existing tests should be enough.

Author: Cheng Lian <lian@databricks.com>

Closes #11820 from liancheng/spark-14004-single-qualifier.
2016-03-19 00:22:17 +08:00
Wenchen Fan 0acb32a3f1 [SPARK-13972][SQ] hive tests should fail if SQL generation failed
## What changes were proposed in this pull request?

Now we should be able to convert all logical plans to SQL string, if they are parsed from hive query. This PR changes the error handling to throw exceptions instead of just log.

We will send new PRs for spotted bugs, and merge this one after all bugs are fixed.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11782 from cloud-fan/test.
2016-03-18 23:16:14 +08:00