Commit graph

5608 commits

Author SHA1 Message Date
Xianyang Liu 87ab0cec65 [SPARK-21072][SQL] TreeNode.mapChildren should only apply to the children node.
## What changes were proposed in this pull request?

Just as the function name and comments of `TreeNode.mapChildren` mentioned, the function should be apply to all currently node children. So, the follow code should judge whether it is the children node.

https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala#L342

## How was this patch tested?

Existing tests.

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #18284 from ConeyLiu/treenode.
2017-06-16 12:10:09 +08:00
Xiao Li 5d35d5c15c [SPARK-21112][SQL] ALTER TABLE SET TBLPROPERTIES should not overwrite COMMENT
### What changes were proposed in this pull request?
`ALTER TABLE SET TBLPROPERTIES` should not overwrite `COMMENT` even if the input property does not have the property of `COMMENT`. This PR is to fix the issue.

### How was this patch tested?
Covered by the existing tests.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18318 from gatorsmile/fixTableComment.
2017-06-16 10:11:23 +08:00
ALeksander Eskilson b32b2123dd [SPARK-18016][SQL][CATALYST] Code Generation: Constant Pool Limit - Class Splitting
## What changes were proposed in this pull request?

This pull-request exclusively includes the class splitting feature described in #16648. When code for a given class would grow beyond 1600k bytes, a private, nested sub-class is generated into which subsequent functions are inlined. Additional sub-classes are generated as the code threshold is met subsequent times. This code includes 3 changes:

1. Includes helper maps, lists, and functions for keeping track of sub-classes during code generation (included in the `CodeGenerator` class). These helper functions allow nested classes and split functions to be initialized/declared/inlined to the appropriate locations in the various projection classes.
2. Changes `addNewFunction` to return a string to support instances where a split function is inlined to a nested class and not the outer class (and so must be invoked using the class-qualified name). Uses of `addNewFunction` throughout the codebase are modified so that the returned name is properly used.
3. Removes instances of the `this` keyword when used on data inside generated classes. All state declared in the outer class is by default global and accessible to the nested classes. However, if a reference to global state in a nested class is prepended with the `this` keyword, it would attempt to reference state belonging to the nested class (which would not exist), rather than the correct variable belonging to the outer class.

## How was this patch tested?

Added a test case to the `GeneratedProjectionSuite` that increases the number of columns tested in various projections to a threshold that would previously have triggered a `JaninoRuntimeException` for the Constant Pool.

Note: This PR does not address the second Constant Pool issue with code generation (also mentioned in #16648): excess global mutable state. A second PR may be opened to resolve that issue.

Author: ALeksander Eskilson <alek.eskilson@cerner.com>

Closes #18075 from bdrillard/class_splitting_only.
2017-06-15 13:45:08 +08:00
Xiao Li 2051428173 [SPARK-20980][SQL] Rename wholeFile to multiLine for both CSV and JSON
### What changes were proposed in this pull request?
The current option name `wholeFile` is misleading for CSV users. Currently, it is not representing a record per file. Actually, one file could have multiple records. Thus, we should rename it. Now, the proposal is `multiLine`.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18202 from gatorsmile/renameCVSOption.
2017-06-15 13:18:19 +08:00
Reynold Xin fffeb6d7c3 [SPARK-21092][SQL] Wire SQLConf in logical plan and expressions
## What changes were proposed in this pull request?
It is really painful to not have configs in logical plan and expressions. We had to add all sorts of hacks (e.g. pass SQLConf explicitly in functions). This patch exposes SQLConf in logical plan, using a thread local variable and a getter closure that's set once there is an active SparkSession.

The implementation is a bit of a hack, since we didn't anticipate this need in the beginning (config was only exposed in physical plan). The implementation is described in `SQLConf.get`.

In terms of future work, we should follow up to clean up CBO (remove the need for passing in config).

## How was this patch tested?
Updated relevant tests for constraint propagation.

Author: Reynold Xin <rxin@databricks.com>

Closes #18299 from rxin/SPARK-21092.
2017-06-14 22:11:41 -07:00
Reynold Xin e254e868f1 [SPARK-21091][SQL] Move constraint code into QueryPlanConstraints
## What changes were proposed in this pull request?
This patch moves constraint related code into a separate trait QueryPlanConstraints, so we don't litter QueryPlan with a lot of constraint private functions.

## How was this patch tested?
This is a simple move refactoring and should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #18298 from rxin/SPARK-21091.
2017-06-14 14:28:21 -07:00
Xiao Li 77a2fc5b52 Revert "[SPARK-20941][SQL] Fix SubqueryExec Reuse"
This reverts commit f7cf2096fd.
2017-06-14 11:48:32 -07:00
Xiao Li df766a4714 [SPARK-21089][SQL] Fix DESC EXTENDED/FORMATTED to Show Table Properties
### What changes were proposed in this pull request?

Since both table properties and storage properties share the same key values, table properties are not shown in the output of DESC EXTENDED/FORMATTED when the storage properties are not empty.

This PR is to fix the above issue by renaming them to different keys.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18294 from gatorsmile/tableProperties.
2017-06-14 11:13:16 -07:00
gatorsmile 0c88e8d372 [SPARK-21085][SQL] Failed to read the partitioned table created by Spark 2.1
### What changes were proposed in this pull request?
Before the PR, Spark is unable to read the partitioned table created by Spark 2.1 when the table schema does not put the partitioning column at the end of the schema.
[assert(partitionFields.map(_.name) == partitionColumnNames)](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala#L234-L236)

When reading the table metadata from the metastore, we also need to reorder the columns.

### How was this patch tested?
Added test cases to check both Hive-serde and data source tables.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18295 from gatorsmile/reorderReadSchema.
2017-06-14 16:28:06 +08:00
Yuming Wang 4d01aa4648 [SPARK-20754][SQL][FOLLOWUP] Add Function Alias For MOD/POSITION.
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/18106 Support TRUNC (number),  We should also add function alias for `MOD `and `POSITION`.

`POSITION(substr IN str) `is a synonym for `LOCATE(substr,str)`. same as MySQL: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_position

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18206 from wangyum/SPARK-20754-mod&position.
2017-06-13 23:39:06 -07:00
lianhuiwang 8b5b2e272f [SPARK-20986][SQL] Reset table's statistics after PruneFileSourcePartitions rule.
## What changes were proposed in this pull request?
After PruneFileSourcePartitions rule, It needs reset table's statistics because PruneFileSourcePartitions can filter some unnecessary partitions. So the statistics need to be changed.

## How was this patch tested?
add unit test.

Author: lianhuiwang <lianhuiwang09@gmail.com>

Closes #18205 from lianhuiwang/SPARK-20986.
2017-06-14 09:57:56 +08:00
liuxian 7ba8bf288d [SPARK-21016][CORE] Improve code fault tolerance for converting string to number
## What changes were proposed in this pull request?
When converting `string` to `number`(int, long or double),  if the string has a space before or after,will lead to unnecessary mistakes.

## How was this patch tested?
unit test

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18238 from 10110346/lx-wip-0608.
2017-06-13 10:12:28 -07:00
Liang-Chi Hsieh bcf3643f94 [SPARK-21051][SQL] Add hash map metrics to aggregate
## What changes were proposed in this pull request?

This adds the average hash map probe metrics to hash aggregate.

`BytesToBytesMap` already has API to get the metrics, this PR adds an API to `UnsafeFixedWidthAggregationMap` to access it.

Preparing a test for this metrics seems tricky, because we don't know what collision keys are. For now, the test case generates random data large enough to have desired probe.

TODO in later PR: add hash map metrics to join.

## How was this patch tested?

Added test to SQLMetricsSuite.

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

Closes #18258 from viirya/SPARK-20953.
2017-06-13 10:10:35 -07:00
Rishabh Bhardwaj 9b2c877bec [SPARK-21039][SPARK CORE] Use treeAggregate instead of aggregate in DataFrame.stat.bloomFilter
## What changes were proposed in this pull request?
To use treeAggregate instead of aggregate in DataFrame.stat.bloomFilter to parallelize the operation of merging the bloom filters
(Please fill in changes proposed in this fix)

## How was this patch tested?
unit tests passed
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Rishabh Bhardwaj <rbnext29@gmail.com>
Author: Rishabh Bhardwaj <admin@rishabh.local>
Author: Rishabh Bhardwaj <r0b00ko@rishabh.Dlink>
Author: Rishabh Bhardwaj <admin@Admins-MacBook-Pro.local>
Author: Rishabh Bhardwaj <r0b00ko@rishabh.local>

Closes #18263 from rishabhbhardwaj/SPARK-21039.
2017-06-13 15:09:12 +01:00
Sean Owen 7b7c85ede3 [SPARK-20920][SQL] ForkJoinPool pools are leaked when writing hive tables with many partitions
## What changes were proposed in this pull request?

Don't leave thread pool running from AlterTableRecoverPartitionsCommand DDL command

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #18216 from srowen/SPARK-20920.
2017-06-13 10:48:07 +01:00
Dongjoon Hyun 2639c3ed03 [SPARK-19910][SQL] stack should not reject NULL values due to type mismatch
## What changes were proposed in this pull request?

Since `stack` function generates a table with nullable columns, it should allow mixed null values.

```scala
scala> sql("select stack(3, 1, 2, 3)").printSchema
root
 |-- col0: integer (nullable = true)

scala> sql("select stack(3, 1, 2, null)").printSchema
org.apache.spark.sql.AnalysisException: cannot resolve 'stack(3, 1, 2, NULL)' due to data type mismatch: Argument 1 (IntegerType) != Argument 3 (NullType); line 1 pos 7;
```

## How was this patch tested?

Pass the Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17251 from dongjoon-hyun/SPARK-19910.
2017-06-12 21:18:43 -07:00
Wenchen Fan fc0e6944a5 Revert "[SPARK-21046][SQL] simplify the array offset and length in ColumnVector"
This reverts commit 22dd65f58e.
2017-06-13 09:15:14 +08:00
Shixiong Zhu 74a432d3a3 [SPARK-20979][SS] Add RateSource to generate values for tests and benchmark
## What changes were proposed in this pull request?

This PR adds RateSource for Structured Streaming so that the user can use it to generate data for tests and benchmark easily.

This source generates increment long values with timestamps. Each generated row has two columns: a timestamp column for the generated time and an auto increment long column starting with 0L.

It supports the following options:
- `rowsPerSecond` (e.g. 100, default: 1): How many rows should be generated per second.
- `rampUpTime` (e.g. 5s, default: 0s): How long to ramp up before the generating speed becomes `rowsPerSecond`. Using finer granularities than seconds will be truncated to integer seconds.
- `numPartitions` (e.g. 10, default: Spark's default parallelism): The partition number for the generated rows. The source will try its best to reach `rowsPerSecond`, but the query may be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed.

Here is a simple example that prints 10 rows per seconds:
```
    spark.readStream
      .format("rate")
      .option("rowsPerSecond", "10")
      .load()
      .writeStream
      .format("console")
      .start()
```

The idea came from marmbrus and he did the initial work.

## How was this patch tested?

The added tests.

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #18199 from zsxwing/rate.
2017-06-12 14:58:08 -07:00
Reynold Xin b1436c7496 [SPARK-21059][SQL] LikeSimplification can NPE on null pattern
## What changes were proposed in this pull request?
This patch fixes a bug that can cause NullPointerException in LikeSimplification, when the pattern for like is null.

## How was this patch tested?
Added a new unit test case in LikeSimplificationSuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #18273 from rxin/SPARK-21059.
2017-06-12 14:07:51 -07:00
Dongjoon Hyun 32818d9b37 [SPARK-20345][SQL] Fix STS error handling logic on HiveSQLException
## What changes were proposed in this pull request?

[SPARK-5100](343d3bfafd) added Spark Thrift Server(STS) UI and the following logic to handle exceptions on case `Throwable`.

```scala
HiveThriftServer2.listener.onStatementError(
  statementId, e.getMessage, SparkUtils.exceptionString(e))
```

However, there occurred a missed case after implementing [SPARK-6964](eb19d3f75c)'s `Support Cancellation in the Thrift Server` by adding case `HiveSQLException` before case `Throwable`.

```scala
case e: HiveSQLException =>
  if (getStatus().getState() == OperationState.CANCELED) {
    return
  } else {
    setState(OperationState.ERROR)
    throw e
  }
  // Actually do need to catch Throwable as some failures don't inherit from Exception and
  // HiveServer will silently swallow them.
case e: Throwable =>
  val currentState = getStatus().getState()
  logError(s"Error executing query, currentState $currentState, ", e)
  setState(OperationState.ERROR)
  HiveThriftServer2.listener.onStatementError(
    statementId, e.getMessage, SparkUtils.exceptionString(e))
  throw new HiveSQLException(e.toString)
```

Logically, we had better add `HiveThriftServer2.listener.onStatementError` on case `HiveSQLException`, too.

## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17643 from dongjoon-hyun/SPARK-20345.
2017-06-12 14:05:03 -07:00
aokolnychyi ca4e960aec [SPARK-17914][SQL] Fix parsing of timestamp strings with nanoseconds
The PR contains a tiny change to fix the way Spark parses string literals into timestamps. Currently, some timestamps that contain nanoseconds are corrupted during the conversion from internal UTF8Strings into the internal representation of timestamps.

Consider the following example:
```
spark.sql("SELECT cast('2015-01-02 00:00:00.000000001' as TIMESTAMP)").show(false)
+------------------------------------------------+
|CAST(2015-01-02 00:00:00.000000001 AS TIMESTAMP)|
+------------------------------------------------+
|2015-01-02 00:00:00.000001                      |
+------------------------------------------------+
```

The fix was tested with existing tests. Also, there is a new test to cover cases that did not work previously.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #18252 from aokolnychyi/spark-17914.
2017-06-12 13:06:14 -07:00
Wenchen Fan 22dd65f58e [SPARK-21046][SQL] simplify the array offset and length in ColumnVector
## What changes were proposed in this pull request?

Currently when a `ColumnVector` stores array type elements, we will use 2 arrays for lengths and offsets and implement them individually in on-heap and off-heap column vector.

In this PR, we use one array to represent both offsets and lengths, so that we can treat it as `ColumnVector` and all the logic can go to the base class `ColumnVector`

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18260 from cloud-fan/put.
2017-06-13 00:12:34 +08:00
Dongjoon Hyun a92e095e70 [SPARK-21041][SQL] SparkSession.range should be consistent with SparkContext.range
## What changes were proposed in this pull request?

This PR fixes the inconsistency in `SparkSession.range`.

**BEFORE**
```scala
scala> spark.range(java.lang.Long.MAX_VALUE - 3, java.lang.Long.MIN_VALUE + 2, 1).collect
res2: Array[Long] = Array(9223372036854775804, 9223372036854775805, 9223372036854775806)
```

**AFTER**
```scala
scala> spark.range(java.lang.Long.MAX_VALUE - 3, java.lang.Long.MIN_VALUE + 2, 1).collect
res2: Array[Long] = Array()
```

## How was this patch tested?

Pass the Jenkins with newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18257 from dongjoon-hyun/SPARK-21041.
2017-06-12 20:58:27 +08:00
liuxian d140918093 [SPARK-20665][SQL][FOLLOW-UP] Move test case to MathExpressionsSuite
## What changes were proposed in this pull request?

 add test case to MathExpressionsSuite as #17906

## How was this patch tested?

unit test cases

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18082 from 10110346/wip-lx-0524.
2017-06-11 22:29:09 -07:00
Michal Senkyr f48273c13c [SPARK-18891][SQL] Support for specific Java List subtypes
## What changes were proposed in this pull request?

Add support for specific Java `List` subtypes in deserialization as well as a generic implicit encoder.

All `List` subtypes are supported by using either the size-specifying constructor (one `int` parameter) or the default constructor.

Interfaces/abstract classes use the following implementations:

* `java.util.List`, `java.util.AbstractList` or `java.util.AbstractSequentialList` => `java.util.ArrayList`

## How was this patch tested?

```bash
build/mvn -DskipTests clean package && dev/run-tests
```

Additionally in Spark shell:

```
scala> val jlist = new java.util.LinkedList[Int]; jlist.add(1)
jlist: java.util.LinkedList[Int] = [1]
res0: Boolean = true

scala> Seq(jlist).toDS().map(_.element()).collect()
res1: Array[Int] = Array(1)
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #18009 from michalsenkyr/dataset-java-lists.
2017-06-12 08:53:23 +08:00
Michal Senkyr 0538f3b0ae [SPARK-18891][SQL] Support for Scala Map collection types
## What changes were proposed in this pull request?

Add support for arbitrary Scala `Map` types in deserialization as well as a generic implicit encoder.

Used the builder approach as in #16541 to construct any provided `Map` type upon deserialization.

Please note that this PR also adds (ignored) tests for issue [SPARK-19104 CompileException with Map and Case Class in Spark 2.1.0](https://issues.apache.org/jira/browse/SPARK-19104) but doesn't solve it.

Added support for Java Maps in codegen code (encoders will be added in a different PR) with the following default implementations for interfaces/abstract classes:

* `java.util.Map`, `java.util.AbstractMap` => `java.util.HashMap`
* `java.util.SortedMap`, `java.util.NavigableMap` => `java.util.TreeMap`
* `java.util.concurrent.ConcurrentMap` => `java.util.concurrent.ConcurrentHashMap`
* `java.util.concurrent.ConcurrentNavigableMap` => `java.util.concurrent.ConcurrentSkipListMap`

Resulting codegen for `Seq(Map(1 -> 2)).toDS().map(identity).queryExecution.debug.codegen`:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private boolean CollectObjectsToMap_loopIsNull1;
/* 010 */   private int CollectObjectsToMap_loopValue0;
/* 011 */   private boolean CollectObjectsToMap_loopIsNull3;
/* 012 */   private int CollectObjectsToMap_loopValue2;
/* 013 */   private UnsafeRow deserializetoobject_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 016 */   private scala.collection.immutable.Map mapelements_argValue;
/* 017 */   private UnsafeRow mapelements_result;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 020 */   private UnsafeRow serializefromobject_result;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter1;
/* 025 */
/* 026 */   public GeneratedIterator(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 031 */     partitionIndex = index;
/* 032 */     this.inputs = inputs;
/* 033 */     wholestagecodegen_init_0();
/* 034 */     wholestagecodegen_init_1();
/* 035 */
/* 036 */   }
/* 037 */
/* 038 */   private void wholestagecodegen_init_0() {
/* 039 */     inputadapter_input = inputs[0];
/* 040 */
/* 041 */     deserializetoobject_result = new UnsafeRow(1);
/* 042 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 043 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 044 */
/* 045 */     mapelements_result = new UnsafeRow(1);
/* 046 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 047 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 048 */     serializefromobject_result = new UnsafeRow(1);
/* 049 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 050 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 051 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 052 */
/* 053 */   }
/* 054 */
/* 055 */   private void wholestagecodegen_init_1() {
/* 056 */     this.serializefromobject_arrayWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 057 */
/* 058 */   }
/* 059 */
/* 060 */   protected void processNext() throws java.io.IOException {
/* 061 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 062 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 063 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 064 */       MapData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getMap(0));
/* 065 */
/* 066 */       boolean deserializetoobject_isNull1 = true;
/* 067 */       ArrayData deserializetoobject_value1 = null;
/* 068 */       if (!inputadapter_isNull) {
/* 069 */         deserializetoobject_isNull1 = false;
/* 070 */         if (!deserializetoobject_isNull1) {
/* 071 */           Object deserializetoobject_funcResult = null;
/* 072 */           deserializetoobject_funcResult = inputadapter_value.keyArray();
/* 073 */           if (deserializetoobject_funcResult == null) {
/* 074 */             deserializetoobject_isNull1 = true;
/* 075 */           } else {
/* 076 */             deserializetoobject_value1 = (ArrayData) deserializetoobject_funcResult;
/* 077 */           }
/* 078 */
/* 079 */         }
/* 080 */         deserializetoobject_isNull1 = deserializetoobject_value1 == null;
/* 081 */       }
/* 082 */
/* 083 */       boolean deserializetoobject_isNull3 = true;
/* 084 */       ArrayData deserializetoobject_value3 = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull3 = false;
/* 087 */         if (!deserializetoobject_isNull3) {
/* 088 */           Object deserializetoobject_funcResult1 = null;
/* 089 */           deserializetoobject_funcResult1 = inputadapter_value.valueArray();
/* 090 */           if (deserializetoobject_funcResult1 == null) {
/* 091 */             deserializetoobject_isNull3 = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value3 = (ArrayData) deserializetoobject_funcResult1;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull3 = deserializetoobject_value3 == null;
/* 098 */       }
/* 099 */       scala.collection.immutable.Map deserializetoobject_value = null;
/* 100 */
/* 101 */       if ((deserializetoobject_isNull1 && !deserializetoobject_isNull3) ||
/* 102 */         (!deserializetoobject_isNull1 && deserializetoobject_isNull3)) {
/* 103 */         throw new RuntimeException("Invalid state: Inconsistent nullability of key-value");
/* 104 */       }
/* 105 */
/* 106 */       if (!deserializetoobject_isNull1) {
/* 107 */         if (deserializetoobject_value1.numElements() != deserializetoobject_value3.numElements()) {
/* 108 */           throw new RuntimeException("Invalid state: Inconsistent lengths of key-value arrays");
/* 109 */         }
/* 110 */         int deserializetoobject_dataLength = deserializetoobject_value1.numElements();
/* 111 */
/* 112 */         scala.collection.mutable.Builder CollectObjectsToMap_builderValue5 = scala.collection.immutable.Map$.MODULE$.newBuilder();
/* 113 */         CollectObjectsToMap_builderValue5.sizeHint(deserializetoobject_dataLength);
/* 114 */
/* 115 */         int deserializetoobject_loopIndex = 0;
/* 116 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 117 */           CollectObjectsToMap_loopValue0 = (int) (deserializetoobject_value1.getInt(deserializetoobject_loopIndex));
/* 118 */           CollectObjectsToMap_loopValue2 = (int) (deserializetoobject_value3.getInt(deserializetoobject_loopIndex));
/* 119 */           CollectObjectsToMap_loopIsNull1 = deserializetoobject_value1.isNullAt(deserializetoobject_loopIndex);
/* 120 */           CollectObjectsToMap_loopIsNull3 = deserializetoobject_value3.isNullAt(deserializetoobject_loopIndex);
/* 121 */
/* 122 */           if (CollectObjectsToMap_loopIsNull1) {
/* 123 */             throw new RuntimeException("Found null in map key!");
/* 124 */           }
/* 125 */
/* 126 */           scala.Tuple2 CollectObjectsToMap_loopValue4;
/* 127 */
/* 128 */           if (CollectObjectsToMap_loopIsNull3) {
/* 129 */             CollectObjectsToMap_loopValue4 = new scala.Tuple2(CollectObjectsToMap_loopValue0, null);
/* 130 */           } else {
/* 131 */             CollectObjectsToMap_loopValue4 = new scala.Tuple2(CollectObjectsToMap_loopValue0, CollectObjectsToMap_loopValue2);
/* 132 */           }
/* 133 */
/* 134 */           CollectObjectsToMap_builderValue5.$plus$eq(CollectObjectsToMap_loopValue4);
/* 135 */
/* 136 */           deserializetoobject_loopIndex += 1;
/* 137 */         }
/* 138 */
/* 139 */         deserializetoobject_value = (scala.collection.immutable.Map) CollectObjectsToMap_builderValue5.result();
/* 140 */       }
/* 141 */
/* 142 */       boolean mapelements_isNull = true;
/* 143 */       scala.collection.immutable.Map mapelements_value = null;
/* 144 */       if (!false) {
/* 145 */         mapelements_argValue = deserializetoobject_value;
/* 146 */
/* 147 */         mapelements_isNull = false;
/* 148 */         if (!mapelements_isNull) {
/* 149 */           Object mapelements_funcResult = null;
/* 150 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 151 */           if (mapelements_funcResult == null) {
/* 152 */             mapelements_isNull = true;
/* 153 */           } else {
/* 154 */             mapelements_value = (scala.collection.immutable.Map) mapelements_funcResult;
/* 155 */           }
/* 156 */
/* 157 */         }
/* 158 */         mapelements_isNull = mapelements_value == null;
/* 159 */       }
/* 160 */
/* 161 */       MapData serializefromobject_value = null;
/* 162 */       if (!mapelements_isNull) {
/* 163 */         final int serializefromobject_length = mapelements_value.size();
/* 164 */         final Object[] serializefromobject_convertedKeys = new Object[serializefromobject_length];
/* 165 */         final Object[] serializefromobject_convertedValues = new Object[serializefromobject_length];
/* 166 */         int serializefromobject_index = 0;
/* 167 */         final scala.collection.Iterator serializefromobject_entries = mapelements_value.iterator();
/* 168 */         while(serializefromobject_entries.hasNext()) {
/* 169 */           final scala.Tuple2 serializefromobject_entry = (scala.Tuple2) serializefromobject_entries.next();
/* 170 */           int ExternalMapToCatalyst_key1 = (Integer) serializefromobject_entry._1();
/* 171 */           int ExternalMapToCatalyst_value1 = (Integer) serializefromobject_entry._2();
/* 172 */
/* 173 */           boolean ExternalMapToCatalyst_value_isNull1 = false;
/* 174 */
/* 175 */           if (false) {
/* 176 */             throw new RuntimeException("Cannot use null as map key!");
/* 177 */           } else {
/* 178 */             serializefromobject_convertedKeys[serializefromobject_index] = (Integer) ExternalMapToCatalyst_key1;
/* 179 */           }
/* 180 */
/* 181 */           if (false) {
/* 182 */             serializefromobject_convertedValues[serializefromobject_index] = null;
/* 183 */           } else {
/* 184 */             serializefromobject_convertedValues[serializefromobject_index] = (Integer) ExternalMapToCatalyst_value1;
/* 185 */           }
/* 186 */
/* 187 */           serializefromobject_index++;
/* 188 */         }
/* 189 */
/* 190 */         serializefromobject_value = new org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedKeys), new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedValues));
/* 191 */       }
/* 192 */       serializefromobject_holder.reset();
/* 193 */
/* 194 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 195 */
/* 196 */       if (mapelements_isNull) {
/* 197 */         serializefromobject_rowWriter.setNullAt(0);
/* 198 */       } else {
/* 199 */         // Remember the current cursor so that we can calculate how many bytes are
/* 200 */         // written later.
/* 201 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 202 */
/* 203 */         if (serializefromobject_value instanceof UnsafeMapData) {
/* 204 */           final int serializefromobject_sizeInBytes = ((UnsafeMapData) serializefromobject_value).getSizeInBytes();
/* 205 */           // grow the global buffer before writing data.
/* 206 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 207 */           ((UnsafeMapData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 208 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 209 */
/* 210 */         } else {
/* 211 */           final ArrayData serializefromobject_keys = serializefromobject_value.keyArray();
/* 212 */           final ArrayData serializefromobject_values = serializefromobject_value.valueArray();
/* 213 */
/* 214 */           // preserve 8 bytes to write the key array numBytes later.
/* 215 */           serializefromobject_holder.grow(8);
/* 216 */           serializefromobject_holder.cursor += 8;
/* 217 */
/* 218 */           // Remember the current cursor so that we can write numBytes of key array later.
/* 219 */           final int serializefromobject_tmpCursor1 = serializefromobject_holder.cursor;
/* 220 */
/* 221 */           if (serializefromobject_keys instanceof UnsafeArrayData) {
/* 222 */             final int serializefromobject_sizeInBytes1 = ((UnsafeArrayData) serializefromobject_keys).getSizeInBytes();
/* 223 */             // grow the global buffer before writing data.
/* 224 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes1);
/* 225 */             ((UnsafeArrayData) serializefromobject_keys).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 226 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes1;
/* 227 */
/* 228 */           } else {
/* 229 */             final int serializefromobject_numElements = serializefromobject_keys.numElements();
/* 230 */             serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 231 */
/* 232 */             for (int serializefromobject_index1 = 0; serializefromobject_index1 < serializefromobject_numElements; serializefromobject_index1++) {
/* 233 */               if (serializefromobject_keys.isNullAt(serializefromobject_index1)) {
/* 234 */                 serializefromobject_arrayWriter.setNullInt(serializefromobject_index1);
/* 235 */               } else {
/* 236 */                 final int serializefromobject_element = serializefromobject_keys.getInt(serializefromobject_index1);
/* 237 */                 serializefromobject_arrayWriter.write(serializefromobject_index1, serializefromobject_element);
/* 238 */               }
/* 239 */             }
/* 240 */           }
/* 241 */
/* 242 */           // Write the numBytes of key array into the first 8 bytes.
/* 243 */           Platform.putLong(serializefromobject_holder.buffer, serializefromobject_tmpCursor1 - 8, serializefromobject_holder.cursor - serializefromobject_tmpCursor1);
/* 244 */
/* 245 */           if (serializefromobject_values instanceof UnsafeArrayData) {
/* 246 */             final int serializefromobject_sizeInBytes2 = ((UnsafeArrayData) serializefromobject_values).getSizeInBytes();
/* 247 */             // grow the global buffer before writing data.
/* 248 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes2);
/* 249 */             ((UnsafeArrayData) serializefromobject_values).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 250 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes2;
/* 251 */
/* 252 */           } else {
/* 253 */             final int serializefromobject_numElements1 = serializefromobject_values.numElements();
/* 254 */             serializefromobject_arrayWriter1.initialize(serializefromobject_holder, serializefromobject_numElements1, 4);
/* 255 */
/* 256 */             for (int serializefromobject_index2 = 0; serializefromobject_index2 < serializefromobject_numElements1; serializefromobject_index2++) {
/* 257 */               if (serializefromobject_values.isNullAt(serializefromobject_index2)) {
/* 258 */                 serializefromobject_arrayWriter1.setNullInt(serializefromobject_index2);
/* 259 */               } else {
/* 260 */                 final int serializefromobject_element1 = serializefromobject_values.getInt(serializefromobject_index2);
/* 261 */                 serializefromobject_arrayWriter1.write(serializefromobject_index2, serializefromobject_element1);
/* 262 */               }
/* 263 */             }
/* 264 */           }
/* 265 */
/* 266 */         }
/* 267 */
/* 268 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 269 */       }
/* 270 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 271 */       append(serializefromobject_result);
/* 272 */       if (shouldStop()) return;
/* 273 */     }
/* 274 */   }
/* 275 */ }
```

Codegen for `java.util.Map`:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private boolean CollectObjectsToMap_loopIsNull1;
/* 010 */   private int CollectObjectsToMap_loopValue0;
/* 011 */   private boolean CollectObjectsToMap_loopIsNull3;
/* 012 */   private int CollectObjectsToMap_loopValue2;
/* 013 */   private UnsafeRow deserializetoobject_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 016 */   private java.util.HashMap mapelements_argValue;
/* 017 */   private UnsafeRow mapelements_result;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 020 */   private UnsafeRow serializefromobject_result;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter1;
/* 025 */
/* 026 */   public GeneratedIterator(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 031 */     partitionIndex = index;
/* 032 */     this.inputs = inputs;
/* 033 */     wholestagecodegen_init_0();
/* 034 */     wholestagecodegen_init_1();
/* 035 */
/* 036 */   }
/* 037 */
/* 038 */   private void wholestagecodegen_init_0() {
/* 039 */     inputadapter_input = inputs[0];
/* 040 */
/* 041 */     deserializetoobject_result = new UnsafeRow(1);
/* 042 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 043 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 044 */
/* 045 */     mapelements_result = new UnsafeRow(1);
/* 046 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 047 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 048 */     serializefromobject_result = new UnsafeRow(1);
/* 049 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 050 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 051 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 052 */
/* 053 */   }
/* 054 */
/* 055 */   private void wholestagecodegen_init_1() {
/* 056 */     this.serializefromobject_arrayWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 057 */
/* 058 */   }
/* 059 */
/* 060 */   protected void processNext() throws java.io.IOException {
/* 061 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 062 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 063 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 064 */       MapData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getMap(0));
/* 065 */
/* 066 */       boolean deserializetoobject_isNull1 = true;
/* 067 */       ArrayData deserializetoobject_value1 = null;
/* 068 */       if (!inputadapter_isNull) {
/* 069 */         deserializetoobject_isNull1 = false;
/* 070 */         if (!deserializetoobject_isNull1) {
/* 071 */           Object deserializetoobject_funcResult = null;
/* 072 */           deserializetoobject_funcResult = inputadapter_value.keyArray();
/* 073 */           if (deserializetoobject_funcResult == null) {
/* 074 */             deserializetoobject_isNull1 = true;
/* 075 */           } else {
/* 076 */             deserializetoobject_value1 = (ArrayData) deserializetoobject_funcResult;
/* 077 */           }
/* 078 */
/* 079 */         }
/* 080 */         deserializetoobject_isNull1 = deserializetoobject_value1 == null;
/* 081 */       }
/* 082 */
/* 083 */       boolean deserializetoobject_isNull3 = true;
/* 084 */       ArrayData deserializetoobject_value3 = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull3 = false;
/* 087 */         if (!deserializetoobject_isNull3) {
/* 088 */           Object deserializetoobject_funcResult1 = null;
/* 089 */           deserializetoobject_funcResult1 = inputadapter_value.valueArray();
/* 090 */           if (deserializetoobject_funcResult1 == null) {
/* 091 */             deserializetoobject_isNull3 = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value3 = (ArrayData) deserializetoobject_funcResult1;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull3 = deserializetoobject_value3 == null;
/* 098 */       }
/* 099 */       java.util.HashMap deserializetoobject_value = null;
/* 100 */
/* 101 */       if ((deserializetoobject_isNull1 && !deserializetoobject_isNull3) ||
/* 102 */         (!deserializetoobject_isNull1 && deserializetoobject_isNull3)) {
/* 103 */         throw new RuntimeException("Invalid state: Inconsistent nullability of key-value");
/* 104 */       }
/* 105 */
/* 106 */       if (!deserializetoobject_isNull1) {
/* 107 */         if (deserializetoobject_value1.numElements() != deserializetoobject_value3.numElements()) {
/* 108 */           throw new RuntimeException("Invalid state: Inconsistent lengths of key-value arrays");
/* 109 */         }
/* 110 */         int deserializetoobject_dataLength = deserializetoobject_value1.numElements();
/* 111 */         java.util.Map CollectObjectsToMap_builderValue5 = new java.util.HashMap(deserializetoobject_dataLength);
/* 112 */
/* 113 */         int deserializetoobject_loopIndex = 0;
/* 114 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 115 */           CollectObjectsToMap_loopValue0 = (int) (deserializetoobject_value1.getInt(deserializetoobject_loopIndex));
/* 116 */           CollectObjectsToMap_loopValue2 = (int) (deserializetoobject_value3.getInt(deserializetoobject_loopIndex));
/* 117 */           CollectObjectsToMap_loopIsNull1 = deserializetoobject_value1.isNullAt(deserializetoobject_loopIndex);
/* 118 */           CollectObjectsToMap_loopIsNull3 = deserializetoobject_value3.isNullAt(deserializetoobject_loopIndex);
/* 119 */
/* 120 */           if (CollectObjectsToMap_loopIsNull1) {
/* 121 */             throw new RuntimeException("Found null in map key!");
/* 122 */           }
/* 123 */
/* 124 */           CollectObjectsToMap_builderValue5.put(CollectObjectsToMap_loopValue0, CollectObjectsToMap_loopValue2);
/* 125 */
/* 126 */           deserializetoobject_loopIndex += 1;
/* 127 */         }
/* 128 */
/* 129 */         deserializetoobject_value = (java.util.HashMap) CollectObjectsToMap_builderValue5;
/* 130 */       }
/* 131 */
/* 132 */       boolean mapelements_isNull = true;
/* 133 */       java.util.HashMap mapelements_value = null;
/* 134 */       if (!false) {
/* 135 */         mapelements_argValue = deserializetoobject_value;
/* 136 */
/* 137 */         mapelements_isNull = false;
/* 138 */         if (!mapelements_isNull) {
/* 139 */           Object mapelements_funcResult = null;
/* 140 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 141 */           if (mapelements_funcResult == null) {
/* 142 */             mapelements_isNull = true;
/* 143 */           } else {
/* 144 */             mapelements_value = (java.util.HashMap) mapelements_funcResult;
/* 145 */           }
/* 146 */
/* 147 */         }
/* 148 */         mapelements_isNull = mapelements_value == null;
/* 149 */       }
/* 150 */
/* 151 */       MapData serializefromobject_value = null;
/* 152 */       if (!mapelements_isNull) {
/* 153 */         final int serializefromobject_length = mapelements_value.size();
/* 154 */         final Object[] serializefromobject_convertedKeys = new Object[serializefromobject_length];
/* 155 */         final Object[] serializefromobject_convertedValues = new Object[serializefromobject_length];
/* 156 */         int serializefromobject_index = 0;
/* 157 */         final java.util.Iterator serializefromobject_entries = mapelements_value.entrySet().iterator();
/* 158 */         while(serializefromobject_entries.hasNext()) {
/* 159 */           final java.util.Map$Entry serializefromobject_entry = (java.util.Map$Entry) serializefromobject_entries.next();
/* 160 */           int ExternalMapToCatalyst_key1 = (Integer) serializefromobject_entry.getKey();
/* 161 */           int ExternalMapToCatalyst_value1 = (Integer) serializefromobject_entry.getValue();
/* 162 */
/* 163 */           boolean ExternalMapToCatalyst_value_isNull1 = false;
/* 164 */
/* 165 */           if (false) {
/* 166 */             throw new RuntimeException("Cannot use null as map key!");
/* 167 */           } else {
/* 168 */             serializefromobject_convertedKeys[serializefromobject_index] = (Integer) ExternalMapToCatalyst_key1;
/* 169 */           }
/* 170 */
/* 171 */           if (false) {
/* 172 */             serializefromobject_convertedValues[serializefromobject_index] = null;
/* 173 */           } else {
/* 174 */             serializefromobject_convertedValues[serializefromobject_index] = (Integer) ExternalMapToCatalyst_value1;
/* 175 */           }
/* 176 */
/* 177 */           serializefromobject_index++;
/* 178 */         }
/* 179 */
/* 180 */         serializefromobject_value = new org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedKeys), new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedValues));
/* 181 */       }
/* 182 */       serializefromobject_holder.reset();
/* 183 */
/* 184 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 185 */
/* 186 */       if (mapelements_isNull) {
/* 187 */         serializefromobject_rowWriter.setNullAt(0);
/* 188 */       } else {
/* 189 */         // Remember the current cursor so that we can calculate how many bytes are
/* 190 */         // written later.
/* 191 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 192 */
/* 193 */         if (serializefromobject_value instanceof UnsafeMapData) {
/* 194 */           final int serializefromobject_sizeInBytes = ((UnsafeMapData) serializefromobject_value).getSizeInBytes();
/* 195 */           // grow the global buffer before writing data.
/* 196 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 197 */           ((UnsafeMapData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 198 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 199 */
/* 200 */         } else {
/* 201 */           final ArrayData serializefromobject_keys = serializefromobject_value.keyArray();
/* 202 */           final ArrayData serializefromobject_values = serializefromobject_value.valueArray();
/* 203 */
/* 204 */           // preserve 8 bytes to write the key array numBytes later.
/* 205 */           serializefromobject_holder.grow(8);
/* 206 */           serializefromobject_holder.cursor += 8;
/* 207 */
/* 208 */           // Remember the current cursor so that we can write numBytes of key array later.
/* 209 */           final int serializefromobject_tmpCursor1 = serializefromobject_holder.cursor;
/* 210 */
/* 211 */           if (serializefromobject_keys instanceof UnsafeArrayData) {
/* 212 */             final int serializefromobject_sizeInBytes1 = ((UnsafeArrayData) serializefromobject_keys).getSizeInBytes();
/* 213 */             // grow the global buffer before writing data.
/* 214 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes1);
/* 215 */             ((UnsafeArrayData) serializefromobject_keys).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 216 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes1;
/* 217 */
/* 218 */           } else {
/* 219 */             final int serializefromobject_numElements = serializefromobject_keys.numElements();
/* 220 */             serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 221 */
/* 222 */             for (int serializefromobject_index1 = 0; serializefromobject_index1 < serializefromobject_numElements; serializefromobject_index1++) {
/* 223 */               if (serializefromobject_keys.isNullAt(serializefromobject_index1)) {
/* 224 */                 serializefromobject_arrayWriter.setNullInt(serializefromobject_index1);
/* 225 */               } else {
/* 226 */                 final int serializefromobject_element = serializefromobject_keys.getInt(serializefromobject_index1);
/* 227 */                 serializefromobject_arrayWriter.write(serializefromobject_index1, serializefromobject_element);
/* 228 */               }
/* 229 */             }
/* 230 */           }
/* 231 */
/* 232 */           // Write the numBytes of key array into the first 8 bytes.
/* 233 */           Platform.putLong(serializefromobject_holder.buffer, serializefromobject_tmpCursor1 - 8, serializefromobject_holder.cursor - serializefromobject_tmpCursor1);
/* 234 */
/* 235 */           if (serializefromobject_values instanceof UnsafeArrayData) {
/* 236 */             final int serializefromobject_sizeInBytes2 = ((UnsafeArrayData) serializefromobject_values).getSizeInBytes();
/* 237 */             // grow the global buffer before writing data.
/* 238 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes2);
/* 239 */             ((UnsafeArrayData) serializefromobject_values).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 240 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes2;
/* 241 */
/* 242 */           } else {
/* 243 */             final int serializefromobject_numElements1 = serializefromobject_values.numElements();
/* 244 */             serializefromobject_arrayWriter1.initialize(serializefromobject_holder, serializefromobject_numElements1, 4);
/* 245 */
/* 246 */             for (int serializefromobject_index2 = 0; serializefromobject_index2 < serializefromobject_numElements1; serializefromobject_index2++) {
/* 247 */               if (serializefromobject_values.isNullAt(serializefromobject_index2)) {
/* 248 */                 serializefromobject_arrayWriter1.setNullInt(serializefromobject_index2);
/* 249 */               } else {
/* 250 */                 final int serializefromobject_element1 = serializefromobject_values.getInt(serializefromobject_index2);
/* 251 */                 serializefromobject_arrayWriter1.write(serializefromobject_index2, serializefromobject_element1);
/* 252 */               }
/* 253 */             }
/* 254 */           }
/* 255 */
/* 256 */         }
/* 257 */
/* 258 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 259 */       }
/* 260 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 261 */       append(serializefromobject_result);
/* 262 */       if (shouldStop()) return;
/* 263 */     }
/* 264 */   }
/* 265 */ }
```

## How was this patch tested?

```
build/mvn -DskipTests clean package && dev/run-tests
```

Additionally in Spark shell:

```
scala> Seq(collection.mutable.HashMap(1 -> 2, 2 -> 3)).toDS().map(_ += (3 -> 4)).collect()
res0: Array[scala.collection.mutable.HashMap[Int,Int]] = Array(Map(2 -> 3, 1 -> 2, 3 -> 4))
```

Author: Michal Senkyr <mike.senkyr@gmail.com>
Author: Michal Šenkýř <mike.senkyr@gmail.com>

Closes #16986 from michalsenkyr/dataset-map-builder.
2017-06-12 08:47:01 +08:00
Zhenhua Wang a7c61c100b [SPARK-21031][SQL] Add alterTableStats to store spark's stats and let alterTable keep existing stats
## What changes were proposed in this pull request?

Currently, hive's stats are read into `CatalogStatistics`, while spark's stats are also persisted through `CatalogStatistics`. As a result, hive's stats can be unexpectedly propagated into spark' stats.

For example, for a catalog table, we read stats from hive, e.g. "totalSize" and put it into `CatalogStatistics`. Then, by using "ALTER TABLE" command, we will store the stats in `CatalogStatistics` into metastore as spark's stats (because we don't know whether it's from spark or not). But spark's stats should be only generated by "ANALYZE" command. This is unexpected from this command.

Secondly, now that we have spark's stats in metastore, after inserting new data, although hive updated "totalSize" in metastore, we still cannot get the right `sizeInBytes` in `CatalogStatistics`, because we respect spark's stats (should not exist) over hive's stats.

A running example is shown in [JIRA](https://issues.apache.org/jira/browse/SPARK-21031).

To fix this, we add a new method `alterTableStats` to store spark's stats, and let `alterTable` keep existing stats.

## How was this patch tested?

Added new tests.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #18248 from wzhfy/separateHiveStats.
2017-06-12 08:23:04 +08:00
sujithjay 3a840048ed Fixed typo in sql.functions
## What changes were proposed in this pull request?

I fixed a typo in the Scaladoc for the method `def struct(cols: Column*): Column`. 'retained' was misspelt as 'remained'.

## How was this patch tested?
Before:

Creates a new struct column.
   If the input column is a column in a `DataFrame`, or a derived column expression
   that is named (i.e. aliased), its name would be **remained** as the StructField's name,
   otherwise, the newly generated StructField's name would be auto generated as
   `col` with a suffix `index + 1`, i.e. col1, col2, col3, ...

After:

   Creates a new struct column.
   If the input column is a column in a `DataFrame`, or a derived column expression
   that is named (i.e. aliased), its name would be **retained** as the StructField's name,
   otherwise, the newly generated StructField's name would be auto generated as
   `col` with a suffix `index + 1`, i.e. col1, col2, col3, ...

Author: sujithjay <sujith@logistimo.com>

Closes #18254 from sujithjay/fix-typo.
2017-06-11 18:23:57 +01:00
liuxian 5301a19a0e [SPARK-20620][TEST] Improve some unit tests for NullExpressionsSuite and TypeCoercionSuite
## What changes were proposed in this pull request?
add more  datatype for some unit tests

## How was this patch tested?
unit tests

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17880 from 10110346/wip_lx_0506.
2017-06-10 10:42:23 -07:00
Xiao Li 8e96acf71c [SPARK-20211][SQL] Fix the Precision and Scale of Decimal Values when the Input is BigDecimal between -1.0 and 1.0
### What changes were proposed in this pull request?
The precision and scale of decimal values are wrong when the input is BigDecimal between -1.0 and 1.0.

The BigDecimal's precision is the digit count starts from the leftmost nonzero digit based on the [JAVA's BigDecimal definition](https://docs.oracle.com/javase/7/docs/api/java/math/BigDecimal.html). However, our Decimal decision follows the database decimal standard, which is the total number of digits, including both to the left and the right of the decimal point. Thus, this PR is to fix the issue by doing the conversion.

Before this PR, the following queries failed:
```SQL
select 1 > 0.0001
select floor(0.0001)
select ceil(0.0001)
```

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18244 from gatorsmile/bigdecimal.
2017-06-10 10:28:14 -07:00
Reynold Xin b78e3849b2 [SPARK-21042][SQL] Document Dataset.union is resolution by position
## What changes were proposed in this pull request?
Document Dataset.union is resolution by position, not by name, since this has been a confusing point for a lot of users.

## How was this patch tested?
N/A - doc only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #18256 from rxin/SPARK-21042.
2017-06-09 18:29:33 -07:00
Xiao Li 571635488d [SPARK-20918][SQL] Use FunctionIdentifier as function identifiers in FunctionRegistry
### What changes were proposed in this pull request?
Currently, the unquoted string of a function identifier is being used as the function identifier in the function registry. This could cause the incorrect the behavior when users use `.` in the function names. This PR is to take the `FunctionIdentifier` as the identifier in the function registry.

- Add one new function `createOrReplaceTempFunction` to `FunctionRegistry`
```Scala
final def createOrReplaceTempFunction(name: String, builder: FunctionBuilder): Unit
```

### How was this patch tested?
Add extra test cases to verify the inclusive bug fixes.

Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #18142 from gatorsmile/fuctionRegistry.
2017-06-09 10:16:30 -07:00
Dongjoon Hyun 6e95897e88 [SPARK-20954][SQL] DESCRIBE [EXTENDED] result should be compatible with previous Spark
## What changes were proposed in this pull request?

After [SPARK-20067](https://issues.apache.org/jira/browse/SPARK-20067), `DESCRIBE` and `DESCRIBE EXTENDED` shows the following result. This is incompatible with Spark 2.1.1. This PR removes the column header line in case of those command.

**MASTER** and **BRANCH-2.2**
```scala
scala> sql("desc t").show(false)
+----------+---------+-------+
|col_name  |data_type|comment|
+----------+---------+-------+
|# col_name|data_type|comment|
|a         |int      |null   |
+----------+---------+-------+
```

**SPARK 2.1.1** and **this PR**
```scala
scala> sql("desc t").show(false)
+--------+---------+-------+
|col_name|data_type|comment|
+--------+---------+-------+
|a       |int      |null   |
+--------+---------+-------+
```

## How was this patch tested?

Pass the Jenkins with the updated test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18203 from dongjoon-hyun/SPARK-20954.
2017-06-08 16:46:56 -07:00
Xiao Li 1a527bde49 [SPARK-20976][SQL] Unify Error Messages for FAILFAST mode
### What changes were proposed in this pull request?
Before 2.2, we indicate the job was terminated because of `FAILFAST` mode.
```
Malformed line in FAILFAST mode: {"a":{, b:3}
```
If possible, we should keep it. This PR is to unify the error messages.

### How was this patch tested?
Modified the existing messages.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18196 from gatorsmile/messFailFast.
2017-06-08 12:10:31 -07:00
Sean Owen 847efe1265 [SPARK-20914][DOCS] Javadoc contains code that is invalid
## What changes were proposed in this pull request?

Fix Java, Scala Dataset examples in scaladoc, which didn't compile.

## How was this patch tested?

Existing compilation/test

Author: Sean Owen <sowen@cloudera.com>

Closes #18215 from srowen/SPARK-20914.
2017-06-08 10:56:23 +01:00
guoxiaolong 0ca69c4ccf [SPARK-20966][WEB-UI][SQL] Table data is not sorted by startTime time desc, time is not formatted and redundant code in JDBC/ODBC Server page.
## What changes were proposed in this pull request?

1. Question 1 : Table data is not sorted by startTime time desc in JDBC/ODBC Server page.

fix before :
![2](https://cloud.githubusercontent.com/assets/26266482/26718483/bf4a0fa8-47b3-11e7-9a27-dc6a67165b16.png)

fix after :
![21](https://cloud.githubusercontent.com/assets/26266482/26718544/eb7376c8-47b3-11e7-9117-1bc68dfec92c.png)

2. Question 2 :  time is not formatted in JDBC/ODBC Server page.

fix before :
![1](https://cloud.githubusercontent.com/assets/26266482/26718573/0497d86a-47b4-11e7-945b-582aaa103949.png)

fix after :
![11](https://cloud.githubusercontent.com/assets/26266482/26718602/21371ad0-47b4-11e7-9587-c5114d10ab2c.png)

3. Question 3 :  Redundant code in the ThriftServerSessionPage.scala.
The function of 'generateSessionStatsTable'  has not been used

## How was this patch tested?

manual tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>
Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>

Closes #18186 from guoxiaolongzte/SPARK-20966.
2017-06-07 10:18:40 +01:00
Bogdan Raducanu cb83ca1433 [SPARK-20854][TESTS] Removing duplicate test case
## What changes were proposed in this pull request?

Removed a duplicate case in "SPARK-20854: select hint syntax with expressions"

## How was this patch tested?
Existing tests.

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #18217 from bogdanrdc/SPARK-20854-2.
2017-06-06 22:51:10 -07:00
Wenchen Fan c92949ac23 [SPARK-20972][SQL] rename HintInfo.isBroadcastable to broadcast
## What changes were proposed in this pull request?

`HintInfo.isBroadcastable` is actually not an accurate name, it's used to force the planner to broadcast a plan no matter what the data size is, via the hint mechanism. I think `forceBroadcast` is a better name.

And `isBroadcastable` only have 2 possible values: `Some(true)` and `None`, so we can just use boolean type for it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18189 from cloud-fan/stats.
2017-06-06 22:50:06 -07:00
Reza Safi b61a401da8 [SPARK-20926][SQL] Removing exposures to guava library caused by directly accessing SessionCatalog's tableRelationCache
There could be test failures because DataStorageStrategy, HiveMetastoreCatalog and also HiveSchemaInferenceSuite were exposed to guava library by directly accessing SessionCatalog's tableRelationCacheg. These failures occur when guava shading is in place.

## What changes were proposed in this pull request?
This change removes those guava exposures by introducing new methods in SessionCatalog and also changing DataStorageStrategy, HiveMetastoreCatalog and HiveSchemaInferenceSuite so that they use those proxy methods.

## How was this patch tested?

Unit tests passed after applying these changes.

Author: Reza Safi <rezasafi@cloudera.com>

Closes #18148 from rezasafi/branch-2.2.

(cherry picked from commit 1388fdd707)
2017-06-06 09:54:13 -07:00
Feng Liu 88a23d3de0 [SPARK-20991][SQL] BROADCAST_TIMEOUT conf should be a TimeoutConf
## What changes were proposed in this pull request?

The construction of BROADCAST_TIMEOUT conf should take the TimeUnit argument as a TimeoutConf.

Author: Feng Liu <fengliu@databricks.com>

Closes #18208 from liufengdb/fix_timeout.
2017-06-05 17:48:28 -07:00
Shixiong Zhu bc537e40ad [SPARK-20957][SS][TESTS] Fix o.a.s.sql.streaming.StreamingQueryManagerSuite listing
## What changes were proposed in this pull request?

When stopping StreamingQuery, StreamExecution will set `streamDeathCause` then notify StreamingQueryManager to remove this query. So it's possible that when `q2.exception.isDefined` returns `true`, StreamingQueryManager's active list still has `q2`.

This PR just puts the checks into `eventually` to fix the flaky test.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18180 from zsxwing/SPARK-20957.
2017-06-05 14:34:10 -07:00
Wenchen Fan dec9aa3b37 [SPARK-20961][SQL] generalize the dictionary in ColumnVector
## What changes were proposed in this pull request?

As the first step of https://issues.apache.org/jira/browse/SPARK-20960 , to make `ColumnVector` public, this PR generalize `ColumnVector.dictionary` to not couple with parquet.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18183 from cloud-fan/dictionary.
2017-06-04 13:43:51 -07:00
Wieland Hoffmann c70c38eb93 [DOCS] Fix a typo in Encoder.clsTag
## What changes were proposed in this pull request?

Fixes a typo: `and` -> `an`

## How was this patch tested?

Not at all.

Author: Wieland Hoffmann <mineo@users.noreply.github.com>

Closes #17759 from mineo/patch-1.
2017-06-03 10:12:37 +01:00
Ruben Berenguel Montoro 6cbc61d107 [SPARK-19732][SQL][PYSPARK] Add fill functions for nulls in bool fields of datasets
## What changes were proposed in this pull request?

Allow fill/replace of NAs with booleans, both in Python and Scala

## How was this patch tested?

Unit tests, doctests

This PR is original work from me and I license this work to the Spark project

Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>
Author: Ruben Berenguel <ruben@mostlymaths.net>

Closes #18164 from rberenguel/SPARK-19732-fillna-bools.
2017-06-03 14:56:42 +09:00
Zhenhua Wang 6de41e951f [SPARK-17078][SQL][FOLLOWUP] Simplify explain cost command
## What changes were proposed in this pull request?

Usually when using explain cost command, users want to see the stats of plan. Since stats is only showed in optimized plan, it is more direct and convenient to include only optimized plan and physical plan in the output.

## How was this patch tested?

Enhanced existing test.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #18190 from wzhfy/simplifyExplainCost.
2017-06-02 17:36:00 -07:00
Yin Huai 0eb1fc6cd5 Revert "[SPARK-20946][SQL] simplify the config setting logic in SparkSession.getOrCreate"
This reverts commit e11d90bf8d.
2017-06-02 15:36:21 -07:00
Xiao Li 2a780ac7fe [MINOR][SQL] Update the description of spark.sql.files.ignoreCorruptFiles and spark.sql.columnNameOfCorruptRecord
### What changes were proposed in this pull request?
1. The description of `spark.sql.files.ignoreCorruptFiles` is not accurate. When the file does not exist, we will issue the error message.
```
org.apache.spark.sql.AnalysisException: Path does not exist: file:/nonexist/path;
```

2. `spark.sql.columnNameOfCorruptRecord` also affects the CSV format. The current description only mentions JSON format.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18184 from gatorsmile/updateMessage.
2017-06-02 12:58:29 -07:00
Wenchen Fan e11d90bf8d [SPARK-20946][SQL] simplify the config setting logic in SparkSession.getOrCreate
## What changes were proposed in this pull request?

The current conf setting logic is a little complex and has duplication, this PR simplifies it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18172 from cloud-fan/session.
2017-06-02 10:05:05 -07:00
Wenchen Fan d1b80ab922 [SPARK-20967][SQL] SharedState.externalCatalog is not really lazy
## What changes were proposed in this pull request?

`SharedState.externalCatalog` is marked as a `lazy val` but actually it's not lazy. We access `externalCatalog` while initializing `SharedState` and thus eliminate the effort of `lazy val`. When creating `ExternalCatalog` we will try to connect to the metastore and may throw an error, so it makes sense to make it a `lazy val` in `SharedState`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18187 from cloud-fan/minor.
2017-06-02 09:58:01 -07:00
Bogdan Raducanu 2134196a9c [SPARK-20854][SQL] Extend hint syntax to support expressions
## What changes were proposed in this pull request?

SQL hint syntax:
* support expressions such as strings, numbers, etc. instead of only identifiers as it is currently.
* support multiple hints, which was missing compared to the DataFrame syntax.

DataFrame API:
* support any parameters in DataFrame.hint instead of just strings

## How was this patch tested?
Existing tests. New tests in PlanParserSuite. New suite DataFrameHintSuite.

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #18086 from bogdanrdc/SPARK-20854.
2017-06-01 15:50:40 -07:00
Xiao Li f7cf2096fd [SPARK-20941][SQL] Fix SubqueryExec Reuse
### What changes were proposed in this pull request?
Before this PR, Subquery reuse does not work. Below are three issues:
- Subquery reuse does not work.
- It is sharing the same `SQLConf` (`spark.sql.exchange.reuse`) with the one for Exchange Reuse.
- No test case covers the rule Subquery reuse.

This PR is to fix the above three issues.
- Ignored the physical operator `SubqueryExec` when comparing two plans.
- Added a dedicated conf `spark.sql.subqueries.reuse` for controlling Subquery Reuse
- Added a test case for verifying the behavior

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18169 from gatorsmile/subqueryReuse.
2017-06-01 09:52:18 -07:00
Yuming Wang 6d05c1c1da [SPARK-20910][SQL] Add build-in SQL function - UUID
## What changes were proposed in this pull request?

Add build-int SQL function - UUID.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18136 from wangyum/SPARK-20910.
2017-06-01 16:15:24 +09:00
Yuming Wang c8045f8b48 [MINOR][SQL] Fix a few function description error.
## What changes were proposed in this pull request?

Fix a few function description error.

## How was this patch tested?

manual tests

![descissues](https://cloud.githubusercontent.com/assets/5399861/26619392/d547736c-4610-11e7-85d7-aeeb09c02cc8.gif)

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18157 from wangyum/DescIssues.
2017-05-31 23:17:15 -07:00
Shixiong Zhu 2bc3272880 [SPARK-20894][SS] Resolve the checkpoint location in driver and use the resolved path in state store
## What changes were proposed in this pull request?

When the user runs a Structured Streaming query in a cluster, if the driver uses the local file system, StateStore running in executors will throw a file-not-found exception. However, the current error is not obvious.

This PR makes StreamExecution resolve the path in driver and uses the full path including the scheme part (such as `hdfs:/`, `file:/`) in StateStore.

Then if the above error happens, StateStore will throw an error with this full path which starts with `file:/`, and it makes this error obvious: the checkpoint location is on the local file system.

One potential minor issue is that the user cannot use different default file system settings in driver and executors (e.g., use a public HDFS address in driver and a private HDFS address in executors) after this change. However, since the batch query also has this issue (See 4bb6a53ebd/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L402)), it doesn't make things worse.

## How was this patch tested?

The new added test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18149 from zsxwing/SPARK-20894.
2017-05-31 17:24:37 -07:00
gatorsmile de934e6718 [SPARK-19236][SQL][FOLLOW-UP] Added createOrReplaceGlobalTempView method
### What changes were proposed in this pull request?
This PR does the following tasks:
- Added  since
- Added the Python API
- Added test cases

### How was this patch tested?
Added test cases to both Scala and Python

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18147 from gatorsmile/createOrReplaceGlobalTempView.
2017-05-31 11:38:43 -07:00
Liu Shaohui d0f36bcb10 [SPARK-20633][SQL] FileFormatWriter should not wrap FetchFailedException
## What changes were proposed in this pull request?

Explicitly handle the FetchFailedException in FileFormatWriter, so it does not get wrapped.

Note that this is no longer strictly necessary after SPARK-19276, but it improves error messages and also will help avoid others stumbling across this in the future.

## How was this patch tested?

Existing unit tests.

Closes https://github.com/apache/spark/pull/17893

Author: Liu Shaohui <liushaohui@xiaomi.com>

Closes #18145 from squito/SPARK-20633.
2017-05-31 10:53:31 -05:00
Jacek Laskowski beed5e20af [DOCS][MINOR] Scaladoc fixes (aka typo hunting)
## What changes were proposed in this pull request?

Minor changes to scaladoc

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18074 from jaceklaskowski/scaladoc-fixes.
2017-05-31 11:24:37 +01:00
Wenchen Fan 1f5dddffa3 Revert "[SPARK-20392][SQL] Set barrier to prevent re-entering a tree"
This reverts commit 8ce0d8ffb6.
2017-05-30 21:14:55 -07:00
Wenchen Fan 10e526e7e6 [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL UI tab
## What changes were proposed in this pull request?

Currently the `DataFrameWriter` operations have several problems:

1. non-file-format data source writing action doesn't show up in the SQL tab in Spark UI
2. file-format data source writing action shows a scan node in the SQL tab, without saying anything about writing. (streaming also have this issue, but not fixed in this PR)
3. Spark SQL CLI actions don't show up in the SQL tab.

This PR fixes all of them, by refactoring the `ExecuteCommandExec` to make it have children.

 close https://github.com/apache/spark/pull/17540

## How was this patch tested?

existing tests.

Also test the UI manually. For a simple command: `Seq(1 -> "a").toDF("i", "j").write.parquet("/tmp/qwe")`

before this PR:
<img width="266" alt="qq20170523-035840 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326050/24e18ba2-3f6c-11e7-8817-6dd275bf6ac5.png">
after this PR:
<img width="287" alt="qq20170523-035708 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326054/2ad7f460-3f6c-11e7-8053-d68325beb28f.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18064 from cloud-fan/execution.
2017-05-30 20:12:32 -07:00
Tathagata Das fa757ee1d4 [SPARK-20883][SPARK-20376][SS] Refactored StateStore APIs and added conf to choose implementation
## What changes were proposed in this pull request?

A bunch of changes to the StateStore APIs and implementation.
Current state store API has a bunch of problems that causes too many transient objects causing memory pressure.

- `StateStore.get(): Option` forces creation of Some/None objects for every get. Changed this to return the row or null.
- `StateStore.iterator(): (UnsafeRow, UnsafeRow)` forces creation of new tuple for each record returned. Changed this to return a UnsafeRowTuple which can be reused across records.
- `StateStore.updates()` requires the implementation to keep track of updates, while this is used minimally (only by Append mode in streaming aggregations). Removed updates() and updated StateStoreSaveExec accordingly.
- `StateStore.filter(condition)` and `StateStore.remove(condition)` has been merge into a single API `getRange(start, end)` which allows a state store to do optimized range queries (i.e. avoid full scans). Stateful operators have been updated accordingly.
- Removed a lot of unnecessary row copies Each operator copied rows before calling StateStore.put() even if the implementation does not require it to be copied. It is left up to the implementation on whether to copy the row or not.

Additionally,
- Added a name to the StateStoreId so that each operator+partition can use multiple state stores (different names)
- Added a configuration that allows the user to specify which implementation to use.
- Added new metrics to understand the time taken to update keys, remove keys and commit all changes to the state store. These metrics will be visible on the plan diagram in the SQL tab of the UI.
- Refactored unit tests such that they can be reused to test any implementation of StateStore.

## How was this patch tested?
Old and new unit tests

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

Closes #18107 from tdas/SPARK-20376.
2017-05-30 15:33:06 -07:00
Xiao Li 4bb6a53ebd [SPARK-20924][SQL] Unable to call the function registered in the not-current database
### What changes were proposed in this pull request?
We are unable to call the function registered in the not-current database.
```Scala
sql("CREATE DATABASE dAtABaSe1")
sql(s"CREATE FUNCTION dAtABaSe1.test_avg AS '${classOf[GenericUDAFAverage].getName}'")
sql("SELECT dAtABaSe1.test_avg(1)")
```
The above code returns an error:
```
Undefined function: 'dAtABaSe1.test_avg'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7
```

This PR is to fix the above issue.
### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18146 from gatorsmile/qualifiedFunction.
2017-05-30 14:06:19 -07:00
Josh Rosen 798a04fd76 HOTFIX: fix Scalastyle break introduced in 4d57981cfb 2017-05-30 12:22:23 -07:00
Arman 4d57981cfb [SPARK-19236][CORE] Added createOrReplaceGlobalTempView method
## What changes were proposed in this pull request?

Added the createOrReplaceGlobalTempView method for dataset

Author: Arman <arman.yazdani.10@gmail.com>

Closes #16598 from arman1371/patch-1.
2017-05-30 11:09:21 -07:00
Liang-Chi Hsieh 35b644bd03 [SPARK-20916][SQL] Improve error message for unaliased subqueries in FROM clause
## What changes were proposed in this pull request?

We changed the parser to reject unaliased subqueries in the FROM clause in SPARK-20690. However, the error message that we now give isn't very helpful:

    scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
    org.apache.spark.sql.catalyst.parser.ParseException:
    mismatched input 'FROM' expecting {<EOF>, 'WHERE', 'GROUP', 'ORDER', 'HAVING', 'LIMIT', 'LATERAL', 'WINDOW', 'UNION', 'EXCEPT', 'MINUS', 'INTERSECT', 'SORT', 'CLUSTER', 'DISTRIBUTE'}(line 1, pos 9)

We should modify the parser to throw a more clear error for such queries:

    scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
    org.apache.spark.sql.catalyst.parser.ParseException:
    The unaliased subqueries in the FROM clause are not supported.(line 1, pos 14)

## How was this patch tested?

Modified existing tests to reflect this change.

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

Closes #18141 from viirya/SPARK-20916.
2017-05-30 06:28:43 -07:00
Yuming Wang 80fb24b85d [MINOR] Fix some indent issues.
## What changes were proposed in this pull request?

Fix some indent issues.

## How was this patch tested?

existing tests.

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18133 from wangyum/IndentIssues.
2017-05-30 12:15:54 +01:00
Yuming Wang d797ed0ef1 [SPARK-20909][SQL] Add build-int SQL function - DAYOFWEEK
## What changes were proposed in this pull request?

Add build-int SQL function - DAYOFWEEK

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18134 from wangyum/SPARK-20909.
2017-05-30 15:40:50 +09:00
Yuming Wang 1c7db00c74 [SPARK-8184][SQL] Add additional function description for weekofyear
## What changes were proposed in this pull request?

Add additional function description for weekofyear.

## How was this patch tested?

 manual tests

![weekofyear](https://cloud.githubusercontent.com/assets/5399861/26525752/08a1c278-4394-11e7-8988-7cbf82c3a999.gif)

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18132 from wangyum/SPARK-8184.
2017-05-29 16:10:22 -07:00
Kazuaki Ishizaki c9749068ec [SPARK-20907][TEST] Use testQuietly for test suites that generate long log output
## What changes were proposed in this pull request?

Supress console output by using `testQuietly` in test suites

## How was this patch tested?

Tested by `"SPARK-19372: Filter can be executed w/o generated code due to JVM code size limit"` in `DataFrameSuite`

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

Closes #18135 from kiszk/SPARK-20907.
2017-05-29 12:17:14 -07:00
Kazuaki Ishizaki ef9fd920c3 [SPARK-20750][SQL] Built-in SQL Function Support - REPLACE
## What changes were proposed in this pull request?

This PR adds built-in SQL function `(REPLACE(<string_expression>, <search_string> [, <replacement_string>])`

`REPLACE()` return that string that is replaced all occurrences with given string.

## How was this patch tested?

added new test suites

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

Closes #18047 from kiszk/SPARK-20750.
2017-05-29 11:47:31 -07:00
Tejas Patil f9b59abeae [SPARK-20758][SQL] Add Constant propagation optimization
## What changes were proposed in this pull request?

See class doc of `ConstantPropagation` for the approach used.

## How was this patch tested?

- Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17993 from tejasapatil/SPARK-20758_const_propagation.
2017-05-29 12:21:34 +02:00
Zhenhua Wang 9d0db5a7f8 [SPARK-20881][SQL] Clearly document the mechanism to choose between two sources of statistics
## What changes were proposed in this pull request?

Now, we have two sources of statistics, i.e. Spark's stats and Hive's stats. Spark's stats is generated by running "analyze" command in Spark. Once it's available, we respect this stats over Hive's.

This pr is to clearly document in related code the mechanism to choose between these two sources of stats.

## How was this patch tested?

Not related.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #18105 from wzhfy/cboSwitchStats.
2017-05-28 13:32:45 -07:00
Takeshi Yamamuro 24d34281d7 [SPARK-20841][SQL] Support table column aliases in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support table column aliases in FROM clause.

## How was this patch tested?
Added tests in `PlanParserSuite`,  `SQLQueryTestSuite`, and `PlanParserSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18079 from maropu/SPARK-20841.
2017-05-28 13:23:18 -07:00
Xiao Li 06c155c90d [SPARK-20908][SQL] Cache Manager: Hint should be ignored in plan matching
### What changes were proposed in this pull request?

In Cache manager, the plan matching should ignore Hint.
```Scala
      val df1 = spark.range(10).join(broadcast(spark.range(10)))
      df1.cache()
      spark.range(10).join(spark.range(10)).explain()
```
The output plan of the above query shows that the second query is  not using the cached data of the first query.
```
BroadcastNestedLoopJoin BuildRight, Inner
:- *Range (0, 10, step=1, splits=2)
+- BroadcastExchange IdentityBroadcastMode
   +- *Range (0, 10, step=1, splits=2)
```

After the fix, the plan becomes
```
InMemoryTableScan [id#20L, id#23L]
   +- InMemoryRelation [id#20L, id#23L], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
         +- BroadcastNestedLoopJoin BuildRight, Inner
            :- *Range (0, 10, step=1, splits=2)
            +- BroadcastExchange IdentityBroadcastMode
               +- *Range (0, 10, step=1, splits=2)
```

### How was this patch tested?
Added a test.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18131 from gatorsmile/HintCache.
2017-05-27 21:32:18 -07:00
liuxian 3969a8078e [SPARK-20876][SQL] If the input parameter is float type for ceil or floor,the result is not we expected
## What changes were proposed in this pull request?

spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12345
For this case, the result we expected is `12346`
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12345
For this case, the result we expected is `-12346`

Because in `Ceil` or `Floor`, `inputTypes` has no FloatType, so it is converted to LongType.
## How was this patch tested?

After the modification:
spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12346
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12346

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18103 from 10110346/wip-lx-0525-1.
2017-05-27 16:23:45 -07:00
Wenchen Fan 08ede46b89 [SPARK-20897][SQL] cached self-join should not fail
## What changes were proposed in this pull request?

The failed test case is, we have a `SortMergeJoinExec` for a self-join, which means we have a `ReusedExchange` node in the query plan. It works fine without caching, but throws an exception in `SortMergeJoinExec.outputPartitioning` if we cache it.

The root cause is, `ReusedExchange` doesn't propagate the output partitioning from its child, so in `SortMergeJoinExec.outputPartitioning` we create `PartitioningCollection` with a hash partitioning and an unknown partitioning, and fail.

This bug is mostly fine, because inserting the `ReusedExchange` is the last step to prepare the physical plan, we won't call `SortMergeJoinExec.outputPartitioning` anymore after this.

However, if the dataframe is cached, the physical plan of it becomes `InMemoryTableScanExec`, which contains another physical plan representing the cached query, and it has gone through the entire planning phase and may have `ReusedExchange`. Then the planner call `InMemoryTableScanExec.outputPartitioning`, which then calls `SortMergeJoinExec.outputPartitioning` and trigger this bug.

## How was this patch tested?

a new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18121 from cloud-fan/bug.
2017-05-27 16:16:51 -07:00
Yuming Wang a0f8a072e3 [SPARK-20748][SQL] Add built-in SQL function CH[A]R.
## What changes were proposed in this pull request?
Add built-in SQL function `CH[A]R`:
For `CHR(bigint|double n)`, returns the ASCII character having the binary equivalent to `n`. If n is larger than 256 the result is equivalent to CHR(n % 256)

## How was this patch tested?
unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18019 from wangyum/SPARK-20748.
2017-05-26 20:59:14 -07:00
setjet c491e2ed90 [SPARK-20873][SQL] Improve the error message for unsupported Column Type
## What changes were proposed in this pull request?
Upon encountering an invalid columntype, the column type object is printed, rather than the type.
This  change improves this by outputting its name.

## How was this patch tested?
Added a simple  unit test to verify the contents of the raised exception

Author: setjet <rubenljanssen@gmail.com>

Closes #18097 from setjet/spark-20873.
2017-05-26 15:07:28 -07:00
Michael Armbrust d935e0a9d9 [SPARK-20844] Remove experimental from Structured Streaming APIs
Now that Structured Streaming has been out for several Spark release and has large production use cases, the `Experimental` label is no longer appropriate.  I've left `InterfaceStability.Evolving` however, as I think we may make a few changes to the pluggable Source & Sink API in Spark 2.3.

Author: Michael Armbrust <michael@databricks.com>

Closes #18065 from marmbrus/streamingGA.
2017-05-26 13:33:23 -07:00
Liang-Chi Hsieh 8ce0d8ffb6 [SPARK-20392][SQL] Set barrier to prevent re-entering a tree
## What changes were proposed in this pull request?

It is reported that there is performance downgrade when applying ML pipeline for dataset with many columns but few rows.

A big part of the performance downgrade comes from some operations (e.g., `select`) on DataFrame/Dataset which re-create new DataFrame/Dataset with a new `LogicalPlan`. The cost can be ignored in the usage of SQL, normally.

However, it's not rare to chain dozens of pipeline stages in ML. When the query plan grows incrementally during running those stages, the total cost spent on re-creation of DataFrame grows too. In particular, the `Analyzer` will go through the big query plan even most part of it is analyzed.

By eliminating part of the cost, the time to run the example code locally is reduced from about 1min to about 30 secs.

In particular, the time applying the pipeline locally is mostly spent on calling transform of the 137 `Bucketizer`s. Before the change, each call of `Bucketizer`'s transform can cost about 0.4 sec. So the total time spent on all `Bucketizer`s' transform is about 50 secs. After the change, each call only costs about 0.1 sec.

<del>We also make `boundEnc` as lazy variable to reduce unnecessary running time.</del>

### Performance improvement

The codes and datasets provided by Barry Becker to re-produce this issue and benchmark can be found on the JIRA.

Before this patch: about 1 min
After this patch: about 20 secs

## How was this patch tested?

Existing tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #17770 from viirya/SPARK-20392.
2017-05-26 13:45:55 +08:00
setjet 2dbe0c5288 [SPARK-20775][SQL] Added scala support from_json
## What changes were proposed in this pull request?

from_json function required to take in a java.util.Hashmap. For other functions, a java wrapper is provided which casts a java hashmap to a scala map. Only a java function is provided in this case, forcing scala users to pass in a java.util.Hashmap.

Added the missing wrapper.

## How was this patch tested?
Added a unit test for passing in a scala map

Author: setjet <rubenljanssen@gmail.com>

Closes #18094 from setjet/spark-20775.
2017-05-26 10:21:39 +08:00
hyukjinkwon e9f983df27 [SPARK-19707][SPARK-18922][TESTS][SQL][CORE] Fix test failures/the invalid path check for sc.addJar on Windows
## What changes were proposed in this pull request?

This PR proposes two things:

- A follow up for SPARK-19707 (Improving the invalid path check for sc.addJar on Windows as well).

```
org.apache.spark.SparkContextSuite:
 - add jar with invalid path *** FAILED *** (32 milliseconds)
   2 was not equal to 1 (SparkContextSuite.scala:309)
   ...
```

- Fix path vs URI related test failures on Windows.

```
org.apache.spark.storage.LocalDirsSuite:
 - SPARK_LOCAL_DIRS override also affects driver *** FAILED *** (0 milliseconds)
   new java.io.File("/NONEXISTENT_PATH").exists() was true (LocalDirsSuite.scala:50)
   ...

 - Utils.getLocalDir() throws an exception if any temporary directory cannot be retrieved *** FAILED *** (15 milliseconds)
   Expected exception java.io.IOException to be thrown, but no exception was thrown. (LocalDirsSuite.scala:64)
   ...
```

```
org.apache.spark.sql.hive.HiveSchemaInferenceSuite:
 - orc: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-dae61ab3-a851-4dd3-bf4e-be97c501f254
   ...

 - parquet: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fa3aff89-a66e-4376-9a37-2a9b87596939
   ...

 - orc: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (141 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fb464e59-b049-481b-9c75-f53295c9fc2c
   ...

 - parquet: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (125 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-9487568e-80a4-42b3-b0a5-d95314c4ccbc
   ...

 - orc: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (156 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-0d2dfa45-1b0f-4958-a8be-1074ed0135a
   ...

 - parquet: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (547 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-6d95d64e-613e-4a59-a0f6-d198c5aa51ee
   ...
```

```
org.apache.spark.sql.execution.command.DDLSuite:
 - create temporary view using *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-3881d9ca-561b-488d-90b9-97587472b853	mp;
   ...

 - insert data to a data source table which has a non-existing location should succeed *** FAILED *** (109 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-4cad3d19-6085-4b75-b407-fe5e9d21df54 did not equal file:///C:/projects/spark/target/tmp/spark-4cad3d19-6085-4b75-b407-fe5e9d21df54 (DDLSuite.scala:1869)
   ...

 - insert into a data source table with a non-existing partition location should succeed *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d did not equal file:///C:/projects/spark/target/tmp/spark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d (DDLSuite.scala:1910)
   ...

 - read data from a data source table which has a non-existing location should succeed *** FAILED *** (93 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-f8c281e2-08c2-4f73-abbf-f3865b702c34 did not equal file:///C:/projects/spark/target/tmp/spark-f8c281e2-08c2-4f73-abbf-f3865b702c34 (DDLSuite.scala:1937)
   ...

 - read data from a data source table with non-existing partition location should succeed *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - create datasource table with a non-existing location *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-387316ae-070c-4e78-9b78-19ebf7b29ec8 did not equal file:///C:/projects/spark/target/tmp/spark-387316ae-070c-4e78-9b78-19ebf7b29ec8 (DDLSuite.scala:1982)
   ...

 - CTAS for external data source table with a non-existing location *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - CTAS for external data source table with a existed location *** FAILED *** (15 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a b *** FAILED *** (125 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a:b *** FAILED *** (143 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a%b *** FAILED *** (109 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a,b *** FAILED *** (109 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - location uri contains a b for datasource table *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-5739cda9-b702-4e14-932c-42e8c4174480a%20b did not equal file:///C:/projects/spark/target/tmp/spark-5739cda9-b702-4e14-932c-42e8c4174480/a%20b (DDLSuite.scala:2084)
   ...

 - location uri contains a:b for datasource table *** FAILED *** (78 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-9bdd227c-840f-4f08-b7c5-4036638f098da:b did not equal file:///C:/projects/spark/target/tmp/spark-9bdd227c-840f-4f08-b7c5-4036638f098d/a:b (DDLSuite.scala:2084)
   ...

 - location uri contains a%b for datasource table *** FAILED *** (78 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-62bb5f1d-fa20-460a-b534-cb2e172a3640a%25b did not equal file:///C:/projects/spark/target/tmp/spark-62bb5f1d-fa20-460a-b534-cb2e172a3640/a%25b (DDLSuite.scala:2084)
   ...

 - location uri contains a b for database *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - location uri contains a:b for database *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - location uri contains a%b for database *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...
```

```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
 - create hive table with a non-existing location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - CTAS for external hive table with a non-existing location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - CTAS for external hive table with a existed location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of parquet table containing a b *** FAILED *** (156 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a:b *** FAILED *** (94 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a%b *** FAILED *** (125 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a,b *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of hive table containing a b *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a:b *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a%b *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a,b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a:b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a%b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...
```

```
org.apache.spark.sql.sources.PathOptionSuite:
 - path option also exist for write path *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc did not equal file:///C:/projects/spark/target/tmp/spark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc (PathOptionSuite.scala:98)
   ...
```

```
org.apache.spark.sql.CachedTableSuite:
 - SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer to this table *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...
```

```
org.apache.spark.sql.execution.DataSourceScanExecRedactionSuite:
 - treeString is redacted *** FAILED *** (250 milliseconds)
   "file:/C:/projects/spark/target/tmp/spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" did not contain "C:\projects\spark\target\tmp\spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" (DataSourceScanExecRedactionSuite.scala:46)
   ...
```

## How was this patch tested?

Tested via AppVeyor for each and checked it passed once each. These should be retested via AppVeyor in this PR.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17987 from HyukjinKwon/windows-20170515.
2017-05-25 17:10:30 +01:00
Liang-Chi Hsieh 6b68d61cf3 [SPARK-20848][SQL][FOLLOW-UP] Shutdown the pool after reading parquet files
## What changes were proposed in this pull request?

This is a follow-up to #18073. Taking a safer approach to shutdown the pool to prevent possible issue. Also using `ThreadUtils.newForkJoinPool` instead to set a better thread name.

## How was this patch tested?

Manually test.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18100 from viirya/SPARK-20848-followup.
2017-05-25 09:55:45 +08:00
liuxian 197f9018a4 [SPARK-20403][SQL] Modify the instructions of some functions
## What changes were proposed in this pull request?
1.    add  instructions of  'cast'  function When using 'show functions'  and 'desc function cast'
       command in spark-sql
2.    Modify the  instructions of functions,such as
     boolean,tinyint,smallint,int,bigint,float,double,decimal,date,timestamp,binary,string

## How was this patch tested?
Before modification:
spark-sql>desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr AS type) - Casts the value `expr` to the target data type `type`.

After modification:
spark-sql> desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`.

spark-sql> desc function cast
Function: cast
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: cast(expr AS type) - Casts the value `expr` to the target data type `type`.

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17698 from 10110346/wip_lx_0418.
2017-05-24 17:32:02 -07:00
Jacek Laskowski 5f8ff2fc9a [SPARK-16202][SQL][DOC] Follow-up to Correct The Description of CreatableRelationProvider's createRelation
## What changes were proposed in this pull request?

Follow-up to SPARK-16202:

1. Remove the duplication of the meaning of `SaveMode` (as one was in fact missing that had proven that the duplication may be incomplete in the future again)

2. Use standard scaladoc tags

/cc gatorsmile rxin yhuai (as they were involved previously)

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18026 from jaceklaskowski/CreatableRelationProvider-SPARK-16202.
2017-05-24 17:24:23 -07:00
Kris Mok c0b3e45e3b [SPARK-20872][SQL] ShuffleExchange.nodeName should handle null coordinator
## What changes were proposed in this pull request?

A one-liner change in `ShuffleExchange.nodeName` to cover the case when `coordinator` is `null`, so that the match expression is exhaustive.

Please refer to [SPARK-20872](https://issues.apache.org/jira/browse/SPARK-20872) for a description of the symptoms.
TL;DR is that inspecting a `ShuffleExchange` (directly or transitively) on the Executor side can hit a case where the `coordinator` field of a `ShuffleExchange` is null, and thus will trigger a `MatchError` in `ShuffleExchange.nodeName()`'s inexhaustive match expression.

Also changed two other match conditions in `ShuffleExchange` on the `coordinator` field to be consistent.

## How was this patch tested?

Manually tested this change with a case where the `coordinator` is null to make sure `ShuffleExchange.nodeName` doesn't throw a `MatchError` any more.

Author: Kris Mok <kris.mok@databricks.com>

Closes #18095 from rednaxelafx/shuffleexchange-nodename.
2017-05-24 17:19:35 -07:00
Reynold Xin a64746677b [SPARK-20867][SQL] Move hints from Statistics into HintInfo class
## What changes were proposed in this pull request?
This is a follow-up to SPARK-20857 to move the broadcast hint from Statistics into a new HintInfo class, so we can be more flexible in adding new hints in the future.

## How was this patch tested?
Updated test cases to reflect the change.

Author: Reynold Xin <rxin@databricks.com>

Closes #18087 from rxin/SPARK-20867.
2017-05-24 13:57:19 -07:00
Liang-Chi Hsieh f72ad303f0 [SPARK-20848][SQL] Shutdown the pool after reading parquet files
## What changes were proposed in this pull request?

From JIRA: On each call to spark.read.parquet, a new ForkJoinPool is created. One of the threads in the pool is kept in the WAITING state, and never stopped, which leads to unbounded growth in number of threads.

We should shutdown the pool after reading parquet files.

## How was this patch tested?

Added a test to ParquetFileFormatSuite.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18073 from viirya/SPARK-20848.
2017-05-25 00:35:40 +08:00
Kirby Linvill 4816c2ef5e [SPARK-15648][SQL] Add teradataDialect for JDBC connection to Teradata
The contribution is my original work and I license the work to the project under the project’s open source license.

Note: the Teradata JDBC connector limits the row size to 64K. The default string datatype equivalent I used is a 255 character/byte length varchar. This effectively limits the max number of string columns to 250 when using the Teradata jdbc connector.

## What changes were proposed in this pull request?

Added a teradataDialect for JDBC connection to Teradata. The Teradata dialect uses VARCHAR(255) in place of TEXT for string datatypes, and CHAR(1) in place of BIT(1) for boolean datatypes.

## How was this patch tested?

I added two unit tests to double check that the types get set correctly for a teradata jdbc url. I also ran a couple manual tests to make sure the jdbc connector worked with teradata and to make sure that an error was thrown if a row could potentially exceed 64K (this error comes from the teradata jdbc connector, not from the spark code). I did not check how string columns longer than 255 characters are handled.

Author: Kirby Linvill <kirby.linvill@teradata.com>
Author: klinvill <kjlinvill@gmail.com>

Closes #16746 from klinvill/master.
2017-05-23 12:00:58 -07:00
Reynold Xin 0d589ba00b [SPARK-20857][SQL] Generic resolved hint node
## What changes were proposed in this pull request?
This patch renames BroadcastHint to ResolvedHint (and Hint to UnresolvedHint) so the hint framework is more generic and would allow us to introduce other hint types in the future without introducing new hint nodes.

## How was this patch tested?
Updated test cases.

Author: Reynold Xin <rxin@databricks.com>

Closes #18072 from rxin/SPARK-20857.
2017-05-23 18:44:49 +02:00
Liang-Chi Hsieh 442287ae29 [SPARK-20399][SQL][FOLLOW-UP] Add a config to fallback string literal parsing consistent with old sql parser behavior
## What changes were proposed in this pull request?

As srowen pointed in 609ba5f2b9 (commitcomment-22221259), the previous tests are not proper.

This follow-up is going to fix the tests.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18048 from viirya/SPARK-20399-follow-up.
2017-05-23 16:09:38 +08:00
Xiao Li a2460be9c3 [SPARK-17410][SPARK-17284] Move Hive-generated Stats Info to HiveClientImpl
### What changes were proposed in this pull request?

After we adding a new field `stats` into `CatalogTable`, we should not expose Hive-specific Stats metadata to `MetastoreRelation`. It complicates all the related codes. It also introduces a bug in `SHOW CREATE TABLE`. The statistics-related table properties should be skipped by `SHOW CREATE TABLE`, since it could be incorrect in the newly created table. See the Hive JIRA: https://issues.apache.org/jira/browse/HIVE-13792

Also fix the issue to fill Hive-generated RowCounts to our stats.

This PR is to handle Hive-specific Stats metadata in `HiveClientImpl`.
### How was this patch tested?

Added a few test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #14971 from gatorsmile/showCreateTableNew.
2017-05-22 17:28:30 -07:00
Yuming Wang 9b09101938 [SPARK-20751][SQL][FOLLOWUP] Add cot test in MathExpressionsSuite
## What changes were proposed in this pull request?

Add cot test in MathExpressionsSuite as https://github.com/apache/spark/pull/17999#issuecomment-302832794.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18039 from wangyum/SPARK-20751-test.
2017-05-22 13:05:05 -07:00
gatorsmile f3ed62a381 [SPARK-20831][SQL] Fix INSERT OVERWRITE data source tables with IF NOT EXISTS
### What changes were proposed in this pull request?
Currently, we have a bug when we specify `IF NOT EXISTS` in `INSERT OVERWRITE` data source tables. For example, given a query:
```SQL
INSERT OVERWRITE TABLE $tableName partition (b=2, c=3) IF NOT EXISTS SELECT 9, 10
```
we will get the following error:
```
unresolved operator 'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true;;
'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true
+- Project [cast(9#423 as int) AS a#429, cast(10#424 as int) AS d#430]
   +- Project [9 AS 9#423, 10 AS 10#424]
      +- OneRowRelation$
```

This PR is to fix the issue to follow the behavior of Hive serde tables
> INSERT OVERWRITE will overwrite any existing data in the table or partition unless IF NOT EXISTS is provided for a partition

### How was this patch tested?
Modified an existing test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18050 from gatorsmile/insertPartitionIfNotExists.
2017-05-22 22:24:50 +08:00
Michal Senkyr a2b3b67624 [SPARK-19089][SQL] Add support for nested sequences
## What changes were proposed in this pull request?

Replaced specific sequence encoders with generic sequence encoder to enable nesting of sequences.

Does not add support for nested arrays as that cannot be solved in this way.

## How was this patch tested?

```bash
build/mvn -DskipTests clean package && dev/run-tests
```

Additionally in Spark shell:

```
scala> Seq(Seq(Seq(1))).toDS.collect()
res0: Array[Seq[Seq[Int]]] = Array(List(List(1)))
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #18011 from michalsenkyr/dataset-seq-nested.
2017-05-22 16:49:19 +08:00
Kazuaki Ishizaki 833c8d4152 [SPARK-20770][SQL] Improve ColumnStats
## What changes were proposed in this pull request?

This PR improves the implementation of `ColumnStats` by using the following appoaches.

1. Declare subclasses of `ColumnStats` as `final`
2. Remove unnecessary call of `row.isNullAt(ordinal)`
3. Remove the dependency on `GenericInternalRow`

For 1., this declaration encourages method inlining and other optimizations of JIT compiler
For 2., in `gatherStats()`, while previous code in subclasses of `ColumnStats` always calls `row.isNullAt()` twice, the PR just calls `row.isNullAt()` only once.
For 3., `collectedStatistics()` returns `Array[Any]` instead of `GenericInternalRow`. This removes the dependency of unnecessary package and reduces the number of allocations of `GenericInternalRow`.

In addition to that, in the future, `gatherValueStats()`, which is specialized for each data type, can be effectively called from the generated code without using generic data structure `InternalRow`.

## How was this patch tested?

Tested by existing test suite

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

Closes #18002 from kiszk/SPARK-20770.
2017-05-22 16:23:23 +08:00
caoxuewen 3c9eef35a8 [SPARK-20786][SQL] Improve ceil and floor handle the value which is not expected
## What changes were proposed in this pull request?

spark-sql>SELECT ceil(1234567890123456);
1234567890123456

spark-sql>SELECT ceil(12345678901234567);
12345678901234568

spark-sql>SELECT ceil(123456789012345678);
123456789012345680

when the length of the getText is greater than 16. long to double will be precision loss.

but mysql handle the value is ok.

mysql> SELECT ceil(1234567890123456);
+------------------------+
| ceil(1234567890123456) |
+------------------------+
|       1234567890123456 |
+------------------------+
1 row in set (0.00 sec)

mysql> SELECT ceil(12345678901234567);
+-------------------------+
| ceil(12345678901234567) |
+-------------------------+
|       12345678901234567 |
+-------------------------+
1 row in set (0.00 sec)

mysql> SELECT ceil(123456789012345678);
+--------------------------+
| ceil(123456789012345678) |
+--------------------------+
|       123456789012345678 |
+--------------------------+
1 row in set (0.00 sec)

## How was this patch tested?

Supplement the unit test.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #18016 from heary-cao/ceil_long.
2017-05-21 22:39:07 -07:00
Tathagata Das 9d6661c829 [SPARK-20792][SS] Support same timeout operations in mapGroupsWithState function in batch queries as in streaming queries
## What changes were proposed in this pull request?

Currently, in the batch queries, timeout is disabled (i.e. GroupStateTimeout.NoTimeout) which means any GroupState.setTimeout*** operation would throw UnsupportedOperationException. This makes it weird when converting a streaming query into a batch query by changing the input DF from streaming to a batch DF. If the timeout was enabled and used, then the batch query will start throwing UnsupportedOperationException.

This PR creates the dummy state in batch queries with the provided timeoutConf so that it behaves in the same way. The code has been refactored to make it obvious when the state is being created for a batch query or a streaming query.

## How was this patch tested?
Additional tests

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

Closes #18024 from tdas/SPARK-20792.
2017-05-21 13:07:25 -07:00
liuxian ea3b1e352a [SPARK-20763][SQL] The function of month and day return the value which is not we expected.
## What changes were proposed in this pull request?
spark-sql>select month("1582-09-28");
spark-sql>10
For this case, the expected result is 9, but it is 10.

spark-sql>select day("1582-04-18");
spark-sql>28
For this case, the expected result is 18, but it is 28.

when the date  before "1582-10-04", the function of `month` and `day` return the value which is not we expected.

## How was this patch tested?
unit tests

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17997 from 10110346/wip_lx_0516.
2017-05-19 10:25:21 -07:00
Yuming Wang bff021dfaf [SPARK-20751][SQL] Add built-in SQL Function - COT
## What changes were proposed in this pull request?

Add built-in SQL Function - COT.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17999 from wangyum/SPARK-20751.
2017-05-19 09:40:22 -07:00
tpoterba 3f2cd51ee0 [SPARK-20773][SQL] ParquetWriteSupport.writeFields is quadratic in number of fields
Fix quadratic List indexing in ParquetWriteSupport.

I noticed this function while profiling some code with today. It showed up as a significant factor in a table with twenty columns; with hundreds of columns, it could dominate any other function call.

## What changes were proposed in this pull request?

The writeFields method iterates from 0 until number of fields, indexing into rootFieldWriters for each element. rootFieldWriters is a List, so indexing is a linear operation. The complexity of the writeFields method is thus quadratic in the number of fields.

Solution: explicitly convert rootFieldWriters to Array (implicitly converted to WrappedArray) for constant-time indexing.

## How was this patch tested?

This is a one-line change for performance reasons.

Author: tpoterba <tpoterba@broadinstitute.org>
Author: Tim Poterba <tpoterba@gmail.com>

Closes #18005 from tpoterba/tpoterba-patch-1.
2017-05-19 14:17:12 +02:00
Ala Luszczak ce8edb8bf4 [SPARK-20798] GenerateUnsafeProjection should check if a value is null before calling the getter
## What changes were proposed in this pull request?

GenerateUnsafeProjection.writeStructToBuffer() did not honor the assumption that the caller must make sure that a value is not null before using the getter. This could lead to various errors. This change fixes that behavior.

Example of code generated before:
```scala
/* 059 */         final UTF8String fieldName = value.getUTF8String(0);
/* 060 */         if (value.isNullAt(0)) {
/* 061 */           rowWriter1.setNullAt(0);
/* 062 */         } else {
/* 063 */           rowWriter1.write(0, fieldName);
/* 064 */         }
```

Example of code generated now:
```scala
/* 060 */         boolean isNull1 = value.isNullAt(0);
/* 061 */         UTF8String value1 = isNull1 ? null : value.getUTF8String(0);
/* 062 */         if (isNull1) {
/* 063 */           rowWriter1.setNullAt(0);
/* 064 */         } else {
/* 065 */           rowWriter1.write(0, value1);
/* 066 */         }
```

## How was this patch tested?

Adds GenerateUnsafeProjectionSuite.

Author: Ala Luszczak <ala@databricks.com>

Closes #18030 from ala/fix-generate-unsafe-projection.
2017-05-19 13:18:48 +02:00
hyukjinkwon 8fb3d5c6da [SPARK-20364][SQL] Disable Parquet predicate pushdown for fields having dots in the names
## What changes were proposed in this pull request?

This is an alternative workaround by simply avoiding the predicate pushdown for columns having dots in the names. This is an approach different with https://github.com/apache/spark/pull/17680.

The downside of this PR is, literally it does not push down filters on the column having dots in Parquet files at all (both no record level and no rowgroup level) whereas the downside of the approach in that PR, it does not use the Parquet's API properly but in a hacky way to support this case.

I assume we prefer a safe way here by using the Parquet API properly but this does close that PR as we are basically just avoiding here.

This way looks a simple workaround and probably it is fine given the problem looks arguably rather corner cases (although it might end up with reading whole row groups under the hood but either looks not the best).

Currently, if there are dots in the column name, predicate pushdown seems being failed in Parquet.

**With dots**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("col.dots").write.parquet(path)
spark.read.parquet(path).where("`col.dots` IS NOT NULL").show()
```

```
+--------+
|col.dots|
+--------+
+--------+
```

**Without dots**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("coldots").write.parquet(path)
spark.read.parquet(path).where("`coldots` IS NOT NULL").show()
```

```
+-------+
|coldots|
+-------+
|      1|
+-------+
```

**After**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("col.dots").write.parquet(path)
spark.read.parquet(path).where("`col.dots` IS NOT NULL").show()
```

```
+--------+
|col.dots|
+--------+
|       1|
+--------+
```

## How was this patch tested?

Unit tests added in `ParquetFilterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18000 from HyukjinKwon/SPARK-20364-workaround.
2017-05-18 10:52:23 -07:00
Xingbo Jiang b7aac15d56 [SPARK-20700][SQL] InferFiltersFromConstraints stackoverflows for query (v2)
## What changes were proposed in this pull request?

In the previous approach we used `aliasMap` to link an `Attribute` to the expression with potentially the form `f(a, b)`, but we only searched the `expressions` and `children.expressions` for this, which is not enough when an `Alias` may lies deep in the logical plan. In that case, we can't generate the valid equivalent constraint classes and thus we fail at preventing the recursive deductions.

We fix this problem by collecting all `Alias`s from the logical plan.

## How was this patch tested?

No additional test case is added, but do modified one test case to cover this situation.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #18020 from jiangxb1987/inferConstrants.
2017-05-17 23:32:31 -07:00
Liang-Chi Hsieh 7463a88be6 [SPARK-20690][SQL] Subqueries in FROM should have alias names
## What changes were proposed in this pull request?

We add missing attributes into Filter in Analyzer. But we shouldn't do it through subqueries like this:

    select 1 from  (select 1 from onerow t1 LIMIT 1) where  t1.c1=1

This query works in current codebase. However, the outside where clause shouldn't be able to refer `t1.c1` attribute.

The root cause is we allow subqueries in FROM have no alias names previously, it is confusing and isn't supported by various databases such as MySQL, Postgres, Oracle. We shouldn't support it too.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #17935 from viirya/SPARK-20690.
2017-05-17 12:57:35 +08:00
Herman van Hovell 69bb7715f9 [SQL][TRIVIAL] Lower parser log level to debug
## What changes were proposed in this pull request?
Currently the parser logs the query it is parsing at `info` level. This is too high, this PR lowers the log level to `debug`.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #18006 from hvanhovell/lower_parser_log_level.
2017-05-16 15:58:50 -07:00
Kazuaki Ishizaki 6f62e9d9b9 [SPARK-19372][SQL] Fix throwing a Java exception at df.fliter() due to 64KB bytecode size limit
## What changes were proposed in this pull request?

When an expression for `df.filter()` has many nodes (e.g. 400), the size of Java bytecode for the generated Java code is more than 64KB. It produces an Java exception. As a result, the execution fails.
This PR continues to execute by calling `Expression.eval()` disabling code generation if an exception has been caught.

## How was this patch tested?

Add a test suite into `DataFrameSuite`

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

Closes #17087 from kiszk/SPARK-19372.
2017-05-16 14:47:21 -07:00
Takuya UESHIN c8c878a416 [SPARK-20588][SQL] Cache TimeZone instances.
## What changes were proposed in this pull request?

Because the method `TimeZone.getTimeZone(String ID)` is synchronized on the TimeZone class, concurrent call of this method will become a bottleneck.
This especially happens when casting from string value containing timezone info to timestamp value, which uses `DateTimeUtils.stringToTimestamp()` and gets TimeZone instance on the site.

This pr makes a cache of the generated TimeZone instances to avoid the synchronization.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17933 from ueshin/issues/SPARK-20588.
2017-05-15 16:52:22 -07:00
Dongjoon Hyun bbd163d589 [SPARK-20735][SQL][TEST] Enable cross join in TPCDSQueryBenchmark
## What changes were proposed in this pull request?

Since [SPARK-17298](https://issues.apache.org/jira/browse/SPARK-17298), some queries (q28, q61, q77, q88, q90) in the test suites fail with a message "_Use the CROSS JOIN syntax to allow cartesian products between these relations_".

This benchmark is used as a reference model for Spark TPC-DS, so this PR aims to enable the correct configuration in `TPCDSQueryBenchmark.scala`.

## How was this patch tested?

Manual. (Run TPCDSQueryBenchmark)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17977 from dongjoon-hyun/SPARK-20735.
2017-05-15 11:24:30 -07:00
Tathagata Das 499ba2cb47 [SPARK-20717][SS] Minor tweaks to the MapGroupsWithState behavior
## What changes were proposed in this pull request?

Timeout and state data are two independent entities and should be settable independently. Therefore, in the same call of the user-defined function, one should be able to set the timeout before initializing the state and also after removing the state. Whether timeouts can be set or not, should not depend on the current state, and vice versa.

However, a limitation of the current implementation is that state cannot be null while timeout is set. This is checked lazily after the function call has completed.

## How was this patch tested?
- Updated existing unit tests that test the behavior of GroupState.setTimeout*** wrt to the current state
- Added new tests that verify the disallowed cases where state is undefined but timeout is set.

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

Closes #17957 from tdas/SPARK-20717.
2017-05-15 10:48:10 -07:00
Tejas Patil d2416925c4 [SPARK-17729][SQL] Enable creating hive bucketed tables
## What changes were proposed in this pull request?

Hive allows inserting data to bucketed table without guaranteeing bucketed and sorted-ness based on these two configs : `hive.enforce.bucketing` and `hive.enforce.sorting`.

What does this PR achieve ?
- Spark will disallow users from writing outputs to hive bucketed tables by default (given that output won't adhere with Hive's semantics).
- IF user still wants to write to hive bucketed table, the only resort is to use `hive.enforce.bucketing=false` and `hive.enforce.sorting=false` which means user does NOT care about bucketing guarantees.

Changes done in this PR:
- Extract table's bucketing information in `HiveClientImpl`
- While writing table info to metastore, `HiveClientImpl` now populates the bucketing information in the hive `Table` object
- `InsertIntoHiveTable` allows inserts to bucketed table only if both `hive.enforce.bucketing` and `hive.enforce.sorting` are `false`

Ability to create bucketed tables will enable adding test cases to Spark while I add more changes related to hive bucketing support. Design doc for hive hive bucketing support : https://docs.google.com/document/d/1a8IDh23RAkrkg9YYAeO51F4aGO8-xAlupKwdshve2fc/edit#

## How was this patch tested?
- Added test for creating bucketed and sorted table.
- Added test to ensure that INSERTs fail if strict bucket / sort is enforced
- Added test to ensure that INSERTs can go through if strict bucket / sort is NOT enforced
- Added test to validate that bucketing information shows up in output of DESC FORMATTED
- Added test to ensure that `SHOW CREATE TABLE` works for hive bucketed tables

Author: Tejas Patil <tejasp@fb.com>

Closes #17644 from tejasapatil/SPARK-17729_create_bucketed_table.
2017-05-16 01:47:23 +08:00
Tathagata Das 271175e2bd [SPARK-20716][SS] StateStore.abort() should not throw exceptions
## What changes were proposed in this pull request?

StateStore.abort() should do a best effort attempt to clean up temporary resources. It should not throw errors, especially because its called in a TaskCompletionListener, because this error could hide previous real errors in the task.

## How was this patch tested?
No unit test.

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

Closes #17958 from tdas/SPARK-20716.
2017-05-15 10:46:38 -07:00
Takeshi Yamamuro b0888d1ac3 [SPARK-20730][SQL] Add an optimizer rule to combine nested Concat
## What changes were proposed in this pull request?
This pr added a new Optimizer rule to combine nested Concat. The master supports a pipeline operator '||' to concatenate strings in #17711 (This pr is follow-up). Since the parser currently generates nested Concat expressions, the optimizer needs to combine the nested expressions.

## How was this patch tested?
Added tests in `CombineConcatSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17970 from maropu/SPARK-20730.
2017-05-15 16:24:55 +08:00
Wenchen Fan 1283c3d11a [SPARK-20725][SQL] partial aggregate should behave correctly for sameResult
## What changes were proposed in this pull request?

For aggregate function with `PartialMerge` or `Final` mode, the input is aggregate buffers instead of the actual children expressions. So the actual children expressions won't affect the result, we should normalize the expr id for them.

## How was this patch tested?

a new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17964 from cloud-fan/tmp.
2017-05-13 12:09:06 -07:00
hyukjinkwon 3f98375d8a [SPARK-18772][SQL] Avoid unnecessary conversion try for special floats in JSON
## What changes were proposed in this pull request?

This PR is based on  https://github.com/apache/spark/pull/16199 and extracts the valid change from https://github.com/apache/spark/pull/9759 to resolve SPARK-18772

This avoids additional conversion try with `toFloat` and `toDouble`.

For avoiding additional conversions, please refer the codes below:

**Before**

```scala
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._

scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:30:41 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)
java.lang.NumberFormatException: For input string: "nan"
...
```

**After**

```scala
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._

scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:44:30 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.RuntimeException: Cannot parse nan as DoubleType.
...
```

## How was this patch tested?

Unit tests added in `JsonSuite`.

Closes #16199

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Nathan Howell <nhowell@godaddy.com>

Closes #17956 from HyukjinKwon/SPARK-18772.
2017-05-13 20:56:04 +08:00
Xiao Li b84ff7eb62 [SPARK-20719][SQL] Support LIMIT ALL
### What changes were proposed in this pull request?
`LIMIT ALL` is the same as omitting the `LIMIT` clause. It is supported by both PrestgreSQL and Presto. This PR is to support it by adding it in the parser.

### How was this patch tested?
Added a test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17960 from gatorsmile/LimitAll.
2017-05-12 15:26:10 -07:00
zuotingbing e3d2022e4b [SPARK-20594][SQL] The staging directory should be a child directory starts with "." to avoid being deleted if we set hive.exec.stagingdir under the table directory.
JIRA Issue: https://issues.apache.org/jira/browse/SPARK-20594

## What changes were proposed in this pull request?

The staging directory should be a child directory starts with "." to avoid being deleted before moving staging directory to table directory if we set hive.exec.stagingdir under the table directory.

## How was this patch tested?

Added unit tests

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #17858 from zuotingbing/spark-stagingdir.
2017-05-12 11:24:07 -07:00
Tathagata Das 0d3a63193c [SPARK-20714][SS] Fix match error when watermark is set with timeout = no timeout / processing timeout
## What changes were proposed in this pull request?

When watermark is set, and timeout conf is NoTimeout or ProcessingTimeTimeout (both do not need the watermark), the query fails at runtime with the following exception.
```
MatchException: Some(org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate1a9b798e) (of class scala.Some)
    org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:120)
    org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:116)
    org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:70)
    org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:65)
    org.apache.spark.sql.execution.streaming.state.StateStoreRDD.compute(StateStoreRDD.scala:64)
```

The match did not correctly handle cases where watermark was defined by the timeout was different from EventTimeTimeout.

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

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

Closes #17954 from tdas/SPARK-20714.
2017-05-12 10:49:50 -07:00
Takeshi Yamamuro b526f70c16 [SPARK-19951][SQL] Add string concatenate operator || to Spark SQL
## What changes were proposed in this pull request?
This pr added code to support `||` for string concatenation. This string operation is supported in PostgreSQL and MySQL.

## How was this patch tested?
Added tests in `SparkSqlParserSuite`

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17711 from maropu/SPARK-19951.
2017-05-12 09:55:51 -07:00
Takeshi Yamamuro 92ea7fd7b6 [SPARK-20710][SQL] Support aliases in CUBE/ROLLUP/GROUPING SETS
## What changes were proposed in this pull request?
This pr added  `Analyzer` code for supporting aliases in CUBE/ROLLUP/GROUPING SETS (This is follow-up of #17191).

## How was this patch tested?
Added tests in `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17948 from maropu/SPARK-20710.
2017-05-12 20:48:30 +08:00
wangzhenhua 54b4f2ad43 [SPARK-20718][SQL][FOLLOWUP] Fix canonicalization for HiveTableScanExec
## What changes were proposed in this pull request?

Fix canonicalization for different filter orders in `HiveTableScanExec`.

## How was this patch tested?

Added a new test case.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17962 from wzhfy/canonicalizeHiveTableScanExec.
2017-05-12 20:43:22 +08:00
Ryan Blue b236933907 [SPARK-17424] Fix unsound substitution bug in ScalaReflection.
## What changes were proposed in this pull request?

This method gets a type's primary constructor and fills in type parameters with concrete types. For example, `MapPartitions[T, U] -> MapPartitions[Int, String]`. This Substitution fails when the actual type args are empty because they are still unknown. Instead, when there are no resolved types to subsitute, this returns the original args with unresolved type parameters.
## How was this patch tested?

This doesn't affect substitutions where the type args are determined. This fixes our case where the actual type args are empty and our job runs successfully.

Author: Ryan Blue <blue@apache.org>

Closes #15062 from rdblue/SPARK-17424-fix-unsound-reflect-substitution.
2017-05-12 20:38:36 +08:00
Sean Owen fc8a2b6ee6 [SPARK-20554][BUILD] Remove usage of scala.language.reflectiveCalls
## What changes were proposed in this pull request?

Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17949 from srowen/SPARK-20554.
2017-05-12 09:55:04 +01:00
hyukjinkwon 720708ccdd [SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`).

- Support single argument for `to_timestamp` similarly with APIs in other languages.

  For example, the one below works

  ```
  import org.apache.spark.sql.functions._
  Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show()
  ```

  prints

  ```
  +----------------------------------------+
  |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')|
  +----------------------------------------+
  |                     2016-12-31 00:12:00|
  +----------------------------------------+
  ```

  whereas this does not work in SQL.

  **Before**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7
  ```

  **After**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  2016-12-31 00:12:00
  ```

- Related document improvement for SQL function descriptions and other API descriptions accordingly.

  **Before**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
  Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00.0
  ```

  **After**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage:
      to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to
        a date. Returns null with invalid input. By default, it follows casting rules to a date if
        the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_date('2009-07-30 04:17:52');
         2009-07-30
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
   Usage:
      to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to
        a timestamp. Returns null with invalid input. By default, it follows casting rules to
        a timestamp if the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31 00:12:00');
         2016-12-31 00:12:00
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00
  ```

## How was this patch tested?

Added tests in `datetime.sql`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 16:42:58 +08:00
wangzhenhua c8da535600 [SPARK-20718][SQL] FileSourceScanExec with different filter orders should be the same after canonicalization
## What changes were proposed in this pull request?

Since `constraints` in `QueryPlan` is a set, the order of filters can differ. Usually this is ok because of canonicalization. However, in `FileSourceScanExec`, its data filters and partition filters are sequences, and their orders are not canonicalized. So `def sameResult` returns different results for different orders of data/partition filters. This leads to, e.g. different decision for `ReuseExchange`, and thus results in unstable performance.

## How was this patch tested?

Added a new test for `FileSourceScanExec.sameResult`.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17959 from wzhfy/canonicalizeFileSourceScanExec.
2017-05-12 13:42:48 +08:00
liuxian 2b36eb696f [SPARK-20665][SQL] Bround" and "Round" function return NULL
## What changes were proposed in this pull request?
   spark-sql>select bround(12.3, 2);
   spark-sql>NULL
For this case,  the expected result is 12.3, but it is null.
So ,when the second parameter is bigger than "decimal.scala", the result is not we expected.
"round" function  has the same problem. This PR can solve the problem for both of them.

## How was this patch tested?
unit test cases in MathExpressionsSuite and MathFunctionsSuite

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17906 from 10110346/wip_lx_0509.
2017-05-12 11:38:50 +08:00
Liang-Chi Hsieh 609ba5f2b9 [SPARK-20399][SQL] Add a config to fallback string literal parsing consistent with old sql parser behavior
## What changes were proposed in this pull request?

The new SQL parser is introduced into Spark 2.0. All string literals are unescaped in parser. Seems it bring an issue regarding the regex pattern string.

The following codes can reproduce it:

    val data = Seq("\u0020\u0021\u0023", "abc")
    val df = data.toDF()

    // 1st usage: works in 1.6
    // Let parser parse pattern string
    val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'")
    // 2nd usage: works in 1.6, 2.x
    // Call Column.rlike so the pattern string is a literal which doesn't go through parser
    val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$"))

    // In 2.x, we need add backslashes to make regex pattern parsed correctly
    val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'")

Follow the discussion in #17736, this patch adds a config to fallback to 1.6 string literal parsing and mitigate migration issue.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #17887 from viirya/add-config-fallback-string-parsing.
2017-05-12 11:15:10 +08:00
Takeshi Yamamuro 04901dd03a [SPARK-20431][SQL] Specify a schema by using a DDL-formatted string
## What changes were proposed in this pull request?
This pr supported a DDL-formatted string in `DataFrameReader.schema`.
This fix could make users easily define a schema without importing  `o.a.spark.sql.types._`.

## How was this patch tested?
Added tests in `DataFrameReaderWriterSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17719 from maropu/SPARK-20431.
2017-05-11 11:06:29 -07:00
Takeshi Yamamuro 3aa4e464a8 [SPARK-20416][SQL] Print UDF names in EXPLAIN
## What changes were proposed in this pull request?
This pr added `withName` in `UserDefinedFunction` for printing UDF names in EXPLAIN

## How was this patch tested?
Added tests in `UDFSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17712 from maropu/SPARK-20416.
2017-05-11 09:49:05 -07:00
Takeshi Yamamuro 8c67aa7f00 [SPARK-20311][SQL] Support aliases for table value functions
## What changes were proposed in this pull request?
This pr added parsing rules to support aliases in table value functions.
The previous pr (#17666) has been reverted because of the regression. This new pr fixed the regression and add tests in `SQLQueryTestSuite`.

## How was this patch tested?
Added tests in `PlanParserSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17928 from maropu/SPARK-20311-3.
2017-05-11 18:09:31 +08:00
Wenchen Fan b4c99f4369 [SPARK-20569][SQL] RuntimeReplaceable functions should not take extra parameters
## What changes were proposed in this pull request?

`RuntimeReplaceable` always has a constructor with the expression to replace with, and this constructor should not be the function builder.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17876 from cloud-fan/minor.
2017-05-11 00:41:15 -07:00
Robert Kruszewski 65accb813a [SPARK-17029] make toJSON not go through rdd form but operate on dataset always
## What changes were proposed in this pull request?

Don't convert toRdd when doing toJSON
## How was this patch tested?

Existing unit tests

Author: Robert Kruszewski <robertk@palantir.com>

Closes #14615 from robert3005/robertk/correct-tojson.
2017-05-11 15:26:48 +08:00
Ala Luszczak 5c2c4dcce5 [SPARK-19447] Remove remaining references to generated rows metric
## What changes were proposed in this pull request?

b486ffc86d left behind references to "number of generated rows" metrics, that should have been removed.

## How was this patch tested?

Existing unit tests.

Author: Ala Luszczak <ala@databricks.com>

Closes #17939 from ala/SPARK-19447-fix.
2017-05-10 08:41:04 -07:00
wangzhenhua 76e4a5566b [SPARK-20678][SQL] Ndv for columns not in filter condition should also be updated
## What changes were proposed in this pull request?

In filter estimation, we update column stats for those columns in filter condition. However, if the number of rows decreases after the filter (i.e. the overall selectivity is less than 1), we need to update (scale down) the number of distinct values (NDV) for all columns, no matter they are in filter conditions or not.

This pr also fixes the inconsistency of rounding mode for ndv and rowCount.

## How was this patch tested?

Added new tests.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17918 from wzhfy/scaleDownNdvAfterFilter.
2017-05-10 19:42:49 +08:00
Wenchen Fan 789bdbe3d0 [SPARK-20688][SQL] correctly check analysis for scalar sub-queries
## What changes were proposed in this pull request?

In `CheckAnalysis`, we should call `checkAnalysis` for `ScalarSubquery` at the beginning, as later we will call `plan.output` which is invalid if `plan` is not resolved.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17930 from cloud-fan/tmp.
2017-05-10 19:30:00 +08:00
NICHOLAS T. MARION b512233a45 [SPARK-20393][WEBU UI] Strengthen Spark to prevent XSS vulnerabilities
## What changes were proposed in this pull request?

Add stripXSS and stripXSSMap to Spark Core's UIUtils. Calling these functions at any point that getParameter is called against a HttpServletRequest.

## How was this patch tested?

Unit tests, IBM Security AppScan Standard no longer showing vulnerabilities, manual verification of WebUI pages.

Author: NICHOLAS T. MARION <nmarion@us.ibm.com>

Closes #17686 from n-marion/xss-fix.
2017-05-10 10:59:57 +01:00
Takuya UESHIN 0ef16bd4b0 [SPARK-20668][SQL] Modify ScalaUDF to handle nullability.
## What changes were proposed in this pull request?

When registering Scala UDF, we can know if the udf will return nullable value or not. `ScalaUDF` and related classes should handle the nullability.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17911 from ueshin/issues/SPARK-20668.
2017-05-09 23:48:25 -07:00
Josh Rosen a90c5cd822 [SPARK-20686][SQL] PropagateEmptyRelation incorrectly handles aggregate without grouping
## What changes were proposed in this pull request?

The query

```
SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1
```

should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows.

This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead:

An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows.

If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation.

The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be).

This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation.

## How was this patch tested?

- Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file.
- Updated unit tests in `PropagateEmptyRelationSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17929 from JoshRosen/fix-PropagateEmptyRelation.
2017-05-10 14:36:36 +08:00
hyukjinkwon 3d2131ab4d [SPARK-20590][SQL] Use Spark internal datasource if multiples are found for the same shorten name
## What changes were proposed in this pull request?

One of the common usability problems around reading data in spark (particularly CSV) is that there can often be a conflict between different readers in the classpath.

As an example, if someone launches a 2.x spark shell with the spark-csv package in the classpath, Spark currently fails in an extremely unfriendly way (see databricks/spark-csv#367):

```bash
./bin/spark-shell --packages com.databricks:spark-csv_2.11:1.5.0
scala> val df = spark.read.csv("/foo/bar.csv")
java.lang.RuntimeException: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat, com.databricks.spark.csv.DefaultSource15), please specify the fully qualified class name.
  at scala.sys.package$.error(package.scala:27)
  at org.apache.spark.sql.execution.datasources.DataSource$.lookupDataSource(DataSource.scala:574)
  at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:85)
  at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:85)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:295)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
  at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:533)
  at org.apache.spark.sql.DataFrameReader.csv(DataFrameReader.scala:412)
  ... 48 elided
```

This PR proposes a simple way of fixing this error by picking up the internal datasource if there is single (the datasource that has "org.apache.spark" prefix).

```scala
scala> spark.range(1).write.format("csv").mode("overwrite").save("/tmp/abc")
17/05/10 09:47:44 WARN DataSource: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
```

```scala
scala> spark.range(1).write.format("Csv").mode("overwrite").save("/tmp/abc")
17/05/10 09:47:52 WARN DataSource: Multiple sources found for Csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
```

## How was this patch tested?

Manually tested as below:

```bash
./bin/spark-shell --packages com.databricks:spark-csv_2.11:1.5.0
```

```scala
spark.sparkContext.setLogLevel("WARN")
```

**positive cases**:

```scala
scala> spark.range(1).write.format("csv").mode("overwrite").save("/tmp/abc")
17/05/10 09:47:44 WARN DataSource: Multiple sources found for csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
```

```scala
scala> spark.range(1).write.format("Csv").mode("overwrite").save("/tmp/abc")
17/05/10 09:47:52 WARN DataSource: Multiple sources found for Csv (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat,
com.databricks.spark.csv.DefaultSource15), defaulting to the internal datasource (org.apache.spark.sql.execution.datasources.csv.CSVFileFormat).
```

(newlines were inserted for readability).

```scala
scala> spark.range(1).write.format("com.databricks.spark.csv").mode("overwrite").save("/tmp/abc")
```

```scala
scala> spark.range(1).write.format("org.apache.spark.sql.execution.datasources.csv.CSVFileFormat").mode("overwrite").save("/tmp/abc")
```

**negative cases**:

```scala
scala> spark.range(1).write.format("com.databricks.spark.csv.CsvRelation").save("/tmp/abc")
java.lang.InstantiationException: com.databricks.spark.csv.CsvRelation
...
```

```scala
scala> spark.range(1).write.format("com.databricks.spark.csv.CsvRelatio").save("/tmp/abc")
java.lang.ClassNotFoundException: Failed to find data source: com.databricks.spark.csv.CsvRelatio. Please find packages at http://spark.apache.org/third-party-projects.html
...
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17916 from HyukjinKwon/datasource-detect.
2017-05-10 13:44:47 +08:00
Yuming Wang 771abeb46f [SPARK-17685][SQL] Make SortMergeJoinExec's currentVars is null when calling createJoinKey
## What changes were proposed in this pull request?

The following SQL query cause `IndexOutOfBoundsException` issue when `LIMIT > 1310720`:
```sql
CREATE TABLE tab1(int int, int2 int, str string);
CREATE TABLE tab2(int int, int2 int, str string);
INSERT INTO tab1 values(1,1,'str');
INSERT INTO tab1 values(2,2,'str');
INSERT INTO tab2 values(1,1,'str');
INSERT INTO tab2 values(2,3,'str');

SELECT
  count(*)
FROM
  (
    SELECT t1.int, t2.int2
    FROM (SELECT * FROM tab1 LIMIT 1310721) t1
    INNER JOIN (SELECT * FROM tab2 LIMIT 1310721) t2
    ON (t1.int = t2.int AND t1.int2 = t2.int2)
  ) t;
```

This pull request fix this issue.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17920 from wangyum/SPARK-17685.
2017-05-09 19:45:00 -07:00
uncleGen c0189abc7c [SPARK-20373][SQL][SS] Batch queries with 'Dataset/DataFrame.withWatermark()` does not execute
## What changes were proposed in this pull request?

Any Dataset/DataFrame batch query with the operation `withWatermark` does not execute because the batch planner does not have any rule to explicitly handle the EventTimeWatermark logical plan.
The right solution is to simply remove the plan node, as the watermark should not affect any batch query in any way.

Changes:
- In this PR, we add a new rule `EliminateEventTimeWatermark` to check if we need to ignore the event time watermark. We will ignore watermark in any batch query.

Depends upon:
- [SPARK-20672](https://issues.apache.org/jira/browse/SPARK-20672). We can not add this rule into analyzer directly, because streaming query will be copied to `triggerLogicalPlan ` in every trigger, and the rule will be applied to `triggerLogicalPlan` mistakenly.

Others:
- A typo fix in example.

## How was this patch tested?

add new unit test.

Author: uncleGen <hustyugm@gmail.com>

Closes #17896 from uncleGen/SPARK-20373.
2017-05-09 15:08:09 -07:00
Yin Huai f79aa285cf Revert "[SPARK-20311][SQL] Support aliases for table value functions"
This reverts commit 714811d0b5.
2017-05-09 14:47:45 -07:00
Reynold Xin ac1ab6b9db Revert "[SPARK-12297][SQL] Hive compatibility for Parquet Timestamps"
This reverts commit 22691556e5.

See JIRA ticket for more information.
2017-05-09 11:35:59 -07:00
Sean Owen 25ee816e09 [SPARK-19876][BUILD] Move Trigger.java to java source hierarchy
## What changes were proposed in this pull request?

Simply moves `Trigger.java` to `src/main/java` from `src/main/scala`
See https://github.com/apache/spark/pull/17219

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17921 from srowen/SPARK-19876.2.
2017-05-09 10:22:23 -07:00
Reynold Xin d099f414d2 [SPARK-20674][SQL] Support registering UserDefinedFunction as named UDF
## What changes were proposed in this pull request?
For some reason we don't have an API to register UserDefinedFunction as named UDF. It is a no brainer to add one, in addition to the existing register functions we have.

## How was this patch tested?
Added a test case in UDFSuite for the new API.

Author: Reynold Xin <rxin@databricks.com>

Closes #17915 from rxin/SPARK-20674.
2017-05-09 09:24:28 -07:00
Takeshi Yamamuro 714811d0b5 [SPARK-20311][SQL] Support aliases for table value functions
## What changes were proposed in this pull request?
This pr added parsing rules to support aliases in table value functions.

## How was this patch tested?
Added tests in `PlanParserSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17666 from maropu/SPARK-20311.
2017-05-09 20:22:51 +08:00
Xiao Li 0d00c768a8 [SPARK-20667][SQL][TESTS] Cleanup the cataloged metadata after completing the package of sql/core and sql/hive
## What changes were proposed in this pull request?

So far, we do not drop all the cataloged objects after each package. Sometimes, we might hit strange test case errors because the previous test suite did not drop the cataloged/temporary objects (tables/functions/database). At least, we can first clean up the environment when completing the package of `sql/core` and `sql/hive`.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17908 from gatorsmile/reset.
2017-05-09 20:10:50 +08:00
sujith71955 42cc6d13ed [SPARK-20380][SQL] Unable to set/unset table comment property using ALTER TABLE SET/UNSET TBLPROPERTIES ddl
### What changes were proposed in this pull request?
Table comment was not getting  set/unset using **ALTER TABLE  SET/UNSET TBLPROPERTIES** query
eg: ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment)
 when user alter the table properties  and adds/updates table comment,table comment which is a field  of **CatalogTable**  instance is not getting updated and  old table comment if exists was shown to user, inorder  to handle this issue, update the comment field value in **CatalogTable** with the newly added/modified comment along with other table level properties when user executes **ALTER TABLE  SET TBLPROPERTIES** query.

This pr has also taken care of unsetting the table comment when user executes query  **ALTER TABLE  UNSET TBLPROPERTIES** inorder to unset or remove table comment.
eg: ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment')

### How was this patch tested?
Added test cases  as part of **SQLQueryTestSuite** for verifying  table comment using desc formatted table query after adding/modifying table comment as part of **AlterTableSetPropertiesCommand** and unsetting the table comment using **AlterTableUnsetPropertiesCommand**.

Author: sujith71955 <sujithchacko.2010@gmail.com>

Closes #17649 from sujith71955/alter_table_comment.
2017-05-07 23:15:00 -07:00
Imran Rashid 22691556e5 [SPARK-12297][SQL] Hive compatibility for Parquet Timestamps
## What changes were proposed in this pull request?

This change allows timestamps in parquet-based hive table to behave as a "floating time", without a timezone, as timestamps are for other file formats.  If the storage timezone is the same as the session timezone, this conversion is a no-op.  When data is read from a hive table, the table property is *always* respected.  This allows spark to not change behavior when reading old data, but read newly written data correctly (whatever the source of the data is).

Spark inherited the original behavior from Hive, but Hive is also updating behavior to use the same  scheme in HIVE-12767 / HIVE-16231.

The default for Spark remains unchanged; created tables do not include the new table property.

This will only apply to hive tables; nothing is added to parquet metadata to indicate the timezone, so data that is read or written directly from parquet files will never have any conversions applied.

## How was this patch tested?

Added a unit test which creates tables, reads and writes data, under a variety of permutations (different storage timezones, different session timezones, vectorized reading on and off).

Author: Imran Rashid <irashid@cloudera.com>

Closes #16781 from squito/SPARK-12297.
2017-05-08 12:16:00 +09:00
Jacek Laskowski 500436b436 [MINOR][SQL][DOCS] Improve unix_timestamp's scaladoc (and typo hunting)
## What changes were proposed in this pull request?

* Docs are consistent (across different `unix_timestamp` variants and their internal expressions)
* typo hunting

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17801 from jaceklaskowski/unix_timestamp.
2017-05-07 13:56:13 -07:00
Xiao Li cafca54c0e [SPARK-20557][SQL] Support JDBC data type Time with Time Zone
### What changes were proposed in this pull request?

This PR is to support JDBC data type TIME WITH TIME ZONE. It can be converted to TIMESTAMP

In addition, before this PR, for unsupported data types, we simply output the type number instead of the type name.

```
java.sql.SQLException: Unsupported type 2014
```
After this PR, the message is like
```
java.sql.SQLException: Unsupported type TIMESTAMP_WITH_TIMEZONE
```

- Also upgrade the H2 version to `1.4.195` which has the type fix for "TIMESTAMP WITH TIMEZONE". However, it is not fully supported. Thus, we capture the exception, but we still need it to partially test the support of "TIMESTAMP WITH TIMEZONE", because Docker tests are not regularly run.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17835 from gatorsmile/h2.
2017-05-06 22:21:19 -07:00
Juliusz Sompolski 5d75b14bf0 [SPARK-20616] RuleExecutor logDebug of batch results should show diff to start of batch
## What changes were proposed in this pull request?

Due to a likely typo, the logDebug msg printing the diff of query plans shows a diff to the initial plan, not diff to the start of batch.

## How was this patch tested?

Now the debug message prints the diff between start and end of batch.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17875 from juliuszsompolski/SPARK-20616.
2017-05-05 15:31:06 -07:00
Jannik Arndt b31648c081 [SPARK-20557][SQL] Support for db column type TIMESTAMP WITH TIME ZONE
## What changes were proposed in this pull request?

SparkSQL can now read from a database table with column type [TIMESTAMP WITH TIME ZONE](https://docs.oracle.com/javase/8/docs/api/java/sql/Types.html#TIMESTAMP_WITH_TIMEZONE).

## How was this patch tested?

Tested against Oracle database.

JoshRosen, you seem to know the class, would you look at this? Thanks!

Author: Jannik Arndt <jannik@jannikarndt.de>

Closes #17832 from JannikArndt/spark-20557-timestamp-with-timezone.
2017-05-05 11:42:55 -07:00
Yucai 41439fd52d [SPARK-20381][SQL] Add SQL metrics of numOutputRows for ObjectHashAggregateExec
## What changes were proposed in this pull request?

ObjectHashAggregateExec is missing numOutputRows, add this metrics for it.

## How was this patch tested?

Added unit tests for the new metrics.

Author: Yucai <yucai.yu@intel.com>

Closes #17678 from yucai/objectAgg_numOutputRows.
2017-05-05 09:51:57 -07:00
madhu 9064f1b044 [SPARK-20495][SQL][CORE] Add StorageLevel to cacheTable API
## What changes were proposed in this pull request?
Currently cacheTable API only supports MEMORY_AND_DISK. This PR adds additional API to take different storage levels.
## How was this patch tested?
unit tests

Author: madhu <phatak.dev@gmail.com>

Closes #17802 from phatak-dev/cacheTableAPI.
2017-05-05 22:44:03 +08:00
Yuming Wang 37cdf077cd [SPARK-19660][SQL] Replace the deprecated property name fs.default.name to fs.defaultFS that newly introduced
## What changes were proposed in this pull request?

Replace the deprecated property name `fs.default.name` to `fs.defaultFS` that newly introduced.

## How was this patch tested?

Existing tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17856 from wangyum/SPARK-19660.
2017-05-05 11:31:59 +01:00
Dongjoon Hyun bfc8c79c8d [SPARK-20566][SQL] ColumnVector should support appendFloats for array
## What changes were proposed in this pull request?

This PR aims to add a missing `appendFloats` API for array into **ColumnVector** class. For double type, there is `appendDoubles` for array [here](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java#L818-L824).

## How was this patch tested?

Pass the Jenkins with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17836 from dongjoon-hyun/SPARK-20566.
2017-05-04 21:04:15 +08:00
hyukjinkwon 13eb37c860 [MINOR][SQL] Fix the test title from =!= to <=>, remove a duplicated test and add a test for =!=
## What changes were proposed in this pull request?

This PR proposes three things as below:

- This test looks not testing `<=>` and identical with the test above, `===`. So, it removes the test.

  ```diff
  -   test("<=>") {
  -     checkAnswer(
  -      testData2.filter($"a" === 1),
  -      testData2.collect().toSeq.filter(r => r.getInt(0) == 1))
  -
  -    checkAnswer(
  -      testData2.filter($"a" === $"b"),
  -      testData2.collect().toSeq.filter(r => r.getInt(0) == r.getInt(1)))
  -   }
  ```

- Replace the test title from `=!=` to `<=>`. It looks the test actually testing `<=>`.

  ```diff
  +  private lazy val nullData = Seq(
  +    (Some(1), Some(1)), (Some(1), Some(2)), (Some(1), None), (None, None)).toDF("a", "b")
  +
    ...
  -  test("=!=") {
  +  test("<=>") {
  -    val nullData = spark.createDataFrame(sparkContext.parallelize(
  -      Row(1, 1) ::
  -      Row(1, 2) ::
  -      Row(1, null) ::
  -      Row(null, null) :: Nil),
  -      StructType(Seq(StructField("a", IntegerType), StructField("b", IntegerType))))
  -
         checkAnswer(
           nullData.filter($"b" <=> 1),
    ...
  ```

- Add the tests for `=!=` which looks not existing.

  ```diff
  +  test("=!=") {
  +    checkAnswer(
  +      nullData.filter($"b" =!= 1),
  +      Row(1, 2) :: Nil)
  +
  +    checkAnswer(nullData.filter($"b" =!= null), Nil)
  +
  +    checkAnswer(
  +      nullData.filter($"a" =!= $"b"),
  +      Row(1, 2) :: Nil)
  +  }
  ```

## How was this patch tested?

Manually running the tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17842 from HyukjinKwon/minor-test-fix.
2017-05-03 13:08:25 -07:00
Liwei Lin 6b9e49d12f [SPARK-19965][SS] DataFrame batch reader may fail to infer partitions when reading FileStreamSink's output
## The Problem

Right now DataFrame batch reader may fail to infer partitions when reading FileStreamSink's output:

```
[info] - partitioned writing and batch reading with 'basePath' *** FAILED *** (3 seconds, 928 milliseconds)
[info]   java.lang.AssertionError: assertion failed: Conflicting directory structures detected. Suspicious paths:
[info] 	***/stream.output-65e3fa45-595a-4d29-b3df-4c001e321637
[info] 	***/stream.output-65e3fa45-595a-4d29-b3df-4c001e321637/_spark_metadata
[info]
[info] If provided paths are partition directories, please set "basePath" in the options of the data source to specify the root directory of the table. If there are multiple root directories, please load them separately and then union them.
[info]   at scala.Predef$.assert(Predef.scala:170)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningUtils$.parsePartitions(PartitioningUtils.scala:133)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningUtils$.parsePartitions(PartitioningUtils.scala:98)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning(PartitioningAwareFileIndex.scala:156)
[info]   at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.partitionSpec(InMemoryFileIndex.scala:54)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.partitionSchema(PartitioningAwareFileIndex.scala:55)
[info]   at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:133)
[info]   at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:361)
[info]   at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:160)
[info]   at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:536)
[info]   at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:520)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply$mcV$sp(FileStreamSinkSuite.scala:292)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply(FileStreamSinkSuite.scala:268)
[info]   at org.apache.spark.sql.streaming.FileStreamSinkSuite$$anonfun$8.apply(FileStreamSinkSuite.scala:268)
```

## What changes were proposed in this pull request?

This patch alters `InMemoryFileIndex` to filter out these `basePath`s whose ancestor is the streaming metadata dir (`_spark_metadata`). E.g., the following and other similar dir or files will be filtered out:
- (introduced by globbing `basePath/*`)
   - `basePath/_spark_metadata`
- (introduced by globbing `basePath/*/*`)
   - `basePath/_spark_metadata/0`
   - `basePath/_spark_metadata/1`
   - ...

## How was this patch tested?

Added unit tests

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17346 from lw-lin/filter-metadata.
2017-05-03 11:10:24 -07:00
Reynold Xin 527fc5d0c9 [SPARK-20576][SQL] Support generic hint function in Dataset/DataFrame
## What changes were proposed in this pull request?
We allow users to specify hints (currently only "broadcast" is supported) in SQL and DataFrame. However, while SQL has a standard hint format (/*+ ... */), DataFrame doesn't have one and sometimes users are confused that they can't find how to apply a broadcast hint. This ticket adds a generic hint function on DataFrame that allows using the same hint on DataFrames as well as SQL.

As an example, after this patch, the following will apply a broadcast hint on a DataFrame using the new hint function:

```
df1.join(df2.hint("broadcast"))
```

## How was this patch tested?
Added a test case in DataFrameJoinSuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #17839 from rxin/SPARK-20576.
2017-05-03 09:22:25 -07:00
Liwei Lin 27f543b15f [SPARK-20441][SPARK-20432][SS] Within the same streaming query, one StreamingRelation should only be transformed to one StreamingExecutionRelation
## What changes were proposed in this pull request?

Within the same streaming query, when one `StreamingRelation` is referred multiple times – e.g. `df.union(df)` – we should transform it only to one `StreamingExecutionRelation`, instead of two or more different `StreamingExecutionRelation`s (each of which would have a separate set of source, source logs, ...).

## How was this patch tested?

Added two test cases, each of which would fail without this patch.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17735 from lw-lin/SPARK-20441.
2017-05-03 08:55:02 -07:00
Sean Owen 16fab6b0ef [SPARK-20523][BUILD] Clean up build warnings for 2.2.0 release
## What changes were proposed in this pull request?

Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17803 from srowen/SPARK-20523.
2017-05-03 10:18:35 +01:00
Michael Armbrust 6235132a8c [SPARK-20567] Lazily bind in GenerateExec
It is not valid to eagerly bind with the child's output as this causes failures when we attempt to canonicalize the plan (replacing the attribute references with dummies).

Author: Michael Armbrust <michael@databricks.com>

Closes #17838 from marmbrus/fixBindExplode.
2017-05-02 22:44:27 -07:00
Xiao Li b1e639ab09 [SPARK-19235][SQL][TEST][FOLLOW-UP] Enable Test Cases in DDLSuite with Hive Metastore
### What changes were proposed in this pull request?
This is a follow-up of enabling test cases in DDLSuite with Hive Metastore. It consists of the following remaining tasks:
- Run all the `alter table` and `drop table` DDL tests against data source tables when using Hive metastore.
- Do not run any `alter table` and `drop table` DDL test against Hive serde tables when using InMemoryCatalog.
- Reenable `alter table: set serde partition` and `alter table: set serde` tests for Hive serde tables.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17524 from gatorsmile/cleanupDDLSuite.
2017-05-02 16:49:24 +08:00
Burak Yavuz 86174ea89b [SPARK-20549] java.io.CharConversionException: Invalid UTF-32' in JsonToStructs
## What changes were proposed in this pull request?

A fix for the same problem was made in #17693 but ignored `JsonToStructs`. This PR uses the same fix for `JsonToStructs`.

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17826 from brkyvz/SPARK-20549.
2017-05-02 14:08:16 +08:00
Kazuaki Ishizaki afb21bf22a [SPARK-20537][CORE] Fixing OffHeapColumnVector reallocation
## What changes were proposed in this pull request?

As #17773 revealed `OnHeapColumnVector` may copy a part of the original storage.

`OffHeapColumnVector` reallocation also copies to the new storage data up to 'elementsAppended'. This variable is only updated when using the `ColumnVector.appendX` API, while `ColumnVector.putX` is more commonly used.
This PR copies the new storage data up to the previously-allocated size in`OffHeapColumnVector`.

## How was this patch tested?

Existing test suites

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

Closes #17811 from kiszk/SPARK-20537.
2017-05-02 13:56:41 +08:00
ptkool 259860d23d [SPARK-20463] Add support for IS [NOT] DISTINCT FROM.
## What changes were proposed in this pull request?

Add support for the SQL standard distinct predicate to SPARK SQL.

```
<expression> IS [NOT] DISTINCT FROM <expression>
```

## How was this patch tested?

Tested using unit tests, integration tests, manual tests.

Author: ptkool <michael.styles@shopify.com>

Closes #17764 from ptkool/is_not_distinct_from.
2017-05-01 17:05:35 -07:00
Sean Owen af726cd611 [SPARK-20459][SQL] JdbcUtils throws IllegalStateException: Cause already initialized after getting SQLException
## What changes were proposed in this pull request?

Avoid failing to initCause on JDBC exception with cause initialized to null

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17800 from srowen/SPARK-20459.
2017-05-01 17:01:05 -07:00
Kunal Khamar 6fc6cf88d8 [SPARK-20464][SS] Add a job group and description for streaming queries and fix cancellation of running jobs using the job group
## What changes were proposed in this pull request?

Job group: adding a job group is required to properly cancel running jobs related to a query.
Description: the new description makes it easier to group the batches of a query by sorting by name in the Spark Jobs UI.

## How was this patch tested?

- Unit tests
- UI screenshot

  - Order by job id:
![screen shot 2017-04-27 at 5 10 09 pm](https://cloud.githubusercontent.com/assets/7865120/25509468/15452274-2b6e-11e7-87ba-d929816688cf.png)

  - Order by description:
![screen shot 2017-04-27 at 5 10 22 pm](https://cloud.githubusercontent.com/assets/7865120/25509474/1c298512-2b6e-11e7-99b8-fef1ef7665c1.png)

  - Order by job id (no query name):
![screen shot 2017-04-27 at 5 21 33 pm](https://cloud.githubusercontent.com/assets/7865120/25509482/28c96dc8-2b6e-11e7-8df0-9d3cdbb05e36.png)

  - Order by description (no query name):
![screen shot 2017-04-27 at 5 21 44 pm](https://cloud.githubusercontent.com/assets/7865120/25509489/37674742-2b6e-11e7-9357-b5c38ec16ac4.png)

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17765 from kunalkhamar/sc-6696.
2017-05-01 11:37:30 -07:00
Herman van Hovell 6b44c4d63a [SPARK-20534][SQL] Make outer generate exec return empty rows
## What changes were proposed in this pull request?
Generate exec does not produce `null` values if the generator for the input row is empty and the generate operates in outer mode without join. This is caused by the fact that the `join=false` code path is different from the `join=true` code path, and that the `join=false` code path did deal with outer properly. This PR addresses this issue.

## How was this patch tested?
Updated `outer*` tests in `GeneratorFunctionSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17810 from hvanhovell/SPARK-20534.
2017-05-01 09:46:35 -07:00
hyukjinkwon 1ee494d086 [SPARK-20492][SQL] Do not print empty parentheses for invalid primitive types in parser
## What changes were proposed in this pull request?

Currently, when the type string is invalid, it looks printing empty parentheses. This PR proposes a small improvement in an error message by removing it in the parse as below:

```scala
spark.range(1).select($"col".cast("aa"))
```

**Before**

```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa() is not supported.(line 1, pos 0)

== SQL ==
aa
^^^
```

**After**

```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa is not supported.(line 1, pos 0)

== SQL ==
aa
^^^
```

## How was this patch tested?

Unit tests in `DataTypeParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17784 from HyukjinKwon/SPARK-20492.
2017-04-30 08:24:10 -07:00
hyukjinkwon d228cd0b02 [SPARK-20442][PYTHON][DOCS] Fill up documentations for functions in Column API in PySpark
## What changes were proposed in this pull request?

This PR proposes to fill up the documentation with examples for `bitwiseOR`, `bitwiseAND`, `bitwiseXOR`. `contains`, `asc` and `desc` in `Column` API.

Also, this PR fixes minor typos in the documentation and matches some of the contents between Scala doc and Python doc.

Lastly, this PR suggests to use `spark` rather than `sc` in doc tests in `Column` for Python documentation.

## How was this patch tested?

Doc tests were added and manually tested with the commands below:

`./python/run-tests.py --module pyspark-sql`
`./python/run-tests.py --module pyspark-sql --python-executable python3`
`./dev/lint-python`

Output was checked via `make html` under `./python/docs`. The snapshots will be left on the codes with comments.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17737 from HyukjinKwon/SPARK-20442.
2017-04-29 13:46:40 -07:00
hyukjinkwon 70f1bcd7bc [SPARK-20493][R] De-duplicate parse logics for DDL-like type strings in R
## What changes were proposed in this pull request?

It seems we are using `SQLUtils.getSQLDataType` for type string in structField. It looks we can replace this with `CatalystSqlParser.parseDataType`.

They look similar DDL-like type definitions as below:

```scala
scala> Seq(Tuple1(Tuple1("a"))).toDF.show()
```
```
+---+
| _1|
+---+
|[a]|
+---+
```

```scala
scala> Seq(Tuple1(Tuple1("a"))).toDF.select($"_1".cast("struct<_1:string>")).show()
```
```
+---+
| _1|
+---+
|[a]|
+---+
```

Such type strings looks identical when R’s one as below:

```R
> write.df(sql("SELECT named_struct('_1', 'a') as struct"), "/tmp/aa", "parquet")
> collect(read.df("/tmp/aa", "parquet", structType(structField("struct", "struct<_1:string>"))))
  struct
1      a
```

R’s one is stricter because we are checking the types via regular expressions in R side ahead.

Actual logics there look a bit different but as we check it ahead in R side, it looks replacing it would not introduce (I think) no behaviour changes. To make this sure, the tests dedicated for it were added in SPARK-20105. (It looks `structField` is the only place that calls this method).

## How was this patch tested?

Existing tests - https://github.com/apache/spark/blob/master/R/pkg/inst/tests/testthat/test_sparkSQL.R#L143-L194 should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17785 from HyukjinKwon/SPARK-20493.
2017-04-29 11:02:17 -07:00
Tejas Patil 814a61a867 [SPARK-20487][SQL] Display serde for HiveTableScan node in explained plan
## What changes were proposed in this pull request?

This was a suggestion by rxin at https://github.com/apache/spark/pull/17780#issuecomment-298073408

## How was this patch tested?

- modified existing unit test
- manual testing:

```
scala> hc.sql(" SELECT * FROM tejasp_bucketed_partitioned_1  where name = ''  ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = )
   +- 'UnresolvedRelation `tejasp_bucketed_partitioned_1`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#24L, name#25, ds#26]
+- Filter (name#25 = )
   +- SubqueryAlias tejasp_bucketed_partitioned_1
      +- CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]

== Optimized Logical Plan ==
Filter (isnotnull(name#25) && (name#25 = ))
+- CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]

== Physical Plan ==
*Filter (isnotnull(name#25) && (name#25 = ))
+- HiveTableScan [user_id#24L, name#25, ds#26], CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17806 from tejasapatil/add_serde.
2017-04-28 23:12:26 -07:00
caoxuewen ebff519c5e [SPARK-20471] Remove AggregateBenchmark testsuite warning: Two level hashmap is disabled but vectorized hashmap is enabled
What changes were proposed in this pull request?

remove  AggregateBenchmark testsuite warning:
such as '14:26:33.220 WARN org.apache.spark.sql.execution.aggregate.HashAggregateExec: Two level hashmap is disabled but vectorized hashmap is enabled.'

How was this patch tested?
unit tests: AggregateBenchmark
Modify the 'ignore function for 'test funtion

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #17771 from heary-cao/AggregateBenchmark.
2017-04-28 14:47:17 -07:00
Takeshi Yamamuro 59e3a56444 [SPARK-14471][SQL] Aliases in SELECT could be used in GROUP BY
## What changes were proposed in this pull request?
This pr added a new rule in `Analyzer` to resolve aliases in `GROUP BY`.
The current master throws an exception if `GROUP BY` clauses have aliases in `SELECT`;
```
scala> spark.sql("select a a1, a1 + 1 as b, count(1) from t group by a1")
org.apache.spark.sql.AnalysisException: cannot resolve '`a1`' given input columns: [a]; line 1 pos 51;
'Aggregate ['a1], [a#83L AS a1#87L, ('a1 + 1) AS b#88, count(1) AS count(1)#90L]
+- SubqueryAlias t
   +- Project [id#80L AS a#83L]
      +- Range (0, 10, step=1, splits=Some(8))

  at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:77)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:74)
  at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
```

## How was this patch tested?
Added tests in `SQLQuerySuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17191 from maropu/SPARK-14471.
2017-04-28 14:41:53 +08:00
Xiao Li e3c8160433 [SPARK-20476][SQL] Block users to create a table that use commas in the column names
### What changes were proposed in this pull request?
```SQL
hive> create table t1(`a,` string);
OK
Time taken: 1.399 seconds

hive> create table t2(`a,` string, b string);
FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe: columns has 3 elements while columns.types has 2 elements!)

hive> create table t2(`a,` string, b string) stored as parquet;
FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.IllegalArgumentException: ParquetHiveSerde initialization failed. Number of column name and column type differs. columnNames = [a, , b], columnTypes = [string, string]
```
It has a bug in Hive metastore.

When users do not provide alias name in the SELECT query, we call `toPrettySQL` to generate the alias name. For example, the string `get_json_object(jstring, '$.f1')` will be the alias name for the function call in the statement
```SQL
SELECT key, get_json_object(jstring, '$.f1') FROM tempView
```
Above is not an issue for the SELECT query statements. However, for CTAS, we hit the issue due to a bug in Hive metastore. Hive metastore does not like the column names containing commas and returned a confusing error message, like:
```
17/04/26 23:12:56 ERROR [hive.log(397) -- main]: error in initSerDe: org.apache.hadoop.hive.serde2.SerDeException org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe: columns has 2 elements while columns.types has 1 elements!
org.apache.hadoop.hive.serde2.SerDeException: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe: columns has 2 elements while columns.types has 1 elements!
```

Thus, this PR is to block users to create a table in Hive metastore when the table table has a column containing commas in the name.

### How was this patch tested?
Added a test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17781 from gatorsmile/blockIllegalColumnNames.
2017-04-28 14:16:40 +08:00
Wenchen Fan b90bf520fd [SPARK-12837][CORE] Do not send the name of internal accumulator to executor side
## What changes were proposed in this pull request?

When sending accumulator updates back to driver, the network overhead is pretty big as there are a lot of accumulators, e.g. `TaskMetrics` will send about 20 accumulators everytime, there may be a lot of `SQLMetric` if the query plan is complicated.

Therefore, it's critical to reduce the size of serialized accumulator. A simple way is to not send the name of internal accumulators to executor side, as it's unnecessary. When executor sends accumulator updates back to driver, we can look up the accumulator name in `AccumulatorContext` easily. Note that, we still need to send names of normal accumulators, as the user code run at executor side may rely on accumulator names.

In the future, we should reimplement `TaskMetrics` to not rely on accumulators and use custom serialization.

Tried on the example in https://issues.apache.org/jira/browse/SPARK-12837, the size of serialized accumulator has been cut down by about 40%.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17596 from cloud-fan/oom.
2017-04-27 19:38:14 -07:00
Tejas Patil a4aa4665a6 [SPARK-20487][SQL] HiveTableScan node is quite verbose in explained plan
## What changes were proposed in this pull request?

Changed `TreeNode.argString` to handle `CatalogTable` separately (otherwise it would call the default `toString` on the `CatalogTable`)

## How was this patch tested?

- Expanded scope of existing unit test to ensure that verbose information is not present
- Manual testing

Before

```
scala> hc.sql(" SELECT * FROM my_table WHERE name = 'foo' ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = foo)
   +- 'UnresolvedRelation `my_table`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#13L, name#14, ds#15]
+- Filter (name#14 = foo)
   +- SubqueryAlias my_table
      +- CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]

== Optimized Logical Plan ==
Filter (isnotnull(name#14) && (name#14 = foo))
+- CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]

== Physical Plan ==
*Filter (isnotnull(name#14) && (name#14 = foo))
+- HiveTableScan [user_id#13L, name#14, ds#15], CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]
```

After

```
scala> hc.sql(" SELECT * FROM my_table WHERE name = 'foo' ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = foo)
   +- 'UnresolvedRelation `my_table`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#13L, name#14, ds#15]
+- Filter (name#14 = foo)
   +- SubqueryAlias my_table
      +- CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]

== Optimized Logical Plan ==
Filter (isnotnull(name#14) && (name#14 = foo))
+- CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]

== Physical Plan ==
*Filter (isnotnull(name#14) && (name#14 = foo))
+- HiveTableScan [user_id#13L, name#14, ds#15], CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17780 from tejasapatil/SPARK-20487_verbose_plan.
2017-04-27 12:13:16 -07:00
Kris Mok 26ac2ce05c [SPARK-20482][SQL] Resolving Casts is too strict on having time zone set
## What changes were proposed in this pull request?

Relax the requirement that a `TimeZoneAwareExpression` has to have its `timeZoneId` set to be considered resolved.
With this change, a `Cast` (which is a `TimeZoneAwareExpression`) can be considered resolved if the `(fromType, toType)` combination doesn't require time zone information.

Also de-relaxed test cases in `CastSuite` so Casts in that test suite don't get a default`timeZoneId = Option("GMT")`.

## How was this patch tested?

Ran the de-relaxed`CastSuite` and it's passing. Also ran the SQL unit tests and they're passing too.

Author: Kris Mok <kris.mok@databricks.com>

Closes #17777 from rednaxelafx/fix-catalyst-cast-timezone.
2017-04-27 12:08:16 -07:00
Takeshi Yamamuro b4724db19a [SPARK-20425][SQL] Support a vertical display mode for Dataset.show
## What changes were proposed in this pull request?
This pr added a new display mode for `Dataset.show` to print output rows vertically (one line per column value). In the current master, when printing Dataset with many columns, the readability is low like;

```
scala> val df = spark.range(100).selectExpr((0 until 100).map(i => s"rand() AS c$i"): _*)
scala> df.show(3, 0)
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
|c0                |c1                |c2                |c3                 |c4                |c5                |c6                 |c7                |c8                |c9                |c10               |c11                |c12               |c13               |c14               |c15                |c16                |c17                |c18               |c19               |c20                |c21               |c22                |c23               |c24                |c25                |c26                |c27                 |c28                |c29               |c30                |c31                 |c32               |c33               |c34                |c35                |c36                |c37               |c38               |c39                |c40               |c41               |c42                |c43                |c44                |c45               |c46                 |c47                 |c48                |c49                |c50                |c51                |c52                |c53                |c54                 |c55                |c56                |c57                |c58                |c59               |c60               |c61                |c62                |c63               |c64                |c65               |c66               |c67              |c68                |c69                |c70               |c71                |c72               |c73                |c74                |c75                |c76               |c77                |c78               |c79                |c80                |c81                |c82                |c83                |c84                |c85                |c86                |c87               |c88                |c89                |c90               |c91               |c92               |c93                |c94               |c95                |c96               |c97                |c98                |c99                |
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
|0.6306087152476858|0.9174349686288383|0.5511324165035159|0.3320844128641819 |0.7738486877101489|0.2154915886962553|0.4754997600674299 |0.922780639280355 |0.7136894772661909|0.2277580838165979|0.5926874459847249|0.40311408392226633|0.467830264333843 |0.8330466896984213|0.1893258482389527|0.6320849515511165 |0.7530911056912044 |0.06700254871955424|0.370528597355559 |0.2755437445193154|0.23704391110980128|0.8067400174905822|0.13597793616251852|0.1708888820162453|0.01672725007605702|0.983118121881555  |0.25040195628629924|0.060537253723083384|0.20000530582637488|0.3400572407133511|0.9375689433322597 |0.057039316954370256|0.8053269714347623|0.5247817572228813|0.28419308820527944|0.9798908885194533 |0.31805988175678146|0.7034448027077574|0.5400575751346084|0.25336322371116216|0.9361634546853429|0.6118681368289798|0.6295081549153907 |0.13417468943957422|0.41617137072255794|0.7267230869252035|0.023792726137561115|0.5776157058356362  |0.04884204913195467|0.26728716103441275|0.646680370807925  |0.9782712690657244 |0.16434031314818154|0.20985522381321275|0.24739842475440077 |0.26335189682977334|0.19604841662422068|0.10742950487300651|0.20283136488091502|0.3100312319723688|0.886959006630645 |0.25157102269776244|0.34428775168410786|0.3500506818575777|0.3781142441912052 |0.8560316444386715|0.4737104888956839|0.735903101602148|0.02236617130529006|0.8769074095835873 |0.2001426662503153|0.5534032319238532 |0.7289496620397098|0.41955191309992157|0.9337700133660436 |0.34059094378451005|0.6419144759403556|0.08167496930341167|0.9947099478497635|0.48010888605366586|0.22314796858167918|0.17786598882331306|0.7351521162297135 |0.5422057170020095 |0.9521927872726792 |0.7459825486368227 |0.40907708791990627|0.8903819313311575|0.7251413746923618 |0.2977174938745204 |0.9515209660203555|0.9375968604766713|0.5087851740042524|0.4255237544908751 |0.8023768698664653|0.48003189618006703|0.1775841829745185|0.09050775629268382|0.6743909291138167 |0.2498415755876865 |
|0.6866473844170801|0.4774360641212433|0.631696201340726 |0.33979113021468343|0.5663049010847052|0.7280190472258865|0.41370958502324806|0.9977433873622218|0.7671957338989901|0.2788708556233931|0.3355106391656496|0.88478952319287   |0.0333974166999893|0.6061744715862606|0.9617779139652359|0.22484954822341863|0.12770906021550898|0.5577789629508672 |0.2877649024640704|0.5566577406549361|0.9334933255278052 |0.9166720585157266|0.9689249324600591 |0.6367502457478598|0.7993572745928459 |0.23213222324218108|0.11928284054154137|0.6173493362456599  |0.0505122058694798 |0.9050228629552983|0.17112767911121707|0.47395598348370005 |0.5820498657823081|0.6241124650645072|0.18587258258036776|0.14987593554122225|0.3079446253653946 |0.9414228822867968|0.8362276265462365|0.9155655305576353 |0.5121559807153562|0.8963362656525707|0.22765970274318037|0.8177039187132797 |0.8190326635933787 |0.5256005177032199|0.8167598457269669  |0.030936807130934496|0.6733006585281015 |0.4208049626816347 |0.24603085738518538|0.22719198954208153|0.1622280557565281 |0.22217325159218038|0.014684419513742553|0.08987111517447499|0.2157764759142622 |0.8223414104088321 |0.4868624404491777 |0.4016191733088167|0.6169281906889263|0.15603611040433385|0.18289285085714913|0.9538408988218972|0.15037154865295121|0.5364516961987454|0.8077254873163031|0.712600478545675|0.7277477241003857 |0.19822912960348305|0.8305051199208777|0.18631911396566114|0.8909532487898342|0.3470409226992506 |0.35306974180587636|0.9107058868891469 |0.3321327206004986|0.48952332459050607|0.3630403307479373|0.5400046826340376 |0.5387377194310529 |0.42860539421837585|0.23214101630985995|0.21438968839794847|0.15370603160082352|0.04355605642700022|0.6096006707067466 |0.6933354157094292|0.06302172470859002|0.03174631856164001|0.664243581650643 |0.7833239547446621|0.696884598352864 |0.34626385933237736|0.9263495598791336|0.404818892816584  |0.2085585394755507|0.6150004897990109 |0.05391193524302473|0.28188484028329097|
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
only showing top 2 rows
```

`psql`, CLI for PostgreSQL, supports a vertical display mode for this case like:
http://stackoverflow.com/questions/9604723/alternate-output-format-for-psql

```
-RECORD 0-------------------
 c0  | 0.6306087152476858
 c1  | 0.9174349686288383
 c2  | 0.5511324165035159
...
 c98 | 0.05391193524302473
 c99 | 0.28188484028329097
-RECORD 1-------------------
 c0  | 0.6866473844170801
 c1  | 0.4774360641212433
 c2  | 0.631696201340726
...
 c98 | 0.05391193524302473
 c99 | 0.28188484028329097
only showing top 2 rows
```

## How was this patch tested?
Added tests in `DataFrameSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17733 from maropu/SPARK-20425.
2017-04-26 22:18:01 -07:00
Weiqing Yang 2ba1eba371 [SPARK-12868][SQL] Allow adding jars from hdfs
## What changes were proposed in this pull request?
Spark 2.2 is going to be cut, it'll be great if SPARK-12868 can be resolved before that. There have been several PRs for this like [PR#16324](https://github.com/apache/spark/pull/16324) , but all of them are inactivity for a long time or have been closed.

This PR added a SparkUrlStreamHandlerFactory, which relies on 'protocol' to choose the appropriate
UrlStreamHandlerFactory like FsUrlStreamHandlerFactory to create URLStreamHandler.

## How was this patch tested?
1. Add a new unit test.
2. Check manually.
Before: throw an exception with " failed unknown protocol: hdfs"
<img width="914" alt="screen shot 2017-03-17 at 9 07 36 pm" src="https://cloud.githubusercontent.com/assets/8546874/24075277/5abe0a7c-0bd5-11e7-900e-ec3d3105da0b.png">

After:
<img width="1148" alt="screen shot 2017-03-18 at 11 42 18 am" src="https://cloud.githubusercontent.com/assets/8546874/24075283/69382a60-0bd5-11e7-8d30-d9405c3aaaba.png">

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #17342 from weiqingy/SPARK-18910.
2017-04-26 13:54:40 -07:00
Michal Szafranski a277ae80a2 [SPARK-20474] Fixing OnHeapColumnVector reallocation
## What changes were proposed in this pull request?
OnHeapColumnVector reallocation copies to the new storage data up to 'elementsAppended'. This variable is only updated when using the ColumnVector.appendX API, while ColumnVector.putX is more commonly used.

## How was this patch tested?
Tested using existing unit tests.

Author: Michal Szafranski <michal@databricks.com>

Closes #17773 from michal-databricks/spark-20474.
2017-04-26 12:47:37 -07:00
Michal Szafranski 99c6cf9ef1 [SPARK-20473] Enabling missing types in ColumnVector.Array
## What changes were proposed in this pull request?
ColumnVector implementations originally did not support some Catalyst types (float, short, and boolean). Now that they do, those types should be also added to the ColumnVector.Array.

## How was this patch tested?
Tested using existing unit tests.

Author: Michal Szafranski <michal@databricks.com>

Closes #17772 from michal-databricks/spark-20473.
2017-04-26 11:21:25 -07:00
Eric Wasserman 57e1da3946 [SPARK-16548][SQL] Inconsistent error handling in JSON parsing SQL functions
## What changes were proposed in this pull request?

change to using Jackson's `com.fasterxml.jackson.core.JsonFactory`

    public JsonParser createParser(String content)

## How was this patch tested?

existing unit tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Eric Wasserman <ericw@sgn.com>

Closes #17693 from ewasserman/SPARK-20314.
2017-04-26 11:42:43 +08:00
Sameer Agarwal caf392025c [SPARK-18127] Add hooks and extension points to Spark
## What changes were proposed in this pull request?

This patch adds support for customizing the spark session by injecting user-defined custom extensions. This allows a user to add custom analyzer rules/checks, optimizer rules, planning strategies or even a customized parser.

## How was this patch tested?

Unit Tests in SparkSessionExtensionSuite

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17724 from sameeragarwal/session-extensions.
2017-04-25 17:05:20 -07:00
Sameer Agarwal 31345fde82 [SPARK-20451] Filter out nested mapType datatypes from sort order in randomSplit
## What changes were proposed in this pull request?

In `randomSplit`, It is possible that the underlying dataset doesn't guarantee the ordering of rows in its constituent partitions each time a split is materialized which could result in overlapping
splits.

To prevent this, as part of SPARK-12662, we explicitly sort each input partition to make the ordering deterministic. Given that `MapTypes` cannot be sorted this patch explicitly prunes them out from the sort order. Additionally, if the resulting sort order is empty, this patch then materializes the dataset to guarantee determinism.

## How was this patch tested?

Extended `randomSplit on reordered partitions` in `DataFrameStatSuite` to also test for dataframes with mapTypes nested mapTypes.

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17751 from sameeragarwal/randomsplit2.
2017-04-25 13:05:20 +08:00
Josh Rosen f44c8a843c [SPARK-20453] Bump master branch version to 2.3.0-SNAPSHOT
This patch bumps the master branch version to `2.3.0-SNAPSHOT`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17753 from JoshRosen/SPARK-20453.
2017-04-24 21:48:04 -07:00
Xiao Li 776a2c0e91 [SPARK-20439][SQL] Fix Catalog API listTables and getTable when failed to fetch table metadata
### What changes were proposed in this pull request?

`spark.catalog.listTables` and `spark.catalog.getTable` does not work if we are unable to retrieve table metadata due to any reason (e.g., table serde class is not accessible or the table type is not accepted by Spark SQL). After this PR, the APIs still return the corresponding Table without the description and tableType)

### How was this patch tested?
Added a test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17730 from gatorsmile/listTables.
2017-04-24 17:21:42 +08:00
Takeshi Yamamuro b3c572a6b3 [SPARK-20430][SQL] Initialise RangeExec parameters in a driver side
## What changes were proposed in this pull request?
This pr initialised `RangeExec` parameters in a driver side.
In the current master, a query below throws `NullPointerException`;
```
sql("SET spark.sql.codegen.wholeStage=false")
sql("SELECT * FROM range(1)").show

17/04/20 17:11:05 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.NullPointerException
        at org.apache.spark.sql.execution.SparkPlan.sparkContext(SparkPlan.scala:54)
        at org.apache.spark.sql.execution.RangeExec.numSlices(basicPhysicalOperators.scala:343)
        at org.apache.spark.sql.execution.RangeExec$$anonfun$20.apply(basicPhysicalOperators.scala:506)
        at org.apache.spark.sql.execution.RangeExec$$anonfun$20.apply(basicPhysicalOperators.scala:505)
        at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:844)
        at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:844)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
        at org.apache.spark.scheduler.Task.run(Task.scala:108)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:320)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
```

## How was this patch tested?
Added a test in `DataFrameRangeSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17717 from maropu/SPARK-20430.
2017-04-22 09:41:58 -07:00
Kazuaki Ishizaki a750a59597 [SPARK-20341][SQL] Support BigInt's value that does not fit in long value range
## What changes were proposed in this pull request?

This PR avoids an exception in the case where `scala.math.BigInt` has a value that does not fit into long value range (e.g. `Long.MAX_VALUE+1`). When we run the following code by using the current Spark, the following exception is thrown.

This PR keeps the value using `BigDecimal` if we detect such an overflow case by catching `ArithmeticException`.

Sample program:
```
case class BigIntWrapper(value:scala.math.BigInt)```
spark.createDataset(BigIntWrapper(scala.math.BigInt("10000000000000000002"))::Nil).show
```
Exception:
```
Error while encoding: java.lang.ArithmeticException: BigInteger out of long range
staticinvoke(class org.apache.spark.sql.types.Decimal$, DecimalType(38,0), apply, assertnotnull(assertnotnull(input[0, org.apache.spark.sql.BigIntWrapper, true])).value, true) AS value#0
java.lang.RuntimeException: Error while encoding: java.lang.ArithmeticException: BigInteger out of long range
staticinvoke(class org.apache.spark.sql.types.Decimal$, DecimalType(38,0), apply, assertnotnull(assertnotnull(input[0, org.apache.spark.sql.BigIntWrapper, true])).value, true) AS value#0
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:290)
	at org.apache.spark.sql.SparkSession$$anonfun$2.apply(SparkSession.scala:454)
	at org.apache.spark.sql.SparkSession$$anonfun$2.apply(SparkSession.scala:454)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.immutable.List.map(List.scala:285)
	at org.apache.spark.sql.SparkSession.createDataset(SparkSession.scala:454)
	at org.apache.spark.sql.Agg$$anonfun$18.apply$mcV$sp(MySuite.scala:192)
	at org.apache.spark.sql.Agg$$anonfun$18.apply(MySuite.scala:192)
	at org.apache.spark.sql.Agg$$anonfun$18.apply(MySuite.scala:192)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
...
Caused by: java.lang.ArithmeticException: BigInteger out of long range
	at java.math.BigInteger.longValueExact(BigInteger.java:4531)
	at org.apache.spark.sql.types.Decimal.set(Decimal.scala:140)
	at org.apache.spark.sql.types.Decimal$.apply(Decimal.scala:434)
	at org.apache.spark.sql.types.Decimal.apply(Decimal.scala)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:287)
	... 59 more
```

## How was this patch tested?

Add new test suite into `DecimalSuite`

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

Closes #17684 from kiszk/SPARK-20341.
2017-04-21 22:25:35 +08:00
Juliusz Sompolski c9e6035e1f [SPARK-20412] Throw ParseException from visitNonOptionalPartitionSpec instead of returning null values.
## What changes were proposed in this pull request?

If a partitionSpec is supposed to not contain optional values, a ParseException should be thrown, and not nulls returned.
The nulls can later cause NullPointerExceptions in places not expecting them.

## How was this patch tested?

A query like "SHOW PARTITIONS tbl PARTITION(col1='val1', col2)" used to throw a NullPointerException.
Now it throws a ParseException.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17707 from juliuszsompolski/SPARK-20412.
2017-04-21 22:11:24 +08:00
Herman van Hovell e2b3d2367a [SPARK-20420][SQL] Add events to the external catalog
## What changes were proposed in this pull request?
It is often useful to be able to track changes to the `ExternalCatalog`. This PR makes the `ExternalCatalog` emit events when a catalog object is changed. Events are fired before and after the change.

The following events are fired per object:

- Database
  - CreateDatabasePreEvent: event fired before the database is created.
  - CreateDatabaseEvent: event fired after the database has been created.
  - DropDatabasePreEvent: event fired before the database is dropped.
  - DropDatabaseEvent: event fired after the database has been dropped.
- Table
  - CreateTablePreEvent: event fired before the table is created.
  - CreateTableEvent: event fired after the table has been created.
  - RenameTablePreEvent: event fired before the table is renamed.
  - RenameTableEvent: event fired after the table has been renamed.
  - DropTablePreEvent: event fired before the table is dropped.
  - DropTableEvent: event fired after the table has been dropped.
- Function
  - CreateFunctionPreEvent: event fired before the function is created.
  - CreateFunctionEvent: event fired after the function has been created.
  - RenameFunctionPreEvent: event fired before the function is renamed.
  - RenameFunctionEvent: event fired after the function has been renamed.
  - DropFunctionPreEvent: event fired before the function is dropped.
  - DropFunctionPreEvent: event fired after the function has been dropped.

The current events currently only contain the names of the object modified. We add more events, and more details at a later point.

A user can monitor changes to the external catalog by adding a listener to the Spark listener bus checking for `ExternalCatalogEvent`s using the `SparkListener.onOtherEvent` hook. A more direct approach is add listener directly to the `ExternalCatalog`.

## How was this patch tested?
Added the `ExternalCatalogEventSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17710 from hvanhovell/SPARK-20420.
2017-04-21 00:05:03 -07:00
Takeshi Yamamuro 48d760d028 [SPARK-20281][SQL] Print the identical Range parameters of SparkContext APIs and SQL in explain
## What changes were proposed in this pull request?
This pr modified code to print the identical `Range` parameters of SparkContext APIs and SQL in `explain` output. In the current master, they internally use `defaultParallelism` for `splits` by default though, they print different strings in explain output;

```
scala> spark.range(4).explain
== Physical Plan ==
*Range (0, 4, step=1, splits=Some(8))

scala> sql("select * from range(4)").explain
== Physical Plan ==
*Range (0, 4, step=1, splits=None)
```

## How was this patch tested?
Added tests in `SQLQuerySuite` and modified some results in the existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17670 from maropu/SPARK-20281.
2017-04-20 19:40:21 -07:00
Herman van Hovell 760c8d088d [SPARK-20329][SQL] Make timezone aware expression without timezone unresolved
## What changes were proposed in this pull request?
A cast expression with a resolved time zone is not equal to a cast expression without a resolved time zone. The `ResolveAggregateFunction` assumed that these expression were the same, and would fail to resolve `HAVING` clauses which contain a `Cast` expression.

This is in essence caused by the fact that a `TimeZoneAwareExpression` can be resolved without a set time zone. This PR fixes this, and makes a `TimeZoneAwareExpression` unresolved as long as it has no TimeZone set.

## How was this patch tested?
Added a regression test to the `SQLQueryTestSuite.having` file.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17641 from hvanhovell/SPARK-20329.
2017-04-21 10:06:12 +08:00
Juliusz Sompolski 0368eb9d86 [SPARK-20367] Properly unescape column names of partitioning columns parsed from paths.
## What changes were proposed in this pull request?

When infering partitioning schema from paths, the column in parsePartitionColumn should be unescaped with unescapePathName, just like it is being done in e.g. parsePathFragmentAsSeq.

## How was this patch tested?

Added a test to FileIndexSuite.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17703 from juliuszsompolski/SPARK-20367.
2017-04-21 09:49:42 +08:00
Herman van Hovell 0332063553 [SPARK-20410][SQL] Make sparkConf a def in SharedSQLContext
## What changes were proposed in this pull request?
It is kind of annoying that `SharedSQLContext.sparkConf` is a val when overriding test cases, because you cannot call `super` on it. This PR makes it a function.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17705 from hvanhovell/SPARK-20410.
2017-04-20 22:37:04 +02:00
Dilip Biswal d95e4d9d6a [SPARK-20334][SQL] Return a better error message when correlated predicates contain aggregate expression that has mixture of outer and local references.
## What changes were proposed in this pull request?
Address a follow up in [comment](https://github.com/apache/spark/pull/16954#discussion_r105718880)
Currently subqueries with correlated predicates containing aggregate expression having mixture of outer references and local references generate a codegen error like following :

```SQL
SELECT t1a
FROM   t1
GROUP  BY 1
HAVING EXISTS (SELECT 1
               FROM  t2
               WHERE t2a < min(t1a + t2a));
```
Exception snippet.
```
Cannot evaluate expression: min((input[0, int, false] + input[4, int, false]))
	at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.doGenCode(Expression.scala:226)
	at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:87)
	at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:106)
	at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:103)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:103)

```
After this PR, a better error message is issued.
```
org.apache.spark.sql.AnalysisException
Error in query: Found an aggregate expression in a correlated
predicate that has both outer and local references, which is not supported yet.
Aggregate expression: min((t1.`t1a` + t2.`t2a`)),
Outer references: t1.`t1a`,
Local references: t2.`t2a`.;
```
## How was this patch tested?
Added tests in SQLQueryTestSuite.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #17636 from dilipbiswal/subquery_followup1.
2017-04-20 22:35:48 +02:00
Bogdan Raducanu c5a31d160f [SPARK-20407][TESTS] ParquetQuerySuite 'Enabling/disabling ignoreCorruptFiles' flaky test
## What changes were proposed in this pull request?

SharedSQLContext.afterEach now calls DebugFilesystem.assertNoOpenStreams inside eventually.
SQLTestUtils withTempDir calls waitForTasksToFinish before deleting the directory.

## How was this patch tested?
Added new test in ParquetQuerySuite based on the flaky test

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #17701 from bogdanrdc/SPARK-20407.
2017-04-20 18:49:39 +02:00
Wenchen Fan b91873db09 [SPARK-20409][SQL] fail early if aggregate function in GROUP BY
## What changes were proposed in this pull request?

It's illegal to have aggregate function in GROUP BY, and we should fail at analysis phase, if this happens.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17704 from cloud-fan/minor.
2017-04-20 16:59:38 +02:00
Reynold Xin c6f62c5b81 [SPARK-20405][SQL] Dataset.withNewExecutionId should be private
## What changes were proposed in this pull request?
Dataset.withNewExecutionId is only used in Dataset itself and should be private.

## How was this patch tested?
N/A - this is a simple visibility change.

Author: Reynold Xin <rxin@databricks.com>

Closes #17699 from rxin/SPARK-20405.
2017-04-20 14:29:59 +02:00