Commit graph

2987 commits

Author SHA1 Message Date
Liang-Chi Hsieh 38e7835347 [SPARK-19736][SQL] refreshByPath should clear all cached plans with the specified path
## What changes were proposed in this pull request?

`Catalog.refreshByPath` can refresh the cache entry and the associated metadata for all dataframes (if any), that contain the given data source path.

However, `CacheManager.invalidateCachedPath` doesn't clear all cached plans with the specified path. It causes some strange behaviors reported in SPARK-15678.

## 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 #17064 from viirya/fix-refreshByPath.
2017-03-01 00:19:57 -08:00
Liwei Lin 4913c92c2f [SPARK-19633][SS] FileSource read from FileSink
## What changes were proposed in this pull request?

Right now file source always uses `InMemoryFileIndex` to scan files from a given path.

But when reading the outputs from another streaming query, the file source should use `MetadataFileIndex` to list files from the sink log. This patch adds this support.

## `MetadataFileIndex` or `InMemoryFileIndex`
```scala
spark
  .readStream
  .format(...)
  .load("/some/path") // for a non-glob path:
                      //   - use `MetadataFileIndex` when `/some/path/_spark_meta` exists
                      //   - fall back to `InMemoryFileIndex` otherwise
```
```scala
spark
  .readStream
  .format(...)
  .load("/some/path/*/*") // for a glob path: always use `InMemoryFileIndex`
```

## How was this patch tested?

two newly added tests

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16987 from lw-lin/source-read-from-sink.
2017-02-28 22:58:51 -08:00
Jeff Zhang 7315880568 [SPARK-19572][SPARKR] Allow to disable hive in sparkR shell
## What changes were proposed in this pull request?
SPARK-15236 do this for scala shell, this ticket is for sparkR shell. This is not only for sparkR itself, but can also benefit downstream project like livy which use shell.R for its interactive session. For now, livy has no control of whether enable hive or not.

## How was this patch tested?

Tested it manually, run `bin/sparkR --master local --conf spark.sql.catalogImplementation=in-memory` and verify hive is not enabled.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #16907 from zjffdu/SPARK-19572.
2017-02-28 22:21:29 -08:00
hyukjinkwon 7e5359be5c [SPARK-19610][SQL] Support parsing multiline CSV files
## What changes were proposed in this pull request?

This PR proposes the support for multiple lines for CSV by resembling the multiline supports in JSON datasource (in case of JSON, per file).

So, this PR introduces `wholeFile` option which makes the format not splittable and reads each whole file. Since Univocity parser can produces each row from a stream, it should be capable of parsing very large documents when the internal rows are fix in the memory.

## How was this patch tested?

Unit tests in `CSVSuite` and `tests.py`

Manual tests with a single 9GB CSV file in local file system, for example,

```scala
spark.read.option("wholeFile", true).option("inferSchema", true).csv("tmp.csv").count()
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16976 from HyukjinKwon/SPARK-19610.
2017-02-28 13:34:33 -08:00
windpiger ce233f18e3 [SPARK-19463][SQL] refresh cache after the InsertIntoHadoopFsRelationCommand
## What changes were proposed in this pull request?

If we first cache a DataSource table, then we insert some data into the table, we should refresh the data in the cache after the insert command.

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16809 from windpiger/refreshCacheAfterInsert.
2017-02-28 11:59:18 -08:00
Roberto Agostino Vitillo 9734a928a7 [SPARK-19677][SS] Committing a delta file atop an existing one should not fail on HDFS
## What changes were proposed in this pull request?

HDFSBackedStateStoreProvider fails to rename files on HDFS but not on the local filesystem. According to the [implementation notes](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/filesystem/filesystem.html) of `rename()`, the behavior of the local filesystem and HDFS varies:

> Destination exists and is a file
> Renaming a file atop an existing file is specified as failing, raising an exception.
>    - Local FileSystem : the rename succeeds; the destination file is replaced by the source file.
>    - HDFS : The rename fails, no exception is raised. Instead the method call simply returns false.

This patch ensures that `rename()` isn't called if the destination file already exists. It's still semantically correct because Structured Streaming requires that rerunning a batch should generate the same output.

## How was this patch tested?

This patch was tested by running `StateStoreSuite`.

Author: Roberto Agostino Vitillo <ra.vitillo@gmail.com>

Closes #17012 from vitillo/fix_rename.
2017-02-28 10:49:07 -08:00
Wenchen Fan 7c7fc30b4a [SPARK-19678][SQL] remove MetastoreRelation
## What changes were proposed in this pull request?

`MetastoreRelation` is used to represent table relation for hive tables, and provides some hive related information. We will resolve `SimpleCatalogRelation` to `MetastoreRelation` for hive tables, which is unnecessary as these 2 are the same essentially. This PR merges `SimpleCatalogRelation` and `MetastoreRelation`

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17015 from cloud-fan/table-relation.
2017-02-28 09:24:36 -08:00
Yuming Wang 9b8eca65dc [SPARK-19660][CORE][SQL] Replace the configuration property names that are deprecated in the version of Hadoop 2.6
## What changes were proposed in this pull request?

Replace all the Hadoop deprecated configuration property names according to [DeprecatedProperties](https://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-common/DeprecatedProperties.html).

except:
https://github.com/apache/spark/blob/v2.1.0/python/pyspark/sql/tests.py#L1533
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala#L987
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala#L45
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L614

## How was this patch tested?

Existing tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16990 from wangyum/HadoopDeprecatedProperties.
2017-02-28 10:13:42 +00:00
windpiger a350bc16d3 [SPARK-19748][SQL] refresh function has a wrong order to do cache invalidate and regenerate the inmemory var for InMemoryFileIndex with FileStatusCache
## What changes were proposed in this pull request?

If we refresh a InMemoryFileIndex with a FileStatusCache, it will first use the FileStatusCache to re-generate the cachedLeafFiles etc, then call FileStatusCache.invalidateAll.

While the order to do these two actions is wrong, this lead to the refresh action does not take effect.

```
  override def refresh(): Unit = {
    refresh0()
    fileStatusCache.invalidateAll()
  }

  private def refresh0(): Unit = {
    val files = listLeafFiles(rootPaths)
    cachedLeafFiles =
      new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f)
    cachedLeafDirToChildrenFiles = files.toArray.groupBy(_.getPath.getParent)
    cachedPartitionSpec = null
  }
```
## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #17079 from windpiger/fixInMemoryFileIndexRefresh.
2017-02-28 00:16:49 -08:00
uncleGen 7353038353 [SPARK-19749][SS] Name socket source with a meaningful name
## What changes were proposed in this pull request?

Name socket source with a meaningful name

## How was this patch tested?

Jenkins

Author: uncleGen <hustyugm@gmail.com>

Closes #17082 from uncleGen/SPARK-19749.
2017-02-27 18:02:45 -08:00
Eyal Zituny 9f8e392159 [SPARK-19594][STRUCTURED STREAMING] StreamingQueryListener fails to handle QueryTerminatedEvent if more then one listeners exists
## What changes were proposed in this pull request?

currently if multiple streaming queries listeners exists, when a QueryTerminatedEvent is triggered, only one of the listeners will be invoked while the rest of the listeners will ignore the event.
this is caused since the the streaming queries listeners bus holds a set of running queries ids and when a termination event is triggered, after the first listeners is handling the event, the terminated query id is being removed from the set.
in this PR, the query id will be removed from the set only after all the listeners handles the event

## How was this patch tested?

a test with multiple listeners has been added to StreamingQueryListenerSuite

Author: Eyal Zituny <eyal.zituny@equalum.io>

Closes #16991 from eyalzit/master.
2017-02-26 15:57:32 -08:00
Herman van Hovell 8f0511ed49 [SPARK-19650] Commands should not trigger a Spark job
Spark executes SQL commands eagerly. It does this by creating an RDD which contains the command's results. The downside to this is that any action on this RDD triggers a Spark job which is expensive and is unnecessary.

This PR fixes this by avoiding the materialization of an `RDD` for `Command`s; it just materializes the result and puts them in a `LocalRelation`.

Added a regression test to `SQLQuerySuite`.

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

Closes #17027 from hvanhovell/no-job-command.
2017-02-24 23:05:59 -08:00
Xiao Li 4cb025afaf [SPARK-19735][SQL] Remove HOLD_DDLTIME from Catalog APIs
### What changes were proposed in this pull request?
As explained in Hive JIRA https://issues.apache.org/jira/browse/HIVE-12224, HOLD_DDLTIME was broken as soon as it landed. Hive 2.0 removes HOLD_DDLTIME from the API. In Spark SQL, we always set it to FALSE. Like Hive, we should also remove it from our Catalog APIs.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17063 from gatorsmile/removalHoldDDLTime.
2017-02-24 23:03:59 -08:00
wangzhenhua 69d0da6373 [SPARK-17078][SQL] Show stats when explain
## What changes were proposed in this pull request?

Currently we can only check the estimated stats in logical plans by debugging. We need to provide an easier and more efficient way for developers/users.

In this pr, we add EXPLAIN COST command to show stats in the optimized logical plan.
E.g.
```
spark-sql> EXPLAIN COST select count(1) from store_returns;

...
== Optimized Logical Plan ==
Aggregate [count(1) AS count(1)#24L], Statistics(sizeInBytes=16.0 B, rowCount=1, isBroadcastable=false)
+- Project, Statistics(sizeInBytes=4.3 GB, rowCount=5.76E+8, isBroadcastable=false)
   +- Relation[sr_returned_date_sk#3,sr_return_time_sk#4,sr_item_sk#5,sr_customer_sk#6,sr_cdemo_sk#7,sr_hdemo_sk#8,sr_addr_sk#9,sr_store_sk#10,sr_reason_sk#11,sr_ticket_number#12,sr_return_quantity#13,sr_return_amt#14,sr_return_tax#15,sr_return_amt_inc_tax#16,sr_fee#17,sr_return_ship_cost#18,sr_refunded_cash#19,sr_reversed_charge#20,sr_store_credit#21,sr_net_loss#22] parquet, Statistics(sizeInBytes=28.6 GB, rowCount=5.76E+8, isBroadcastable=false)
...
```

## How was this patch tested?

Add test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16594 from wzhfy/showStats.
2017-02-24 10:24:59 -08:00
windpiger 8f33731e79 [SPARK-19664][SQL] put hive.metastore.warehouse.dir in hadoopconf to overwrite its original value
## What changes were proposed in this pull request?

In [SPARK-15959](https://issues.apache.org/jira/browse/SPARK-15959), we bring back the `hive.metastore.warehouse.dir` , while in the logic, when use the value of  `spark.sql.warehouse.dir` to overwrite `hive.metastore.warehouse.dir` , it set it to `sparkContext.conf` which does not overwrite the value is hadoopConf, I think it should put in `sparkContext.hadoopConfiguration` and overwrite the original value of hadoopConf

https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala#L64

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #16996 from windpiger/hivemetawarehouseConf.
2017-02-23 22:57:23 -08:00
Carson Wang eff7b40890 [SPARK-19674][SQL] Ignore driver accumulator updates don't belong to the execution when merging all accumulator updates
## What changes were proposed in this pull request?
In SQLListener.getExecutionMetrics, driver accumulator updates don't belong to the execution should be ignored when merging all accumulator updates to prevent NoSuchElementException.

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

Author: Carson Wang <carson.wang@intel.com>

Closes #17009 from carsonwang/FixSQLMetrics.
2017-02-23 14:31:16 -08:00
Takeshi Yamamuro 09ed6e7711 [SPARK-18699][SQL] Put malformed tokens into a new field when parsing CSV data
## What changes were proposed in this pull request?
This pr added a logic to put malformed tokens into a new field when parsing CSV data  in case of permissive modes. In the current master, if the CSV parser hits these malformed ones, it throws an exception below (and then a job fails);
```
Caused by: java.lang.IllegalArgumentException
	at java.sql.Date.valueOf(Date.java:143)
	at org.apache.spark.sql.catalyst.util.DateTimeUtils$.stringToTime(DateTimeUtils.scala:137)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply$mcJ$sp(CSVInferSchema.scala:272)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272)
	at scala.util.Try.getOrElse(Try.scala:79)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:269)
	at
```
In case that users load large CSV-formatted data, the job failure makes users get some confused. So, this fix set NULL for original columns and put malformed tokens in a new field.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #16928 from maropu/SPARK-18699-2.
2017-02-23 12:09:36 -08:00
Shixiong Zhu 9bf4e2baad [SPARK-19497][SS] Implement streaming deduplication
## What changes were proposed in this pull request?

This PR adds a special streaming deduplication operator to support `dropDuplicates` with `aggregation` and watermark. It reuses the `dropDuplicates` API but creates new logical plan `Deduplication` and new physical plan `DeduplicationExec`.

The following cases are supported:

- one or multiple `dropDuplicates()` without aggregation (with or without watermark)
- `dropDuplicates` before aggregation

Not supported cases:

- `dropDuplicates` after aggregation

Breaking changes:
- `dropDuplicates` without aggregation doesn't work with `complete` or `update` mode.

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16970 from zsxwing/dedup.
2017-02-23 11:25:39 -08:00
Takeshi Yamamuro 769aa0f1d2 [SPARK-19695][SQL] Throw an exception if a columnNameOfCorruptRecord field violates requirements in json formats
## What changes were proposed in this pull request?
This pr comes from #16928 and fixed a json behaviour along with the CSV one.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17023 from maropu/SPARK-19695.
2017-02-22 21:39:20 -08:00
pj.fanning d3147502e7 [SPARK-15615][SQL] Add an API to load DataFrame from Dataset[String] storing JSON
## What changes were proposed in this pull request?

SPARK-15615 proposes replacing the sqlContext.read.json(rdd) with a dataset equivalent.
SPARK-15463 adds a CSV API for reading from Dataset[String] so this keeps the API consistent.
I am deprecating the existing RDD based APIs.

## How was this patch tested?

There are existing tests. I left most tests to use the existing APIs as they delegate to the new json API.

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

Author: pj.fanning <pj.fanning@workday.com>
Author: PJ Fanning <pjfanning@users.noreply.github.com>

Closes #16895 from pjfanning/SPARK-15615.
2017-02-22 18:03:25 -08:00
Xiao Li dc005ed53c [SPARK-19658][SQL] Set NumPartitions of RepartitionByExpression In Parser
### What changes were proposed in this pull request?

Currently, if `NumPartitions` is not set in RepartitionByExpression, we will set it using `spark.sql.shuffle.partitions` during Planner. However, this is not following the general resolution process. This PR is to set it in `Parser` and then `Optimizer` can use the value for plan optimization.

### How was this patch tested?

Added a test case.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16988 from gatorsmile/resolveRepartition.
2017-02-22 17:26:56 -08:00
hyukjinkwon 37112fcfcd [SPARK-19666][SQL] Skip a property without getter in Java schema inference and allow empty bean in encoder creation
## What changes were proposed in this pull request?

This PR proposes to fix two.

**Skip a property without a getter in beans**

Currently, if we use a JavaBean without the getter as below:

```java
public static class BeanWithoutGetter implements Serializable {
  private String a;

  public void setA(String a) {
    this.a = a;
  }
}

BeanWithoutGetter bean = new BeanWithoutGetter();
List<BeanWithoutGetter> data = Arrays.asList(bean);
spark.createDataFrame(data, BeanWithoutGetter.class).show();
```

- Before

It throws an exception as below:

```
java.lang.NullPointerException
	at org.spark_project.guava.reflect.TypeToken.method(TypeToken.java:465)
	at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:126)
	at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:125)
```

- After

```
++
||
++
||
++
```

**Supports empty bean in encoder creation**

```java
public static class EmptyBean implements Serializable {}

EmptyBean bean = new EmptyBean();
List<EmptyBean> data = Arrays.asList(bean);
spark.createDataset(data, Encoders.bean(EmptyBean.class)).show();
```

- Before

throws an exception as below:

```
java.lang.UnsupportedOperationException: Cannot infer type for class EmptyBean because it is not bean-compliant
	at org.apache.spark.sql.catalyst.JavaTypeInference$.org$apache$spark$sql$catalyst$JavaTypeInference$$serializerFor(JavaTypeInference.scala:436)
	at org.apache.spark.sql.catalyst.JavaTypeInference$.serializerFor(JavaTypeInference.scala:341)
```

- After

```
++
||
++
||
++
```

## How was this patch tested?

Unit test in `JavaDataFrameSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17013 from HyukjinKwon/SPARK-19666.
2017-02-22 12:42:23 -08:00
Reynold Xin 0733a54a45 [SPARK-19669][SQL] Open up visibility for sharedState, sessionState, and a few other functions
## What changes were proposed in this pull request?
To ease debugging, most of Spark SQL internals have public level visibility. Two of the most important internal states, sharedState and sessionState, however, are package private. It would make more sense to open these up as well with clear documentation that they are internal.

In addition, users currently have way to set active/default SparkSession, but no way to actually get them back. We should open those up as well.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #17002 from rxin/SPARK-19669.
2017-02-20 12:21:07 -08:00
Wenchen Fan 776b8f17cf [SPARK-19563][SQL] avoid unnecessary sort in FileFormatWriter
## What changes were proposed in this pull request?

In `FileFormatWriter`, we will sort the input rows by partition columns and bucket id and sort columns, if we want to write data out partitioned or bucketed.

However, if the data is already sorted, we will sort it again, which is unnecssary.

This PR removes the sorting logic in `FileFormatWriter` and use `SortExec` instead. We will not add `SortExec` if the data is already sorted.

## How was this patch tested?

I did a micro benchmark manually
```
val df = spark.range(10000000).select($"id", $"id" % 10 as "part").sort("part")
spark.time(df.write.partitionBy("part").parquet("/tmp/test"))
```
The result was about 6.4 seconds before this PR, and is 5.7 seconds afterwards.

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16898 from cloud-fan/writer.
2017-02-19 18:13:12 -08:00
windpiger 65fe902e13 [SPARK-19598][SQL] Remove the alias parameter in UnresolvedRelation
## What changes were proposed in this pull request?

Remove the alias parameter in `UnresolvedRelation`, and use `SubqueryAlias` to replace it.
This can simplify some `match case` situations.

For example, the broadcast hint pull request can have one fewer case https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala#L57-L61

## How was this patch tested?
add some unit tests

Author: windpiger <songjun@outlook.com>

Closes #16956 from windpiger/removeUnresolveTableAlias.
2017-02-19 16:50:16 -08:00
Sean Owen 1487c9af20
[SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features
## What changes were proposed in this pull request?

Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code.

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16964 from srowen/SPARK-19534.
2017-02-19 09:42:50 -08:00
jinxing ba8912e5f3
[SPARK-19450] Replace askWithRetry with askSync.
## What changes were proposed in this pull request?

`askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated.
As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218):

>askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it.

Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`.

## How was this patch tested?
This PR doesn't change code logic, existing unit test can cover.

Author: jinxing <jinxing@meituan.com>

Closes #16790 from jinxing64/SPARK-19450.
2017-02-19 04:34:07 -08:00
Ala Luszczak b486ffc86d [SPARK-19447] Make Range operator generate "recordsRead" metric
## What changes were proposed in this pull request?

The Range was modified to produce "recordsRead" metric instead of "generated rows". The tests were updated and partially moved to SQLMetricsSuite.

## How was this patch tested?

Unit tests.

Author: Ala Luszczak <ala@databricks.com>

Closes #16960 from ala/range-records-read.
2017-02-18 07:51:41 -08:00
Shixiong Zhu 15b144d2bf [SPARK-19617][SS] Fix the race condition when starting and stopping a query quickly
## What changes were proposed in this pull request?

The streaming thread in StreamExecution uses the following ways to check if it should exit:
- Catch an InterruptException.
- `StreamExecution.state` is TERMINATED.

When starting and stopping a query quickly, the above two checks may both fail:
- Hit [HADOOP-14084](https://issues.apache.org/jira/browse/HADOOP-14084) and swallow InterruptException
- StreamExecution.stop is called before `state` becomes `ACTIVE`. Then [runBatches](dcc2d540a5/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala (L252)) changes the state from `TERMINATED` to `ACTIVE`.

If the above cases both happen, the query will hang forever.

This PR changes `state` to `AtomicReference` and uses`compareAndSet` to make sure we only change the state from `INITIALIZING` to `ACTIVE`. It also removes the `runUninterruptibly` hack from ``HDFSMetadata`, because HADOOP-14084 won't cause any problem after we fix the race condition.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16947 from zsxwing/SPARK-19617.
2017-02-17 19:04:45 -08:00
Wenchen Fan 54d23599df [SPARK-18120][SPARK-19557][SQL] Call QueryExecutionListener callback methods for DataFrameWriter methods
## What changes were proposed in this pull request?

We only notify `QueryExecutionListener` for several `Dataset` operations, e.g. collect, take, etc. We should also do the notification for `DataFrameWriter` operations.

## How was this patch tested?

new regression test

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16962 from cloud-fan/insert.
2017-02-16 21:09:14 -08:00
Nathan Howell 21fde57f15 [SPARK-18352][SQL] Support parsing multiline json files
## What changes were proposed in this pull request?

If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory.

Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired.

These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing.

I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one.

## How was this patch tested?

New and existing unit tests. No performance or load tests have been run.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16386 from NathanHowell/SPARK-18352.
2017-02-16 20:51:19 -08:00
Zheng RuiFeng 54a30c8a70 [SPARK-19436][SQL] Add missing tests for approxQuantile
## What changes were proposed in this pull request?
1, check the behavior with illegal `quantiles` and `relativeError`
2, add tests for `relativeError` > 1
3, update tests for `null` data
4, update some docs for javadoc8

## How was this patch tested?
local test in spark-shell

Author: Zheng RuiFeng <ruifengz@foxmail.com>
Author: Ruifeng Zheng <ruifengz@foxmail.com>

Closes #16776 from zhengruifeng/fix_approxQuantile.
2017-02-16 09:42:13 -08:00
hyukjinkwon 3b4376876f
[MINOR][BUILD] Fix javadoc8 break
## What changes were proposed in this pull request?

These error below seems caused by unidoc that does not understand double commented block.

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:69: error: class, interface, or enum expected
[error]  * MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt; mappingFunction =
[error]                                  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:69: error: class, interface, or enum expected
[error]  * MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt; mappingFunction =
[error]                                                                       ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:70: error: class, interface, or enum expected
[error]  *    new MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt;() {
[error]                                         ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:70: error: class, interface, or enum expected
[error]  *    new MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt;() {
[error]                                                                             ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:72: error: illegal character: '#'
[error]  *      &#64;Override
[error]          ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:72: error: class, interface, or enum expected
[error]  *      &#64;Override
[error]              ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                                                    ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                                                                ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                                                                                     ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected
[error]  *      public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) {
[error]                                                                                                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:76: error: class, interface, or enum expected
[error]  *          boolean shouldRemove = ...; // Decide whether to remove the state
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:77: error: class, interface, or enum expected
[error]  *          if (shouldRemove) {
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:79: error: class, interface, or enum expected
[error]  *          } else {
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:81: error: class, interface, or enum expected
[error]  *            state.update(newState); // Set the new state
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:82: error: class, interface, or enum expected
[error]  *          }
[error]  ^
[error] .../forked/spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:85: error: class, interface, or enum expected
[error]  *          state.update(initialState);
[error]  ^
[error] .../forked/spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:86: error: class, interface, or enum expected
[error]  *        }
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:90: error: class, interface, or enum expected
[error]  * </code></pre>
[error]  ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:92: error: class, interface, or enum expected
[error]  * tparam S User-defined type of the state to be stored for each key. Must be encodable into
[error]            ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:93: error: class, interface, or enum expected
[error]  *           Spark SQL types (see {link Encoder} for more details).
[error]                                          ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:94: error: class, interface, or enum expected
[error]  * since 2.1.1
[error]           ^
```

And another link seems unrecognisable.

```
.../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:16: error: reference not found
[error]  * That is, in every batch of the {link streaming.StreamingQuery StreamingQuery},
[error]
```

Note that this PR does not fix the two breaks as below:

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:43: error: unexpected content
[error]    * see {link DataFrameStatsFunctions.approxQuantile(col:Str* approxQuantile} for
[error]      ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:52: error: bad use of '>'
[error]    * param relativeError The relative target precision to achieve (>= 0).
[error]                                                                     ^
[error]
```

because these seem probably fixed soon in https://github.com/apache/spark/pull/16776 and I intended to avoid potential conflicts.

## How was this patch tested?

Manually via `jekyll build`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16926 from HyukjinKwon/javadoc-break.
2017-02-16 12:35:43 +00:00
Sean Owen 0e2405490f
[SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support
- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings

For the future:

- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16871 from srowen/SPARK-19493.
2017-02-16 12:32:45 +00:00
Tejas Patil f041e55eef [SPARK-19618][SQL] Inconsistency wrt max. buckets allowed from Dataframe API vs SQL
## What changes were proposed in this pull request?

Jira: https://issues.apache.org/jira/browse/SPARK-19618

Moved the check for validating number of buckets from `DataFrameWriter` to `BucketSpec` creation

## How was this patch tested?

- Added more unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #16948 from tejasapatil/SPARK-19618_max_buckets.
2017-02-15 22:45:58 -08:00
Shixiong Zhu fc02ef95cd [SPARK-19603][SS] Fix StreamingQuery explain command
## What changes were proposed in this pull request?

`StreamingQuery.explain` doesn't show the correct streaming physical plan right now because `ExplainCommand` receives a runtime batch plan and its `logicalPlan.isStreaming` is always false.

This PR adds `streaming` parameter to `ExplainCommand` to allow `StreamExecution` to specify that it's a streaming plan.

Examples of the explain outputs:

- streaming DataFrame.explain()
```
== Physical Plan ==
*HashAggregate(keys=[value#518], functions=[count(1)])
+- StateStoreSave [value#518], OperatorStateId(<unknown>,0,0), Append, 0
   +- *HashAggregate(keys=[value#518], functions=[merge_count(1)])
      +- StateStoreRestore [value#518], OperatorStateId(<unknown>,0,0)
         +- *HashAggregate(keys=[value#518], functions=[merge_count(1)])
            +- Exchange hashpartitioning(value#518, 5)
               +- *HashAggregate(keys=[value#518], functions=[partial_count(1)])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
                     +- *MapElements <function1>, obj#517: java.lang.String
                        +- *DeserializeToObject value#513.toString, obj#516: java.lang.String
                           +- StreamingRelation MemoryStream[value#513], [value#513]
```

- StreamingQuery.explain(extended = false)
```
== Physical Plan ==
*HashAggregate(keys=[value#518], functions=[count(1)])
+- StateStoreSave [value#518], OperatorStateId(...,0,0), Complete, 0
   +- *HashAggregate(keys=[value#518], functions=[merge_count(1)])
      +- StateStoreRestore [value#518], OperatorStateId(...,0,0)
         +- *HashAggregate(keys=[value#518], functions=[merge_count(1)])
            +- Exchange hashpartitioning(value#518, 5)
               +- *HashAggregate(keys=[value#518], functions=[partial_count(1)])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
                     +- *MapElements <function1>, obj#517: java.lang.String
                        +- *DeserializeToObject value#543.toString, obj#516: java.lang.String
                           +- LocalTableScan [value#543]
```

- StreamingQuery.explain(extended = true)
```
== Parsed Logical Plan ==
Aggregate [value#518], [value#518, count(1) AS count(1)#524L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String
      +- DeserializeToObject cast(value#543 as string).toString, obj#516: java.lang.String
         +- LocalRelation [value#543]

== Analyzed Logical Plan ==
value: string, count(1): bigint
Aggregate [value#518], [value#518, count(1) AS count(1)#524L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String
      +- DeserializeToObject cast(value#543 as string).toString, obj#516: java.lang.String
         +- LocalRelation [value#543]

== Optimized Logical Plan ==
Aggregate [value#518], [value#518, count(1) AS count(1)#524L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String
      +- DeserializeToObject value#543.toString, obj#516: java.lang.String
         +- LocalRelation [value#543]

== Physical Plan ==
*HashAggregate(keys=[value#518], functions=[count(1)], output=[value#518, count(1)#524L])
+- StateStoreSave [value#518], OperatorStateId(...,0,0), Complete, 0
   +- *HashAggregate(keys=[value#518], functions=[merge_count(1)], output=[value#518, count#530L])
      +- StateStoreRestore [value#518], OperatorStateId(...,0,0)
         +- *HashAggregate(keys=[value#518], functions=[merge_count(1)], output=[value#518, count#530L])
            +- Exchange hashpartitioning(value#518, 5)
               +- *HashAggregate(keys=[value#518], functions=[partial_count(1)], output=[value#518, count#530L])
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518]
                     +- *MapElements <function1>, obj#517: java.lang.String
                        +- *DeserializeToObject value#543.toString, obj#516: java.lang.String
                           +- LocalTableScan [value#543]
```

## How was this patch tested?

The updated unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16934 from zsxwing/SPARK-19603.
2017-02-15 20:51:33 -08:00
Shixiong Zhu 21b4ba2d6f [SPARK-19599][SS] Clean up HDFSMetadataLog
## What changes were proposed in this pull request?

SPARK-19464 removed support for Hadoop 2.5 and earlier, so we can do some cleanup for HDFSMetadataLog.

This PR includes the following changes:
- ~~Remove the workaround codes for HADOOP-10622.~~ Unfortunately, there is another issue [HADOOP-14084](https://issues.apache.org/jira/browse/HADOOP-14084) that prevents us from removing the workaround codes.
- Remove unnecessary `writer: (T, OutputStream) => Unit` and just call `serialize` directly.
- Remove catching FileNotFoundException.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16932 from zsxwing/metadata-cleanup.
2017-02-15 16:21:43 -08:00
Takuya UESHIN 865b2fd84c [SPARK-18937][SQL] Timezone support in CSV/JSON parsing
## What changes were proposed in this pull request?

This is a follow-up pr of #16308.

This pr enables timezone support in CSV/JSON parsing.

We should introduce `timeZone` option for CSV/JSON datasources (the default value of the option is session local timezone).

The datasources should use the `timeZone` option to format/parse to write/read timestamp values.
Notice that while reading, if the timestampFormat has the timezone info, the timezone will not be used because we should respect the timezone in the values.

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values written with the default timezone option, which is `"GMT"` because session local timezone is `"GMT"` here, are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")

scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+

scala> df.write.json("/path/to/gmtjson")
```

```sh
$ cat /path/to/gmtjson/part-*
{"ts":"2016-01-01T00:00:00.000Z"}
```

whereas setting the option to `"PST"`, they are:

```scala
scala> df.write.option("timeZone", "PST").json("/path/to/pstjson")
```

```sh
$ cat /path/to/pstjson/part-*
{"ts":"2015-12-31T16:00:00.000-08:00"}
```

We can properly read these files even if the timezone option is wrong because the timestamp values have timezone info:

```scala
scala> val schema = new StructType().add("ts", TimestampType)
schema: org.apache.spark.sql.types.StructType = StructType(StructField(ts,TimestampType,true))

scala> spark.read.schema(schema).json("/path/to/gmtjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+

scala> spark.read.schema(schema).option("timeZone", "PST").json("/path/to/gmtjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```

And even if `timezoneFormat` doesn't contain timezone info, we can properly read the values with setting correct timezone option:

```scala
scala> df.write.option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson")
```

```sh
$ cat /path/to/jstjson/part-*
{"ts":"2016-01-01T09:00:00"}
```

```scala
// wrong result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").json("/path/to/jstjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 09:00:00|
+-------------------+

// correct result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```

This pr also makes `JsonToStruct` and `StructToJson` `TimeZoneAwareExpression` to be able to evaluate values with timezone option.

## How was this patch tested?

Existing tests and added some tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16750 from ueshin/issues/SPARK-18937.
2017-02-15 13:26:34 -08:00
windpiger 6a9a85b84d [SPARK-19329][SQL] Reading from or writing to a datasource table with a non pre-existing location should succeed
## What changes were proposed in this pull request?

when we insert data into a datasource table use `sqlText`, and the table has an not exists location,
this will throw an Exception.

example:

```
spark.sql("create table t(a string, b int) using parquet")
spark.sql("alter table t set location '/xx'")
spark.sql("insert into table t select 'c', 1")
```

Exception:
```
com.google.common.util.concurrent.UncheckedExecutionException: org.apache.spark.sql.AnalysisException: Path does not exist: /xx;
at com.google.common.cache.LocalCache$LocalLoadingCache.getUnchecked(LocalCache.java:4814)
at com.google.common.cache.LocalCache$LocalLoadingCache.apply(LocalCache.java:4830)
at org.apache.spark.sql.hive.HiveMetastoreCatalog.lookupRelation(HiveMetastoreCatalog.scala:122)
at org.apache.spark.sql.hive.HiveSessionCatalog.lookupRelation(HiveSessionCatalog.scala:69)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveRelations$$lookupTableFromCatalog(Analyzer.scala:456)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$8.applyOrElse(Analyzer.scala:465)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$8.applyOrElse(Analyzer.scala:463)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:60)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:463)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:453)
```

As discussed following comments, we should unify the action when we reading from or writing to a datasource table with a non pre-existing locaiton:

1. reading from a datasource table: return 0 rows
2. writing to a datasource table:  write data successfully

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16672 from windpiger/insertNotExistLocation.
2017-02-15 13:21:48 -08:00
Felix Cheung 671bc08ed5 [SPARK-19399][SPARKR] Add R coalesce API for DataFrame and Column
## What changes were proposed in this pull request?

Add coalesce on DataFrame for down partitioning without shuffle and coalesce on Column

## How was this patch tested?

manual, unit tests

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #16739 from felixcheung/rcoalesce.
2017-02-15 10:45:37 -08:00
Wenchen Fan 8b75f8c1c9 [SPARK-19587][SQL] bucket sorting columns should not be picked from partition columns
## What changes were proposed in this pull request?

We will throw an exception if bucket columns are part of partition columns, this should also apply to sort columns.

This PR also move the checking logic from `DataFrameWriter` to `PreprocessTableCreation`, which is the central place for checking and normailization.

## How was this patch tested?

updated test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16931 from cloud-fan/bucket.
2017-02-15 08:15:03 -08:00
Ala Luszczak b55563c17e [SPARK-19607] Finding QueryExecution that matches provided executionId
## What changes were proposed in this pull request?

Implementing a mapping between executionId and corresponding QueryExecution in SQLExecution.

## How was this patch tested?

Adds a unit test.

Author: Ala Luszczak <ala@databricks.com>

Closes #16940 from ala/execution-id.
2017-02-15 17:06:04 +01:00
sureshthalamati f48c5a57d6 [SPARK-19318][SQL] Fix to treat JDBC connection properties specified by the user in case-sensitive manner.
## What changes were proposed in this pull request?
The reason for test failure is that the property “oracle.jdbc.mapDateToTimestamp” set by the test was getting converted into all lower case. Oracle database expects this property in case-sensitive manner.

This test was passing in previous releases because connection properties were sent as user specified for the test case scenario. Fixes to handle all option uniformly in case-insensitive manner, converted the JDBC connection properties also to lower case.

This PR  enhances CaseInsensitiveMap to keep track of input case-sensitive keys , and uses those when creating connection properties that are passed to the JDBC connection.

Alternative approach PR https://github.com/apache/spark/pull/16847  is to pass original input keys to JDBC data source by adding check in the  Data source class and handle case-insensitivity in the JDBC source code.

## How was this patch tested?
Added new test cases to JdbcSuite , and OracleIntegrationSuite. Ran docker integration tests passed on my laptop, all tests passed successfully.

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #16891 from sureshthalamati/jdbc_case_senstivity_props_fix-SPARK-19318.
2017-02-14 15:34:12 -08:00
Xin Wu 1ab97310e8 [SPARK-19539][SQL] Block duplicate temp table during creation
## What changes were proposed in this pull request?
Current `CREATE TEMPORARY TABLE ... ` is deprecated and recommend users to use `CREATE TEMPORARY VIEW ...` And it does not support `IF NOT EXISTS `clause. However, if there is an existing temporary view defined, it is possible to unintentionally replace this existing view by issuing `CREATE TEMPORARY TABLE ...`  with the same table/view name.

This PR is to disallow `CREATE TEMPORARY TABLE ...` with an existing view name.
Under the cover, `CREATE TEMPORARY TABLE ...` will be changed to create temporary view, however, passing in a flag `replace=false`, instead of currently `true`. So when creating temporary view under the cover, if there is existing view with the same name, the operation will be blocked.

## How was this patch tested?
New unit test case is added and updated some existing test cases to adapt the new behavior

Author: Xin Wu <xinwu@us.ibm.com>

Closes #16878 from xwu0226/block_duplicate_temp_table.
2017-02-13 19:45:58 -08:00
ouyangxiaochen 6e45b547ce [SPARK-19115][SQL] Supporting Create Table Like Location
What changes were proposed in this pull request?

Support CREATE [EXTERNAL] TABLE LIKE LOCATION... syntax for Hive serde and datasource tables.
In this PR,we follow SparkSQL design rules :

    supporting create table like view or physical table or temporary view with location.
    creating a table with location,this table will be an external table other than managed table.

How was this patch tested?

Add new test cases and update existing test cases

Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn>

Closes #16868 from ouyangxiaochen/spark19115.
2017-02-13 19:41:44 -08:00
Shixiong Zhu 3dbff9be06 [SPARK-19542][SS] Delete the temp checkpoint if a query is stopped without errors
## What changes were proposed in this pull request?

When a query uses a temp checkpoint dir, it's better to delete it if it's stopped without errors.

## How was this patch tested?

New unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16880 from zsxwing/delete-temp-checkpoint.
2017-02-13 11:54:54 -08:00
jiangxingbo af63c52fd3 [SPARK-19025][SQL] Remove SQL builder for operators
## What changes were proposed in this pull request?

With the new approach of view resolution, we can get rid of SQL generation on view creation, so let's remove SQL builder for operators.

Note that, since all sql generation for operators is defined in one file (org.apache.spark.sql.catalyst.SQLBuilder), it’d be trivial to recover it in the future.

## How was this patch tested?

N/A

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16869 from jiangxb1987/SQLBuilder.
2017-02-09 19:35:39 +01:00
Ala Luszczak 4064574d03 [SPARK-19514] Making range interruptible.
## What changes were proposed in this pull request?

Previously range operator could not be interrupted. For example, using DAGScheduler.cancelStage(...) on a query with range might have been ineffective.

This change adds periodic checks of TaskContext.isInterrupted to codegen version, and InterruptibleOperator to non-codegen version.

I benchmarked the performance of codegen version on a sample query `spark.range(1000L * 1000 * 1000 * 10).count()` and there is no measurable difference.

## How was this patch tested?

Adds a unit test.

Author: Ala Luszczak <ala@databricks.com>

Closes #16872 from ala/SPARK-19514b.
2017-02-09 19:07:06 +01:00
Liwei Lin 9d9d67c795 [SPARK-19265][SQL][FOLLOW-UP] Configurable tableRelationCache maximum size
## What changes were proposed in this pull request?

SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable.

In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`.

## How was this patch tested?

new test case: `test("conf entry: checkValue()")`

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16736 from lw-lin/conf.
2017-02-09 00:48:47 -05:00
Sean Owen e8d3fca450
[SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and earlier
## What changes were proposed in this pull request?

- Remove support for Hadoop 2.5 and earlier
- Remove reflection and code constructs only needed to support multiple versions at once
- Update docs to reflect newer versions
- Remove older versions' builds and profiles.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16810 from srowen/SPARK-19464.
2017-02-08 12:20:07 +00:00
windpiger d60dde26f9 [SPARK-19488][SQL] fix csv infer schema when the field is Nan/Inf etc
## What changes were proposed in this pull request?

when csv infer schema, it does not use user defined csvoptions to parse the field, such as `inf`, `-inf` which are should be parsed to DoubleType

this pr add  `options.nanValue`, `options.negativeInf`, `options.positiveIn`  to check if the field is a DoubleType

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16834 from windpiger/fixinferInfSchemaCsv.
2017-02-08 14:30:28 +08:00
CodingCat d4cd975718 [SPARK-19499][SS] Add more notes in the comments of Sink.addBatch()
## What changes were proposed in this pull request?

addBatch method in Sink trait is supposed to be a synchronous method to coordinate with the fault-tolerance design in StreamingExecution (being different with the compute() method in DStream)

We need to add more notes in the comments of this method to remind the developers

## How was this patch tested?

existing tests

Author: CodingCat <zhunansjtu@gmail.com>

Closes #16840 from CodingCat/SPARK-19499.
2017-02-07 20:25:18 -08:00
Tathagata Das aeb80348dd [SPARK-19413][SS] MapGroupsWithState for arbitrary stateful operations
## What changes were proposed in this pull request?

`mapGroupsWithState` is a new API for arbitrary stateful operations in Structured Streaming, similar to `DStream.mapWithState`

*Requirements*
- Users should be able to specify a function that can do the following
- Access the input row corresponding to a key
- Access the previous state corresponding to a key
- Optionally, update or remove the state
- Output any number of new rows (or none at all)

*Proposed API*
```
// ------------ New methods on KeyValueGroupedDataset ------------
class KeyValueGroupedDataset[K, V] {
	// Scala friendly
	def mapGroupsWithState[S: Encoder, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => U)
        def flatMapGroupsWithState[S: Encode, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => Iterator[U])
	// Java friendly
       def mapGroupsWithState[S, U](func: MapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
       def flatMapGroupsWithState[S, U](func: FlatMapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
}

// ------------------- New Java-friendly function classes -------------------
public interface MapGroupsWithStateFunction<K, V, S, R> extends Serializable {
  R call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}
public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends Serializable {
  Iterator<R> call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}

// ---------------------- Wrapper class for state data ----------------------
trait State[S] {
	def exists(): Boolean
  	def get(): S 			// throws Exception is state does not exist
	def getOption(): Option[S]
	def update(newState: S): Unit
	def remove(): Unit		// exists() will be false after this
}
```

Key Semantics of the State class
- The state can be null.
- If the state.remove() is called, then state.exists() will return false, and getOption will returm None.
- After that state.update(newState) is called, then state.exists() will return true, and getOption will return Some(...).
- None of the operations are thread-safe. This is to avoid memory barriers.

*Usage*
```
val stateFunc = (word: String, words: Iterator[String, runningCount: KeyedState[Long]) => {
    val newCount = words.size + runningCount.getOption.getOrElse(0L)
    runningCount.update(newCount)
   (word, newCount)
}

dataset					                        // type is Dataset[String]
  .groupByKey[String](w => w)        	                // generates KeyValueGroupedDataset[String, String]
  .mapGroupsWithState[Long, (String, Long)](stateFunc)	// returns Dataset[(String, Long)]
```

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

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

Closes #16758 from tdas/mapWithState.
2017-02-07 20:21:00 -08:00
gatorsmile e33aaa2ac5 [SPARK-19397][SQL] Make option names of LIBSVM and TEXT case insensitive
### What changes were proposed in this pull request?
Prior to Spark 2.1, the option names are case sensitive for all the formats. Since Spark 2.1, the option key names become case insensitive except the format `Text` and `LibSVM `. This PR is to fix these issues.

Also, add a check to know whether the input option vector type is legal for `LibSVM`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16737 from gatorsmile/libSVMTextOptions.
2017-02-08 09:33:18 +08:00
Reynold Xin b7277e03d1 [SPARK-19495][SQL] Make SQLConf slightly more extensible
## What changes were proposed in this pull request?
This pull request makes SQLConf slightly more extensible by removing the visibility limitations on the build* functions.

## How was this patch tested?
N/A - there are no logic changes and everything should be covered by existing unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #16835 from rxin/SPARK-19495.
2017-02-07 18:55:19 +01:00
anabranch 7a7ce272fe [SPARK-16609] Add to_date/to_timestamp with format functions
## What changes were proposed in this pull request?

This pull request adds two new user facing functions:
- `to_date` which accepts an expression and a format and returns a date.
- `to_timestamp` which accepts an expression and a format and returns a timestamp.

For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM)

### Date Function
*Previously*
```
to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp"))
```
*Current*
```
to_date(lit("2016-21-05"), "yyyy-dd-MM")
```

### Timestamp Function
*Previously*
```
unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")
```
*Current*
```
to_timestamp(lit("2016-21-05"), "yyyy-dd-MM")
```
### Tasks

- [X] Add `to_date` to Scala Functions
- [x] Add `to_date` to Python Functions
- [x] Add `to_date` to SQL Functions
- [X] Add `to_timestamp` to Scala Functions
- [x] Add `to_timestamp` to Python Functions
- [x] Add `to_timestamp` to SQL Functions
- [x] Add function to R

## How was this patch tested?

- [x] Add Functions to `DateFunctionsSuite`
- Test new `ParseToTimestamp` Expression (*not necessary*)
- Test new `ParseToDate` Expression (*not necessary*)
- [x] Add test for R
- [x] Add test for Python in test.py

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

Author: anabranch <wac.chambers@gmail.com>
Author: Bill Chambers <bill@databricks.com>
Author: anabranch <bill@databricks.com>

Closes #16138 from anabranch/SPARK-16609.
2017-02-07 15:50:30 +01:00
Ala Luszczak 6ed285c68f [SPARK-19447] Fixing input metrics for range operator.
## What changes were proposed in this pull request?

This change introduces a new metric "number of generated rows". It is used exclusively for Range, which is a leaf in the query tree, yet doesn't read any input data, and therefore cannot report "recordsRead".

Additionally the way in which the metrics are reported by the JIT-compiled version of Range was changed. Previously, it was immediately reported that all the records were produced. This could be confusing for a user monitoring execution progress in the UI. Now, the metric is updated gradually.

In order to avoid negative impact on Range performance, the code generation was reworked. The values are now produced in batches in the tighter inner loop, while the metrics are updated in the outer loop.

The change also contains a number of unit tests, which should help ensure the correctness of metrics for various input sources.

## How was this patch tested?

Unit tests.

Author: Ala Luszczak <ala@databricks.com>

Closes #16829 from ala/SPARK-19447.
2017-02-07 14:21:30 +01:00
hyukjinkwon 3d314d08c9 [SPARK-16101][SQL] Refactoring CSV schema inference path to be consistent with JSON
## What changes were proposed in this pull request?

This PR refactors CSV schema inference path to be consistent with JSON data source and moves some filtering codes having the similar/same logics into `CSVUtils`.

 It makes the methods in classes have consistent arguments with JSON ones. (this PR renames `.../json/InferSchema.scala` → `.../json/JsonInferSchema.scala`)

`CSVInferSchema` and `JsonInferSchema`

``` scala
private[csv] object CSVInferSchema {
  ...

  def infer(
      csv: Dataset[String],
      caseSensitive: Boolean,
      options: CSVOptions): StructType = {
  ...
```

``` scala
private[sql] object JsonInferSchema {
  ...

  def infer(
      json: RDD[String],
      columnNameOfCorruptRecord: String,
      configOptions: JSONOptions): StructType = {
  ...
```

These allow schema inference from `Dataset[String]` directly, meaning the similar functionalities that use `JacksonParser`/`JsonInferSchema` for JSON can be easily implemented by `UnivocityParser`/`CSVInferSchema` for CSV.

This completes refactoring CSV datasource and they are now pretty consistent.

## How was this patch tested?

Existing tests should cover this and

```
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16680 from HyukjinKwon/SPARK-16101-schema-inference.
2017-02-07 21:02:20 +08:00
uncleGen 7a0a630e0f [SPARK-19407][SS] defaultFS is used FileSystem.get instead of getting it from uri scheme
## What changes were proposed in this pull request?

```
Caused by: java.lang.IllegalArgumentException: Wrong FS: s3a://**************/checkpoint/7b2231a3-d845-4740-bfa3-681850e5987f/metadata, expected: file:///
	at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649)
	at org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:82)
	at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:606)
	at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:824)
	at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:601)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421)
	at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1426)
	at org.apache.spark.sql.execution.streaming.StreamMetadata$.read(StreamMetadata.scala:51)
	at org.apache.spark.sql.execution.streaming.StreamExecution.<init>(StreamExecution.scala:100)
	at org.apache.spark.sql.streaming.StreamingQueryManager.createQuery(StreamingQueryManager.scala:232)
	at org.apache.spark.sql.streaming.StreamingQueryManager.startQuery(StreamingQueryManager.scala:269)
	at org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:262)
```

Can easily replicate on spark standalone cluster by providing checkpoint location uri scheme anything other than "file://" and not overriding in config.

WorkAround  --conf spark.hadoop.fs.defaultFS=s3a://somebucket or set it in sparkConf or spark-default.conf

## How was this patch tested?

existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #16815 from uncleGen/SPARK-19407.
2017-02-06 21:03:20 -08:00
Wenchen Fan aff53021cf [SPARK-19080][SQL] simplify data source analysis
## What changes were proposed in this pull request?

The current way of resolving `InsertIntoTable` and `CreateTable` is convoluted: sometimes we replace them with concrete implementation commands during analysis, sometimes during planning phase.

And the error checking logic is also a mess: we may put it in extended analyzer rules, or extended checking rules, or `CheckAnalysis`.

This PR simplifies the data source analysis:

1.  `InsertIntoTable` and `CreateTable` are always unresolved and need to be replaced by concrete implementation commands during analysis.
2. The error checking logic is mainly in 2 rules: `PreprocessTableCreation` and `PreprocessTableInsertion`.

## How was this patch tested?

existing test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16269 from cloud-fan/ddl.
2017-02-07 00:36:57 +08:00
Cheng Lian 7730426cb9 [SPARK-19409][SPARK-17213] Cleanup Parquet workarounds/hacks due to bugs of old Parquet versions
## What changes were proposed in this pull request?

We've already upgraded parquet-mr to 1.8.2. This PR does some further cleanup by removing a workaround of PARQUET-686 and a hack due to PARQUET-363 and PARQUET-278. All three Parquet issues are fixed in parquet-mr 1.8.2.

## How was this patch tested?

Existing unit tests.

Author: Cheng Lian <lian@databricks.com>

Closes #16791 from liancheng/parquet-1.8.2-cleanup.
2017-02-06 09:10:55 +01:00
gatorsmile 65b10ffb38 [SPARK-19279][SQL] Infer Schema for Hive Serde Tables and Block Creating a Hive Table With an Empty Schema
### What changes were proposed in this pull request?
So far, we allow users to create a table with an empty schema: `CREATE TABLE tab1`. This could break many code paths if we enable it. Thus, we should follow Hive to block it.

For Hive serde tables, some serde libraries require the specified schema and record it in the metastore. To get the list, we need to check `hive.serdes.using.metastore.for.schema,` which contains a list of serdes that require user-specified schema. The default values are

- org.apache.hadoop.hive.ql.io.orc.OrcSerde
- org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
- org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe
- org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe
- org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe
- org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
- org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe

### How was this patch tested?
Added test cases for both Hive and data source tables

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16636 from gatorsmile/fixEmptyTableSchema.
2017-02-06 13:30:07 +08:00
hyukjinkwon 2f3c20bbdd [SPARK-19446][SQL] Remove unused findTightestCommonType in TypeCoercion
## What changes were proposed in this pull request?

This PR proposes to

- remove unused `findTightestCommonType` in `TypeCoercion` as suggested in https://github.com/apache/spark/pull/16777#discussion_r99283834
- rename `findTightestCommonTypeOfTwo ` to `findTightestCommonType`.
- fix comments accordingly

The usage was removed while refactoring/fixing in several JIRAs such as SPARK-16714, SPARK-16735 and SPARK-16646

## How was this patch tested?

Existing tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16786 from HyukjinKwon/SPARK-19446.
2017-02-03 22:10:17 -08:00
Liang-Chi Hsieh bf493686eb [SPARK-19411][SQL] Remove the metadata used to mark optional columns in merged Parquet schema for filter predicate pushdown
## What changes were proposed in this pull request?

There is a metadata introduced before to mark the optional columns in merged Parquet schema for filter predicate pushdown. As we upgrade to Parquet 1.8.2 which includes the fix for the pushdown of optional columns, we don't need this metadata now.

## 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 #16756 from viirya/remove-optional-metadata.
2017-02-03 11:58:42 +01:00
Zheng RuiFeng b0985764f0 [SPARK-14352][SQL] approxQuantile should support multi columns
## What changes were proposed in this pull request?

1, add the multi-cols support based on current private api
2, add the multi-cols support to pyspark
## How was this patch tested?

unit tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>
Author: Ruifeng Zheng <ruifengz@foxmail.com>

Closes #12135 from zhengruifeng/quantile4multicols.
2017-02-01 14:11:28 -08:00
hyukjinkwon 5ed397baa7 [SPARK-19296][SQL] Deduplicate url and table in JdbcUtils
## What changes were proposed in this pull request?

This PR deduplicates arguments, `url` and `table` in `JdbcUtils` with `JDBCOptions`.

It avoids to use duplicated arguments, for example, as below:

from

```scala
val jdbcOptions = new JDBCOptions(url, table, map)
JdbcUtils.saveTable(ds, url, table, jdbcOptions)
```

to

```scala
val jdbcOptions = new JDBCOptions(url, table, map)
JdbcUtils.saveTable(ds, jdbcOptions)
```

## How was this patch tested?

Running unit test in `JdbcSuite`/`JDBCWriteSuite`

Building with Scala 2.10 as below:

```
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16753 from HyukjinKwon/SPARK-19296.
2017-02-01 09:43:35 -08:00
hyukjinkwon f1a1f2607d
[SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings in Scala/Java APIs generation
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Support LaTex inline-formula, `\( ... \)` in Scala API documentation
  It seems currently,

  ```
  \( ... \)
  ```

  are rendered as they are, for example,

  <img width="345" alt="2017-01-30 10 01 13" src="https://cloud.githubusercontent.com/assets/6477701/22423960/ab37d54a-e737-11e6-9196-4f6229c0189c.png">

  It seems mistakenly more backslashes were added.

- Fix warnings Scaladoc/Javadoc generation
  This PR fixes t two types of warnings as below:

  ```
  [warn] .../spark/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala:335: Could not find any member to link for "UnsupportedOperationException".
  [warn]   /**
  [warn]   ^
  ```

  ```
  [warn] .../spark/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala:24: Variable var undefined in comment for class VariableSubstitution in class VariableSubstitution
  [warn]  * `${var}`, `${system:var}` and `${env:var}`.
  [warn]      ^
  ```

- Fix Javadoc8 break
  ```
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictionModel.java:7: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictorParams.java:12: error: reference not found
  [error]    *                          E.g., {link VectorUDT} for vector features.
  [error]                                            ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/Predictor.java:10: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/HiveAnalysis.java:5: error: reference not found
  [error]  * Note that, this rule must be run after {link PreprocessTableInsertion}.
  [error]                                                  ^
  ```

## How was this patch tested?

Manually via `sbt unidoc` and `jeykil build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16741 from HyukjinKwon/warn-and-break.
2017-02-01 13:26:16 +00:00
Burak Yavuz 081b7addaf [SPARK-19378][SS] Ensure continuity of stateOperator and eventTime metrics even if there is no new data in trigger
## What changes were proposed in this pull request?

In StructuredStreaming, if a new trigger was skipped because no new data arrived, we suddenly report nothing for the metrics `stateOperator`. We could however easily report the metrics from `lastExecution` to ensure continuity of metrics.

## How was this patch tested?

Regression test in `StreamingQueryStatusAndProgressSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16716 from brkyvz/state-agg.
2017-01-31 16:52:53 -08:00
Wenchen Fan f7c07db852 [SPARK-19152][SQL][FOLLOWUP] simplify CreateHiveTableAsSelectCommand
## What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/16552 , `CreateHiveTableAsSelectCommand` becomes very similar to `CreateDataSourceTableAsSelectCommand`, and we can further simplify it by only creating table in the table-not-exist branch.

This PR also adds hive provider checking in DataStream reader/writer, which is missed in #16552

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16693 from cloud-fan/minor.
2017-01-28 20:38:03 -08:00
Takeshi YAMAMURO 9f523d3192 [SPARK-19338][SQL] Add UDF names in explain
## What changes were proposed in this pull request?
This pr added a variable for a UDF name in `ScalaUDF`.
Then, if the variable filled, `DataFrame#explain` prints the name.

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

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

Closes #16707 from maropu/SPARK-19338.
2017-01-26 09:50:42 -08:00
Takuya UESHIN 2969fb4370 [SPARK-18936][SQL] Infrastructure for session local timezone support.
## What changes were proposed in this pull request?

As of Spark 2.1, Spark SQL assumes the machine timezone for datetime manipulation, which is bad if users are not in the same timezones as the machines, or if different users have different timezones.

We should introduce a session local timezone setting that is used for execution.

An explicit non-goal is locale handling.

### Semantics

Setting the session local timezone means that the timezone-aware expressions listed below should use the timezone to evaluate values, and also it should be used to convert (cast) between string and timestamp or between timestamp and date.

- `CurrentDate`
- `CurrentBatchTimestamp`
- `Hour`
- `Minute`
- `Second`
- `DateFormatClass`
- `ToUnixTimestamp`
- `UnixTimestamp`
- `FromUnixTime`

and below are implicitly timezone-aware through cast from timestamp to date:

- `DayOfYear`
- `Year`
- `Quarter`
- `Month`
- `DayOfMonth`
- `WeekOfYear`
- `LastDay`
- `NextDay`
- `TruncDate`

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values evaluated by some of timezone-aware expressions are:

```scala
scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts                 |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2016-01-01 00:00:00|2016                  |1                      |1                           |0       |0         |0         |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```

whereas setting the session local timezone to `"PST"`, they are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "PST")

scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts                 |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2015-12-31 16:00:00|2015                  |12                     |31                          |16      |0         |0         |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```

Notice that even if you set the session local timezone, it affects only in `DataFrame` operations, neither in `Dataset` operations, `RDD` operations nor in `ScalaUDF`s. You need to properly handle timezone by yourself.

### Design of the fix

I introduced an analyzer to pass session local timezone to timezone-aware expressions and modified DateTimeUtils to take the timezone argument.

## How was this patch tested?

Existing tests and added tests for timezone aware expressions.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16308 from ueshin/issues/SPARK-18350.
2017-01-26 11:51:05 +01:00
Dilip Biswal 9effc2cdcb [TESTS][SQL] Setup testdata at the beginning for tests to run independently
## What changes were proposed in this pull request?

In CachedTableSuite, we are not setting up the test data at the beginning. Some tests fail while trying to run individually. When running the entire suite they run fine.

Here are some of the tests that fail -

- test("SELECT star from cached table")
- test("Self-join cached")

As part of this simplified a couple of tests by calling a support method to count the number of
InMemoryRelations.

## How was this patch tested?

Ran the failing tests individually.

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

Closes #16688 from dilipbiswal/cachetablesuite_simple.
2017-01-25 21:50:45 -08:00
Kousuke Saruta 15ef3740de [SPARK-19334][SQL] Fix the code injection vulnerability related to Generator functions.
## What changes were proposed in this pull request?

Similar to SPARK-15165, codegen is in danger of arbitrary code injection. The root cause is how variable names are created by codegen.
In GenerateExec#codeGenAccessor, a variable name is created like as follows.

```
val value = ctx.freshName(name)
```

The variable `value` is named based on the value of the variable `name` and the value of `name` is from schema given by users so an attacker can attack with queries like as follows.

```
SELECT inline(array(cast(struct(1) AS struct<`=new Object() { {f();} public void f() {throw new RuntimeException("This exception is injected.");} public int x;}.x`:int>)))
```

In the example above, a RuntimeException is thrown but an attacker can replace it with arbitrary code.

## How was this patch tested?

Added a new test case.

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

Closes #16681 from sarutak/SPARK-19334.
2017-01-24 23:35:23 +01:00
windpiger 3c86fdddf4 [SPARK-19152][SQL] DataFrameWriter.saveAsTable support hive append
## What changes were proposed in this pull request?

After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19107), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support.

This PR implement:
DataFrameWriter.saveAsTable work with hive format with append mode

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16552 from windpiger/saveAsTableWithHiveAppend.
2017-01-24 20:40:27 +08:00
hyukjinkwon ec9493b445 [SPARK-16101][HOTFIX] Fix the build with Scala 2.10 by explicit typed argument
## What changes were proposed in this pull request?

I goofed in https://github.com/apache/spark/pull/16669 which introduces the break in scala 2.10.

This fixes

```bash
[error] /home/jenkins/workspace/spark-master-compile-sbt-scala-2.10/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala:65: polymorphic expression cannot be instantiated to expected type;
[error]  found   : [B >: org.apache.spark.sql.types.StructField](B, Int) => Int
[error]  required: org.apache.spark.sql.types.StructField => ?
[error]     fields.map(schema.indexOf).toArray
[error]                       ^
[error] one error found
[error] (sql/compile:compileIncremental) Compilation failed
```

## How was this patch tested?

Manually via

```bash
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```
```
[INFO] ------------------------------------------------------------------------
[INFO] Reactor Summary:
[INFO]
[INFO] Spark Project Parent POM ........................... SUCCESS [  2.719 s]
[INFO] Spark Project Tags ................................. SUCCESS [  3.441 s]
[INFO] Spark Project Sketch ............................... SUCCESS [  3.411 s]
[INFO] Spark Project Networking ........................... SUCCESS [  5.088 s]
[INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [  5.131 s]
[INFO] Spark Project Unsafe ............................... SUCCESS [  5.813 s]
[INFO] Spark Project Launcher ............................. SUCCESS [  6.567 s]
[INFO] Spark Project Core ................................. SUCCESS [01:39 min]
[INFO] Spark Project ML Local Library ..................... SUCCESS [  6.644 s]
[INFO] Spark Project GraphX ............................... SUCCESS [ 11.304 s]
[INFO] Spark Project Streaming ............................ SUCCESS [ 26.275 s]
[INFO] Spark Project Catalyst ............................. SUCCESS [01:04 min]
[INFO] Spark Project SQL .................................. SUCCESS [02:07 min]
[INFO] Spark Project ML Library ........................... SUCCESS [01:20 min]
[INFO] Spark Project Tools ................................ SUCCESS [  8.755 s]
[INFO] Spark Project Hive ................................. SUCCESS [ 51.141 s]
[INFO] Spark Project REPL ................................. SUCCESS [ 13.688 s]
[INFO] Spark Project YARN Shuffle Service ................. SUCCESS [  7.211 s]
[INFO] Spark Project YARN ................................. SUCCESS [ 10.908 s]
[INFO] Spark Project Assembly ............................. SUCCESS [  2.940 s]
[INFO] Spark Project External Flume Sink .................. SUCCESS [  4.386 s]
[INFO] Spark Project External Flume ....................... SUCCESS [  8.589 s]
[INFO] Spark Project External Flume Assembly .............. SUCCESS [  1.891 s]
[INFO] Spark Integration for Kafka 0.8 .................... SUCCESS [  8.458 s]
[INFO] Spark Project Examples ............................. SUCCESS [ 17.706 s]
[INFO] Spark Project External Kafka Assembly .............. SUCCESS [  3.070 s]
[INFO] Spark Integration for Kafka 0.10 ................... SUCCESS [ 11.227 s]
[INFO] Spark Integration for Kafka 0.10 Assembly .......... SUCCESS [  2.982 s]
[INFO] Kafka 0.10 Source for Structured Streaming ......... SUCCESS [  7.494 s]
[INFO] Spark Project Java 8 Tests ......................... SUCCESS [  3.748 s]
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
```

and manual test `CSVSuite` with Scala 2.11 with my IDE.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16684 from HyukjinKwon/hot-fix-type-ensurance.
2017-01-23 23:57:22 -08:00
Shixiong Zhu 60bd91a340 [SPARK-19268][SS] Disallow adaptive query execution for streaming queries
## What changes were proposed in this pull request?

As adaptive query execution may change the number of partitions in different batches, it may break streaming queries. Hence, we should disallow this feature in Structured Streaming.

## How was this patch tested?

`test("SPARK-19268: Adaptive query execution should be disallowed")`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16683 from zsxwing/SPARK-19268.
2017-01-23 22:30:51 -08:00
hyukjinkwon e576c1ed79 [SPARK-9435][SQL] Reuse function in Java UDF to correctly support expressions that require equality comparison between ScalaUDF
## What changes were proposed in this pull request?

Currently, running the codes in Java

```java
spark.udf().register("inc", new UDF1<Long, Long>() {
  Override
  public Long call(Long i) {
    return i + 1;
  }
}, DataTypes.LongType);

spark.range(10).toDF("x").createOrReplaceTempView("tmp");
Row result = spark.sql("SELECT inc(x) FROM tmp GROUP BY inc(x)").head();
Assert.assertEquals(7, result.getLong(0));
```

fails as below:

```
org.apache.spark.sql.AnalysisException: expression 'tmp.`x`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
Aggregate [UDF(x#19L)], [UDF(x#19L) AS UDF(x)#23L]
+- SubqueryAlias tmp, `tmp`
   +- Project [id#16L AS x#19L]
      +- Range (0, 10, step=1, splits=Some(8))

	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:40)
	at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:57)
```

The root cause is because we were creating the function every time when it needs to build as below:

```scala
scala> def inc(i: Int) = i + 1
inc: (i: Int)Int

scala> (inc(_: Int)).hashCode
res15: Int = 1231799381

scala> (inc(_: Int)).hashCode
res16: Int = 2109839984

scala> (inc(_: Int)) == (inc(_: Int))
res17: Boolean = false
```

This seems leading to the comparison failure between `ScalaUDF`s created from Java UDF API, for example, in `Expression.semanticEquals`.

In case of Scala one, it seems already fine.

Both can be tested easily as below if any reviewer is more comfortable with Scala:

```scala
val df = Seq((1, 10), (2, 11), (3, 12)).toDF("x", "y")
val javaUDF = new UDF1[Int, Int]  {
  override def call(i: Int): Int = i + 1
}
// spark.udf.register("inc", javaUDF, IntegerType) // Uncomment this for Java API
// spark.udf.register("inc", (i: Int) => i + 1)    // Uncomment this for Scala API
df.createOrReplaceTempView("tmp")
spark.sql("SELECT inc(y) FROM tmp GROUP BY inc(y)").show()
```

## How was this patch tested?

Unit test in `JavaUDFSuite.java` and `./dev/lint-java`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16553 from HyukjinKwon/SPARK-9435.
2017-01-23 22:20:42 -08:00
jiangxingbo 3bdf3ee860 [SPARK-19272][SQL] Remove the param viewOriginalText from CatalogTable
## What changes were proposed in this pull request?

Hive will expand the view text, so it needs 2 fields: originalText and viewText. Since we don't expand the view text, but only add table properties, perhaps only a single field `viewText` is enough in CatalogTable.

This PR brought in the following changes:
1. Remove the param `viewOriginalText` from `CatalogTable`;
2. Update the output of command `DescribeTableCommand`.

## How was this patch tested?

Tested by exsiting test cases, also updated the failed test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16679 from jiangxb1987/catalogTable.
2017-01-24 12:37:30 +08:00
Wenchen Fan fcfd5d0bba [SPARK-19290][SQL] add a new extending interface in Analyzer for post-hoc resolution
## What changes were proposed in this pull request?

To implement DDL commands, we added several analyzer rules in sql/hive module to analyze DDL related plans. However, our `Analyzer` currently only have one extending interface: `extendedResolutionRules`, which defines extra rules that will be run together with other rules in the resolution batch, and doesn't fit DDL rules well, because:

1. DDL rules may do some checking and normalization, but we may do it many times as the resolution batch will run rules again and again, until fixed point, and it's hard to tell if a DDL rule has already done its checking and normalization. It's fine because DDL rules are idempotent, but it's bad for analysis performance
2. some DDL rules may depend on others, and it's pretty hard to write `if` conditions to guarantee the dependencies. It will be good if we have a batch which run rules in one pass, so that we can guarantee the dependencies by rules order.

This PR adds a new extending interface in `Analyzer`: `postHocResolutionRules`, which defines rules that will be run only once in a batch runs right after the resolution batch.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16645 from cloud-fan/analyzer.
2017-01-23 20:01:10 -08:00
windpiger 0ef1421a64 [SPARK-19284][SQL] append to partitioned datasource table should without custom partition location
## What changes were proposed in this pull request?

when we append data to a existed partitioned datasource table, the InsertIntoHadoopFsRelationCommand.getCustomPartitionLocations currently
return the same location with Hive default, it should return None.

## How was this patch tested?

Author: windpiger <songjun@outlook.com>

Closes #16642 from windpiger/appendSchema.
2017-01-23 19:06:04 +08:00
Dongjoon Hyun c4a6519c44 [SPARK-19218][SQL] Fix SET command to show a result correctly and in a sorted order
## What changes were proposed in this pull request?

This PR aims to fix the following two things.

1. `sql("SET -v").collect()` or `sql("SET -v").show()` raises the following exceptions for String configuration with default value, `null`. For the test, please see [Jenkins result](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71539/testReport/) and 60953bf1f1 in #16624 .

```
sbt.ForkMain$ForkError: java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
createexternalrow(input[0, string, false].toString, input[1, string, false].toString, input[2, string, false].toString, StructField(key,StringType,false), StructField(value,StringType,false), StructField(meaning,StringType,false))
:- input[0, string, false].toString
:  +- input[0, string, false]
:- input[1, string, false].toString
:  +- input[1, string, false]
+- input[2, string, false].toString
   +- input[2, string, false]
```

2. Currently, `SET` and `SET -v` commands show unsorted result.
    We had better show a sorted result for UX. Also, this is compatible with Hive.

**BEFORE**
```
scala> sql("set").show(false)
...
|spark.driver.host              |10.22.16.140                                                                                                                                 |
|spark.driver.port              |63893                                                                                                                                        |
|spark.repl.class.uri           |spark://10.22.16.140:63893/classes                                                                                                           |
...
|spark.app.name                 |Spark shell                                                                                                                                  |
|spark.driver.memory            |4G                                                                                                                                           |
|spark.executor.id              |driver                                                                                                                                       |
|spark.submit.deployMode        |client                                                                                                                                       |
|spark.master                   |local[*]                                                                                                                                     |
|spark.home                     |/Users/dhyun/spark                                                                                                                           |
|spark.sql.catalogImplementation|hive                                                                                                                                         |
|spark.app.id                   |local-1484333618945                                                                                                                          |
```

**AFTER**

```
scala> sql("set").show(false)
...
|spark.app.id                   |local-1484333925649                                                                                                                          |
|spark.app.name                 |Spark shell                                                                                                                                  |
|spark.driver.host              |10.22.16.140                                                                                                                                 |
|spark.driver.memory            |4G                                                                                                                                           |
|spark.driver.port              |64994                                                                                                                                        |
|spark.executor.id              |driver                                                                                                                                       |
|spark.jars                     |                                                                                                                                             |
|spark.master                   |local[*]                                                                                                                                     |
|spark.repl.class.uri           |spark://10.22.16.140:64994/classes                                                                                                           |
|spark.sql.catalogImplementation|hive                                                                                                                                         |
|spark.submit.deployMode        |client                                                                                                                                       |
```

## How was this patch tested?

Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16579 from dongjoon-hyun/SPARK-19218.
2017-01-23 01:21:44 -08:00
Wenchen Fan de6ad3dfa7 [SPARK-19309][SQL] disable common subexpression elimination for conditional expressions
## What changes were proposed in this pull request?

As I pointed out in https://github.com/apache/spark/pull/15807#issuecomment-259143655 , the current subexpression elimination framework has a problem, it always evaluates all common subexpressions at the beginning, even they are inside conditional expressions and may not be accessed.

Ideally we should implement it like scala lazy val, so we only evaluate it when it gets accessed at lease once. https://github.com/apache/spark/issues/15837 tries this approach, but it seems too complicated and may introduce performance regression.

This PR simply stops common subexpression elimination for conditional expressions, with some cleanup.

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16659 from cloud-fan/codegen.
2017-01-23 13:31:26 +08:00
gatorsmile 772035e771 [SPARK-19229][SQL] Disallow Creating Hive Source Tables when Hive Support is Not Enabled
### What changes were proposed in this pull request?
It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables.

### How was this patch tested?
Fixed the test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16587 from gatorsmile/blockHiveTable.
2017-01-22 20:37:37 -08:00
hyukjinkwon 74e65cb74a [SPARK-16101][SQL] Refactoring CSV read path to be consistent with JSON data source
## What changes were proposed in this pull request?

This PR refactors CSV read path to be consistent with JSON data source. It makes the methods in classes have consistent arguments with JSON ones.

`UnivocityParser` and `JacksonParser`

``` scala
private[csv] class UnivocityParser(
    schema: StructType,
    requiredSchema: StructType,
    options: CSVOptions) extends Logging {
  ...

def parse(input: String): Seq[InternalRow] = {
  ...
```

``` scala
class JacksonParser(
    schema: StructType,
    columnNameOfCorruptRecord: String,
    options: JSONOptions) extends Logging {
  ...

def parse(input: String): Option[InternalRow] = {
  ...
```

These allow parsing an iterator (`String` to `InternalRow`) as below for both JSON and CSV:

```scala
iter.flatMap(parser.parse)
```

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16669 from HyukjinKwon/SPARK-16101-read.
2017-01-23 12:23:12 +08:00
Wenchen Fan 3c2ba9fcc4 [SPARK-19305][SQL] partitioned table should always put partition columns at the end of table schema
## What changes were proposed in this pull request?

For data source tables, we will always reorder the specified table schema, or the query in CTAS, to put partition columns at the end. e.g. `CREATE TABLE t(a int, b int, c int, d int) USING parquet PARTITIONED BY (d, b)` will create a table with schema `<a, c, d, b>`

Hive serde tables don't have this problem before, because its CREATE TABLE syntax specifies data schema and partition schema individually.

However, after we unifed the CREATE TABLE syntax, Hive serde table also need to do the reorder. This PR puts the reorder logic in a analyzer rule,  which works with both data source tables and Hive serde tables.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16655 from cloud-fan/schema.
2017-01-21 13:57:50 +08:00
sureshthalamati f174cdc747 [SPARK-14536][SQL] fix to handle null value in array type column for postgres.
## What changes were proposed in this pull request?

JDBC  read  is failing with  NPE due to missing null value check for array data type if the source table has null values in the array type column.  For null values Resultset.getArray()  returns null.
This PR adds null safe check to the Resultset.getArray() value before invoking method on the Array object.
## How was this patch tested?

Updated the PostgresIntegration test suite to test null values. Ran docker integration tests on my laptop.

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #15192 from sureshthalamati/jdbc_array_null_fix-SPARK-14536.
2017-01-20 19:23:20 -08:00
hyukjinkwon 54268b42dc [SPARK-16101][SQL] Refactoring CSV write path to be consistent with JSON data source
## What changes were proposed in this pull request?

This PR refactors CSV write path to be consistent with JSON data source.

This PR makes the methods in classes have consistent arguments with JSON ones.
  - `UnivocityGenerator` and `JacksonGenerator`

    ``` scala
    private[csv] class UnivocityGenerator(
        schema: StructType,
        writer: Writer,
        options: CSVOptions = new CSVOptions(Map.empty[String, String])) {
    ...

    def write ...
    def close ...
    def flush ...
    ```

    ``` scala
    private[sql] class JacksonGenerator(
       schema: StructType,
       writer: Writer,
       options: JSONOptions = new JSONOptions(Map.empty[String, String])) {
    ...

    def write ...
    def close ...
    def flush ...
    ```

- This PR also makes the classes put in together in a consistent manner with JSON.
  - `CsvFileFormat`

    ``` scala
    CsvFileFormat
    CsvOutputWriter
    ```

  - `JsonFileFormat`

    ``` scala
    JsonFileFormat
    JsonOutputWriter
    ```

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16496 from HyukjinKwon/SPARK-16101-write.
2017-01-21 10:43:52 +08:00
Shixiong Zhu ea31f92bb8 [SPARK-19267][SS] Fix a race condition when stopping StateStore
## What changes were proposed in this pull request?

There is a race condition when stopping StateStore which makes `StateStoreSuite.maintenance` flaky. `StateStore.stop` doesn't wait for the running task to finish, and an out-of-date task may fail `doMaintenance` and cancel the new task. Here is a reproducer: dde1b5b106

This PR adds MaintenanceTask to eliminate the race condition.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #16627 from zsxwing/SPARK-19267.
2017-01-20 17:49:26 -08:00
Wenchen Fan 0bf605c2c6 [SPARK-19292][SQL] filter with partition columns should be case-insensitive on Hive tables
## What changes were proposed in this pull request?

When we query a table with a filter on partitioned columns, we will push the partition filter to the metastore to get matched partitions directly.

In `HiveExternalCatalog.listPartitionsByFilter`, we assume the column names in partition filter are already normalized and we don't need to consider case sensitivity. However, `HiveTableScanExec` doesn't follow this assumption. This PR fixes it.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16647 from cloud-fan/bug.
2017-01-19 20:09:48 -08:00
Kazuaki Ishizaki 148a84b370 [SPARK-17912] [SQL] Refactor code generation to get data for ColumnVector/ColumnarBatch
## What changes were proposed in this pull request?

This PR refactors the code generation part to get data from `ColumnarVector` and `ColumnarBatch` by using a trait `ColumnarBatchScan` for ease of reuse. This is because this part will be reused by several components (e.g. parquet reader, Dataset.cache, and others) since `ColumnarBatch` will be first citizen.

This PR is a part of https://github.com/apache/spark/pull/15219. In advance, this PR makes the code generation for  `ColumnarVector` and `ColumnarBatch` reuseable as a trait. In general, this is very useful for other components from the reuseability view, too.
## How was this patch tested?

tested existing test suites

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

Closes #15467 from kiszk/columnarrefactor.
2017-01-19 15:16:05 -08:00
jayadevanmurali 064fadd2a2 [SPARK-19059][SQL] Unable to retrieve data from parquet table whose name startswith underscore
## What changes were proposed in this pull request?
The initial shouldFilterOut() method invocation filter the root path name(table name in the intial call) and remove if it contains _. I moved the check one level below, so it first list files/directories in the given root path and then apply filter.
(Please fill in changes proposed in this fix)

## How was this patch tested?
Added new test case for this scenario
(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: jayadevanmurali <jayadevan.m@tcs.com>
Author: jayadevan <jayadevan.m@tcs.com>

Closes #16635 from jayadevanmurali/branch-0.1-SPARK-19059.
2017-01-19 20:07:52 +08:00
Wenchen Fan 2e62560024 [SPARK-19265][SQL] make table relation cache general and does not depend on hive
## What changes were proposed in this pull request?

We have a table relation plan cache in `HiveMetastoreCatalog`, which caches a lot of things: file status, resolved data source, inferred schema, etc.

However, it doesn't make sense to limit this cache with hive support, we should move it to SQL core module so that users can use this cache without hive support.

It can also reduce the size of `HiveMetastoreCatalog`, so that it's easier to remove it eventually.

main changes:
1. move the table relation cache to `SessionCatalog`
2. `SessionCatalog.lookupRelation` will return `SimpleCatalogRelation` and the analyzer will convert it to `LogicalRelation` or `MetastoreRelation` later, then `HiveSessionCatalog` doesn't need to override `lookupRelation` anymore
3. `FindDataSourceTable` will read/write the table relation cache.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16621 from cloud-fan/plan-cache.
2017-01-19 00:07:48 -08:00
Liwei Lin 569e50680f [SPARK-19168][STRUCTURED STREAMING] StateStore should be aborted upon error
## What changes were proposed in this pull request?

We should call `StateStore.abort()` when there should be any error before the store is committed.

## How was this patch tested?

Manually.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16547 from lw-lin/append-filter.
2017-01-18 10:52:47 -08:00
jiangxingbo f85f29608d [SPARK-19024][SQL] Implement new approach to write a permanent view
## What changes were proposed in this pull request?

On CREATE/ALTER a view, it's no longer needed to generate a SQL text string from the LogicalPlan, instead we store the SQL query text、the output column names of the query plan, and current database to CatalogTable. Permanent views created by this approach can be resolved by current view resolution approach.

The main advantage includes:
1. If you update an underlying view, the current view also gets updated;
2. That gives us a change to get ride of SQL generation for operators.

Major changes of this PR:
1. Generate the view-specific properties(e.g. view default database, view query output column names) during permanent view creation and store them as properties in the CatalogTable;
2. Update the commands `CreateViewCommand` and `AlterViewAsCommand`, get rid of SQL generation from them.

## How was this patch tested?
Existing tests.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16613 from jiangxb1987/view-write-path.
2017-01-18 19:13:01 +08:00
Wenchen Fan 4494cd9716 [SPARK-18243][SQL] Port Hive writing to use FileFormat interface
## What changes were proposed in this pull request?

Inserting data into Hive tables has its own implementation that is distinct from data sources: `InsertIntoHiveTable`, `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`.

Note that one other major difference is that data source tables write directly to the final destination without using some staging directory, and then Spark itself adds the partitions/tables to the catalog. Hive tables actually write to some staging directory, and then call Hive metastore's loadPartition/loadTable function to load those data in. So we still need to keep `InsertIntoHiveTable` to put this special logic. In the future, we should think of writing to the hive table location directly, so that we don't need to call `loadTable`/`loadPartition` at the end and remove `InsertIntoHiveTable`.

This PR removes `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`, and create a `HiveFileFormat` to implement the write logic. In the future, we should also implement the read logic in `HiveFileFormat`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16517 from cloud-fan/insert-hive.
2017-01-17 23:37:59 -08:00
Bogdan Raducanu 2992a0e79e [SPARK-13721][SQL] Support outer generators in DataFrame API
## What changes were proposed in this pull request?

Added outer_explode, outer_posexplode, outer_inline functions and expressions.
Some bug fixing in GenerateExec.scala for CollectionGenerator. Previously it was not correctly handling the case of outer with empty collections, only with nulls.

## How was this patch tested?

New tests added to GeneratorFunctionSuite

Author: Bogdan Raducanu <bogdan.rdc@gmail.com>

Closes #16608 from bogdanrdc/SPARK-13721.
2017-01-17 15:39:24 -08:00
Reynold Xin 83dff87ded [SPARK-18917][SQL] Remove schema check in appending data
## What changes were proposed in this pull request?
In append mode, we check whether the schema of the write is compatible with the schema of the existing data. It can be a significant performance issue in cloud environment to find the existing schema for files. This patch removes the check.

Note that for catalog tables, we always do the check, as discussed in https://github.com/apache/spark/pull/16339#discussion_r96208357

## How was this patch tested?
N/A

Closes #16339.

Author: Reynold Xin <rxin@databricks.com>

Closes #16622 from rxin/SPARK-18917.
2017-01-17 15:06:28 -08:00
Shixiong Zhu a83accfcfd [SPARK-19065][SQL] Don't inherit expression id in dropDuplicates
## What changes were proposed in this pull request?

`dropDuplicates` will create an Alias using the same exprId, so `StreamExecution` should also replace Alias if necessary.

## How was this patch tested?

test("SPARK-19065: dropDuplicates should not create expressions using the same id")

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16564 from zsxwing/SPARK-19065.
2017-01-18 01:57:12 +08:00
hyukjinkwon 6c00c069e3
[SPARK-3249][DOC] Fix links in ScalaDoc that cause warning messages in sbt/sbt unidoc
## What changes were proposed in this pull request?

This PR proposes to fix ambiguous link warnings by simply making them as code blocks for both javadoc and scaladoc.

```
[warn] .../spark/core/src/main/scala/org/apache/spark/Accumulator.scala:20: The link target "SparkContext#accumulator" is ambiguous. Several members fit the target:
[warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala:281: The link target "runMiniBatchSGD" is ambiguous. Several members fit the target:
[warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala:83: The link target "run" is ambiguous. Several members fit the target:
...
```

This PR also fixes javadoc8 break as below:

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                   ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                                                ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                                                                ^
[info] 3 errors
```

## How was this patch tested?

Manually via `sbt unidoc > output.txt` and the checked it via `cat output.txt | grep ambiguous`

and `sbt unidoc | grep error`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16604 from HyukjinKwon/SPARK-3249.
2017-01-17 12:28:15 +00:00
Wenchen Fan 18ee55dd5d [SPARK-19148][SQL] do not expose the external table concept in Catalog
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/16296 , we reached a consensus that we should hide the external/managed table concept to users and only expose custom table path.

This PR renames `Catalog.createExternalTable` to `createTable`(still keep the old versions for backward compatibility), and only set the table type to EXTERNAL if `path` is specified in options.

## How was this patch tested?

new tests in `CatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16528 from cloud-fan/create-table.
2017-01-17 12:54:50 +08:00
Liang-Chi Hsieh 61e48f52d1 [SPARK-19082][SQL] Make ignoreCorruptFiles work for Parquet
## What changes were proposed in this pull request?

We have a config `spark.sql.files.ignoreCorruptFiles` which can be used to ignore corrupt files when reading files in SQL. Currently the `ignoreCorruptFiles` config has two issues and can't work for Parquet:

1. We only ignore corrupt files in `FileScanRDD` . Actually, we begin to read those files as early as inferring data schema from the files. For corrupt files, we can't read the schema and fail the program. A related issue reported at http://apache-spark-developers-list.1001551.n3.nabble.com/Skip-Corrupted-Parquet-blocks-footer-tc20418.html
2. In `FileScanRDD`, we assume that we only begin to read the files when starting to consume the iterator. However, it is possibly the files are read before that. In this case, `ignoreCorruptFiles` config doesn't work too.

This patch targets Parquet datasource. If this direction is ok, we can address the same issue for other datasources like Orc.

Two main changes in this patch:

1. Replace `ParquetFileReader.readAllFootersInParallel` by implementing the logic to read footers in multi-threaded manner

    We can't ignore corrupt files if we use `ParquetFileReader.readAllFootersInParallel`. So this patch implements the logic to do the similar thing in `readParquetFootersInParallel`.

2. In `FileScanRDD`, we need to ignore corrupt file too when we call `readFunction` to return iterator.

One thing to notice is:

We read schema from Parquet file's footer. The method to read footer `ParquetFileReader.readFooter` throws `RuntimeException`, instead of `IOException`, if it can't successfully read the footer. Please check out df9d8e4154/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java (L470). So this patch catches `RuntimeException`.  One concern is that it might also shadow other runtime exceptions other than reading corrupt files.

## 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 #16474 from viirya/fix-ignorecorrupted-parquet-files.
2017-01-16 15:26:41 +08:00
gatorsmile de62ddf7ff [SPARK-19120] Refresh Metadata Cache After Loading Hive Tables
### What changes were proposed in this pull request?
```Scala
        sql("CREATE TABLE tab (a STRING) STORED AS PARQUET")

        // This table fetch is to fill the cache with zero leaf files
        spark.table("tab").show()

        sql(
          s"""
             |LOAD DATA LOCAL INPATH '$newPartitionDir' OVERWRITE
             |INTO TABLE tab
           """.stripMargin)

        spark.table("tab").show()
```

In the above example, the returned result is empty after table loading. The metadata cache could be out of dated after loading new data into the table, because loading/inserting does not update the cache. So far, the metadata cache is only used for data source tables. Thus, for Hive serde tables, only `parquet` and `orc` formats are facing such issues, because the Hive serde tables in the format of  parquet/orc could be converted to data source tables when `spark.sql.hive.convertMetastoreParquet`/`spark.sql.hive.convertMetastoreOrc` is on.

This PR is to refresh the metadata cache after processing the `LOAD DATA` command.

In addition, Spark SQL does not convert **partitioned** Hive tables (orc/parquet) to data source tables in the write path, but the read path is using the metadata cache for both **partitioned** and non-partitioned Hive tables (orc/parquet). That means, writing the partitioned parquet/orc tables still use `InsertIntoHiveTable`, instead of `InsertIntoHadoopFsRelationCommand`. To avoid reading the out-of-dated cache, `InsertIntoHiveTable` needs to refresh the metadata cache for partitioned tables. Note, it does not need to refresh the cache for non-partitioned parquet/orc tables, because it does not call `InsertIntoHiveTable` at all. Based on the comments, this PR will keep the existing logics unchanged. That means, we always refresh the table no matter whether the table is partitioned or not.

### How was this patch tested?
Added test cases in parquetSuites.scala

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16500 from gatorsmile/refreshInsertIntoHiveTable.
2017-01-15 20:40:44 +08:00
windpiger 8942353905 [SPARK-19151][SQL] DataFrameWriter.saveAsTable support hive overwrite
## What changes were proposed in this pull request?

After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19107), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support.

This PR implement:
DataFrameWriter.saveAsTable work with hive format with overwrite mode

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16549 from windpiger/saveAsTableWithHiveOverwrite.
2017-01-14 10:53:33 -08:00
Yucai Yu ad0dadaa25 [SPARK-19180] [SQL] the offset of short should be 2 in OffHeapColumn
## What changes were proposed in this pull request?

the offset of short is 4 in OffHeapColumnVector's putShorts, but actually it should be 2.

## How was this patch tested?

unit test

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

Closes #16555 from yucai/offheap_short.
2017-01-13 13:40:53 -08:00
Andrew Ash b040cef2ed Fix missing close-parens for In filter's toString
Otherwise the open parentheses isn't closed in query plan descriptions of batch scans.

    PushedFilters: [In(COL_A, [1,2,4,6,10,16,219,815], IsNotNull(COL_B), ...

Author: Andrew Ash <andrew@andrewash.com>

Closes #16558 from ash211/patch-9.
2017-01-12 23:14:07 -08:00
gatorsmile 3356b8b6a9 [SPARK-19092][SQL] Save() API of DataFrameWriter should not scan all the saved files
### What changes were proposed in this pull request?
`DataFrameWriter`'s [save() API](5d38f09f47/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala (L207)) is performing a unnecessary full filesystem scan for the saved files. The save() API is the most basic/core API in `DataFrameWriter`. We should avoid it.

The related PR: https://github.com/apache/spark/pull/16090

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16481 from gatorsmile/saveFileScan.
2017-01-13 13:05:53 +08:00
Eric Liang c71b25481a [SPARK-19183][SQL] Add deleteWithJob hook to internal commit protocol API
## What changes were proposed in this pull request?

Currently in SQL we implement overwrites by calling fs.delete() directly on the original data. This is not ideal since we the original files end up deleted even if the job aborts. We should extend the commit protocol to allow file overwrites to be managed as well.

## How was this patch tested?

Existing tests. I also fixed a bunch of tests that were depending on the commit protocol implementation being set to the legacy mapreduce one.

cc rxin cloud-fan

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #16554 from ericl/add-delete-protocol.
2017-01-12 17:45:55 +08:00
hyukjinkwon 24100f162d [SPARK-16848][SQL] Check schema validation for user-specified schema in jdbc and table APIs
## What changes were proposed in this pull request?

This PR proposes to throw an exception for both jdbc APIs when user specified schemas are not allowed or useless.

**DataFrameReader.jdbc(...)**

``` scala
spark.read.schema(StructType(Nil)).jdbc(...)
```

**DataFrameReader.table(...)**

```scala
spark.read.schema(StructType(Nil)).table("usrdb.test")
```

## How was this patch tested?

Unit test in `JDBCSuite` and `DataFrameReaderWriterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14451 from HyukjinKwon/SPARK-16848.
2017-01-11 21:03:48 -08:00
jiangxingbo 30a07071f0 [SPARK-18801][SQL] Support resolve a nested view
## What changes were proposed in this pull request?

We should be able to resolve a nested view. The main advantage is that if you update an underlying view, the current view also gets updated.
The new approach should be compatible with older versions of SPARK/HIVE, that means:
1. The new approach should be able to resolve the views that created by older versions of SPARK/HIVE;
2. The new approach should be able to resolve the views that are currently supported by SPARK SQL.

The new approach mainly brings in the following changes:
1. Add a new operator called `View` to keep track of the CatalogTable that describes the view, and the output attributes as well as the child of the view;
2. Update the `ResolveRelations` rule to resolve the relations and views, note that a nested view should be resolved correctly;
3. Add `viewDefaultDatabase` variable to `CatalogTable` to keep track of the default database name used to resolve a view, if the `CatalogTable` is not a view, then the variable should be `None`;
4. Add `AnalysisContext` to enable us to still support a view created with CTE/Windows query;
5. Enables the view support without enabling Hive support (i.e., enableHiveSupport);
6. Fix a weird behavior: the result of a view query may have different schema if the referenced table has been changed. After this PR, we try to cast the child output attributes to that from the view schema, throw an AnalysisException if cast is not allowed.

Note this is compatible with the views defined by older versions of Spark(before 2.2), which have empty `defaultDatabase` and all the relations in `viewText` have database part defined.

## How was this patch tested?
1. Add new tests in `SessionCatalogSuite` to test the function `lookupRelation`;
2. Add new test case in `SQLViewSuite` to test resolve a nested view.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16233 from jiangxb1987/resolve-view.
2017-01-11 13:44:07 -08:00
wangzhenhua a615513569 [SPARK-19149][SQL] Unify two sets of statistics in LogicalPlan
## What changes were proposed in this pull request?

Currently we have two sets of statistics in LogicalPlan: a simple stats and a stats estimated by cbo, but the computing logic and naming are quite confusing, we need to unify these two sets of stats.

## How was this patch tested?

Just modify existing tests.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16529 from wzhfy/unifyStats.
2017-01-10 22:34:44 -08:00
Wenchen Fan 3b19c74e71 [SPARK-19157][SQL] should be able to change spark.sql.runSQLOnFiles at runtime
## What changes were proposed in this pull request?

The analyzer rule that supports to query files directly will be added to `Analyzer.extendedResolutionRules` when SparkSession is created, according to the `spark.sql.runSQLOnFiles` flag. If the flag is off when we create `SparkSession`, this rule is not added and we can not query files directly even we turn on the flag later.

This PR fixes this bug by always adding that rule to `Analyzer.extendedResolutionRules`.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16531 from cloud-fan/sql-on-files.
2017-01-10 21:33:44 -08:00
Shixiong Zhu bc6c56e940 [SPARK-19140][SS] Allow update mode for non-aggregation streaming queries
## What changes were proposed in this pull request?

This PR allow update mode for non-aggregation streaming queries. It will be same as the append mode if a query has no aggregations.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16520 from zsxwing/update-without-agg.
2017-01-10 17:58:11 -08:00
Dongjoon Hyun a2c6adcc5d
[SPARK-18857][SQL] Don't use Iterator.duplicate for incrementalCollect in Thrift Server
## What changes were proposed in this pull request?

To support `FETCH_FIRST`, SPARK-16563 used Scala `Iterator.duplicate`. However,
Scala `Iterator.duplicate` uses a **queue to buffer all items between both iterators**,
this causes GC and hangs for queries with large number of rows. We should not use this,
especially for `spark.sql.thriftServer.incrementalCollect`.

https://github.com/scala/scala/blob/2.12.x/src/library/scala/collection/Iterator.scala#L1262-L1300

## How was this patch tested?

Pass the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16440 from dongjoon-hyun/SPARK-18857.
2017-01-10 13:27:55 +00:00
hyukjinkwon 4e27578faa
[SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed due to path and resource-not-closed problems on Windows
## What changes were proposed in this pull request?

This PR proposes to fix all the test failures identified by testing with AppVeyor.

**Scala - aborted tests**

```
WindowQuerySuite:
  Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.WindowQuerySuite *** ABORTED *** (156 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive   argetscala-2.11   est-classesdatafilespart_tiny.txt;

OrcSourceSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.orc.OrcSourceSuite *** ABORTED *** (62 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);

ParquetMetastoreSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetMetastoreSuite *** ABORTED *** (4 seconds, 703 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);

ParquetSourceSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetSourceSuite *** ABORTED *** (3 seconds, 907 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-581a6575-454f-4f21-a516-a07f95266143;

KafkaRDDSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaRDDSuite *** ABORTED *** (5 seconds, 212 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-4722304d-213e-4296-b556-951df1a46807

DirectKafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.DirectKafkaStreamSuite *** ABORTED *** (7 seconds, 127 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d0d3eba7-4215-4e10-b40e-bb797e89338e
   at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010)

ReliableKafkaStreamSuite
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.ReliableKafkaStreamSuite *** ABORTED *** (5 seconds, 498 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d33e45a0-287e-4bed-acae-ca809a89d888

KafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaStreamSuite *** ABORTED *** (2 seconds, 892 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-59c9d169-5a56-4519-9ef0-cefdbd3f2e6c

KafkaClusterSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaClusterSuite *** ABORTED *** (1 second, 690 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-3ef402b0-8689-4a60-85ae-e41e274f179d

DirectKafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.DirectKafkaStreamSuite *** ABORTED *** (59 seconds, 626 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-426107da-68cf-4d94-b0d6-1f428f1c53f6

KafkaRDDSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.KafkaRDDSuite *** ABORTED *** (2 minutes, 6 seconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-b9ce7929-5dae-46ab-a0c4-9ef6f58fbc2
```

**Java - failed tests**

```
Test org.apache.spark.streaming.kafka.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-1cee32f4-4390-4321-82c9-e8616b3f0fb0, took 9.61 sec

Test org.apache.spark.streaming.kafka.JavaKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-f42695dd-242e-4b07-847c-f299b8e4676e, took 11.797 sec

Test org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-85c0d062-78cf-459c-a2dd-7973572101ce, took 1.581 sec

Test org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-49eb6b5c-8366-47a6-83f2-80c443c48280, took 17.895 sec

org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-898cf826-d636-4b1c-a61a-c12a364c02e7, took 8.858 sec
```

**Scala - failed tests**

```
PartitionProviderCompatibilitySuite:
 - insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (828 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-bb6337b9-4f99-45ab-ad2c-a787ab965c09

 - SPARK-18635 special chars in partition values - partition management true *** FAILED *** (5 seconds, 360 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);

 - SPARK-18635 special chars in partition values - partition management false *** FAILED *** (141 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);
```

```
UtilsSuite:
 - reading offset bytes of a file (compressed) *** FAILED *** (0 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ecb2b7d5-db8b-43a7-b268-1bf242b5a491

 - reading offset bytes across multiple files (compressed) *** FAILED *** (0 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-25cc47a8-1faa-4da5-8862-cf174df63ce0
```

```
StatisticsSuite:
 - MetastoreRelations fallback to HDFS for size estimation *** FAILED *** (110 milliseconds)
   org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 'csv_table' not found in database 'default';
```

```
SQLQuerySuite:
 - permanent UDTF *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count_temp'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 24

 - describe functions - user defined functions *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7

 - CTAS without serde with location *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-ed673d73-edfc-404e-829e-2e2b9725d94e/c1

 - derived from Hive query file: drop_database_removes_partition_dirs.q *** FAILED *** (47 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_database_removes_partition_dirs_table

 - derived from Hive query file: drop_table_removes_partition_dirs.q *** FAILED *** (0 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_table_removes_partition_dirs_table2

 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH *** FAILED *** (109 milliseconds)
   java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/sql/hive/projectsspark	arget	mpspark-1a122f8c-dfb3-46c4-bab1-f30764baee0e/*part-r*
```

```
HiveDDLSuite:
 - drop external tables in default 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);

 - add/drop partitions - external table *** 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);

 - create/drop database - location without pre-created directory *** 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);

 - create/drop database - location with pre-created directory *** FAILED *** (32 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);

 - drop database containing tables - CASCADE *** FAILED *** (94 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop an empty database - CASCADE *** FAILED *** (63 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop database containing tables - RESTRICT *** FAILED *** (47 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop an empty database - RESTRICT *** FAILED *** (47 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - CREATE TABLE LIKE an external data source table *** FAILED *** (140 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-c5eba16d-07ae-4186-95bb-21c5811cf888;

 - CREATE TABLE LIKE an external Hive serde table *** 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);

 - desc table for data source table - no user-defined schema *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-e8bf5bf5-721a-4cbe-9d6	at scala.collection.immutable.List.foreach(List.scala:381)d-5543a8301c1d;
```

```
MetastoreDataSourcesSuite
 - CTAS: persisted bucketed data source table *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
ShowCreateTableSuite:
 - simple external hive table *** 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);
```

```
PartitionedTablePerfStatsSuite:
 - hive table: partitioned pruned table reports only selected files *** FAILED *** (313 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);

 - datasource table: partitioned pruned table reports only selected files *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-311f45f8-d064-4023-a4bb-e28235bff64d;

 - hive table: lazy partition pruning reads only necessary partition data *** FAILED *** (203 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);

 - datasource table: lazy partition pruning reads only necessary partition data *** FAILED *** (187 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-fde874ca-66bd-4d0b-a40f-a043b65bf957;

 - hive table: lazy partition pruning with file status caching enabled *** FAILED *** (188 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);

 - datasource table: lazy partition pruning with file status caching enabled *** FAILED *** (187 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-e6d20183-dd68-4145-acbe-4a509849accd;

 - hive table: file status caching respects refresh table and refreshByPath *** FAILED *** (172 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);

 - datasource table: file status caching respects refresh table and refreshByPath *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-8b2c9651-2adf-4d58-874f-659007e21463;

 - hive table: file status cache respects size limit *** FAILED *** (219 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);

 - datasource table: file status cache respects size limit *** FAILED *** (171 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-7835ab57-cb48-4d2c-bb1d-b46d5a4c47e4;

 - datasource table: table setup does not scan filesystem *** FAILED *** (266 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-20598d76-c004-42a7-8061-6c56f0eda5e2;

 - hive table: table setup does not scan filesystem *** FAILED *** (266 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: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 281 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);

 - datasource table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 422 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4cfed321-4d1d-4b48-8d34-5c169afff383;

 - hive table: files read and cached when filesource partition management is off *** FAILED *** (234 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);

 - datasource table: all partition data cached in memory when partition management is off *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4bcc0398-15c9-4f6a-811e-12d40f3eec12;

 - SPARK-18700: table loaded only once even when resolved concurrently *** FAILED *** (1 second, 266 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);
```

```
HiveSparkSubmitSuite:
 - temporary Hive UDF: define a UDF and use it *** FAILED *** (2 seconds, 94 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - permanent Hive UDF: define a UDF and use it *** FAILED *** (281 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - permanent Hive UDF: use a already defined permanent function *** FAILED *** (718 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8368: includes jars passed in through --jars *** FAILED *** (3 seconds, 521 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8020: set sql conf in spark conf *** FAILED *** (0 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8489: MissingRequirementError during reflection *** FAILED *** (94 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-9757 Persist Parquet relation with decimal column *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-11009 fix wrong result of Window function in cluster mode *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-14244 fix window partition size attribute binding failure *** FAILED *** (78 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - set spark.sql.warehouse.dir *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - set hive.metastore.warehouse.dir *** FAILED *** (15 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-16901: set javax.jdo.option.ConnectionURL *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-18360: default table path of tables in default database should depend on the location of default database *** FAILED *** (15 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```

```
UtilsSuite:
 - resolveURIs with multiple paths *** FAILED *** (0 milliseconds)
   ".../jar3,file:/C:/pi.py[%23]py.pi,file:/C:/path%..." did not equal ".../jar3,file:/C:/pi.py[#]py.pi,file:/C:/path%..." (UtilsSuite.scala:468)
```

```
CheckpointSuite:
 - recovery with file input stream *** FAILED *** (10 seconds, 205 milliseconds)
   The code passed to eventually never returned normally. Attempted 660 times over 10.014272499999999 seconds. Last failure message: Unexpected internal error near index 1
   \
    ^. (CheckpointSuite.scala:680)
```

## How was this patch tested?

Manually via AppVeyor as below:

**Scala - aborted tests**

```
WindowQuerySuite - all passed
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
  org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
ParquetMetastoreSuite - all passed
ParquetSourceSuite - all passed
KafkaRDDSuite - all passed
DirectKafkaStreamSuite - all passed
ReliableKafkaStreamSuite - all passed
KafkaStreamSuite - all passed
KafkaClusterSuite - all passed
DirectKafkaStreamSuite - all passed
KafkaRDDSuite - all passed
```

**Java - failed tests**

```
org.apache.spark.streaming.kafka.JavaKafkaRDDSuite - all passed
org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka.JavaKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite - all passed
```

**Scala - failed tests**

```
PartitionProviderCompatibilitySuite:
- insert overwrite partition of new datasource table overwrites just partition (1 second, 953 milliseconds)
- SPARK-18635 special chars in partition values - partition management true (6 seconds, 31 milliseconds)
- SPARK-18635 special chars in partition values - partition management false (4 seconds, 578 milliseconds)
```

```
UtilsSuite:
- reading offset bytes of a file (compressed) (203 milliseconds)
- reading offset bytes across multiple files (compressed) (0 milliseconds)
```

```
StatisticsSuite:
- MetastoreRelations fallback to HDFS for size estimation (94 milliseconds)
```

```
SQLQuerySuite:
 - permanent UDTF (407 milliseconds)
 - describe functions - user defined functions (441 milliseconds)
 - CTAS without serde with location (2 seconds, 831 milliseconds)
 - derived from Hive query file: drop_database_removes_partition_dirs.q (734 milliseconds)
 - derived from Hive query file: drop_table_removes_partition_dirs.q (563 milliseconds)
 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH (453 milliseconds)
```

```
HiveDDLSuite:
 - drop external tables in default database (3 seconds, 5 milliseconds)
 - add/drop partitions - external table (2 seconds, 750 milliseconds)
 - create/drop database - location without pre-created directory (500 milliseconds)
 - create/drop database - location with pre-created directory (407 milliseconds)
 - drop database containing tables - CASCADE (453 milliseconds)
 - drop an empty database - CASCADE (375 milliseconds)
 - drop database containing tables - RESTRICT (328 milliseconds)
 - drop an empty database - RESTRICT (391 milliseconds)
 - CREATE TABLE LIKE an external data source table (953 milliseconds)
 - CREATE TABLE LIKE an external Hive serde table (3 seconds, 782 milliseconds)
 - desc table for data source table - no user-defined schema (1 second, 150 milliseconds)
```

```
MetastoreDataSourcesSuite
 - CTAS: persisted bucketed data source table (875 milliseconds)
```

```
ShowCreateTableSuite:
 - simple external hive table (78 milliseconds)
```

```
PartitionedTablePerfStatsSuite:
 - hive table: partitioned pruned table reports only selected files (1 second, 109 milliseconds)
- datasource table: partitioned pruned table reports only selected files (860 milliseconds)
 - hive table: lazy partition pruning reads only necessary partition data (859 milliseconds)
 - datasource table: lazy partition pruning reads only necessary partition data (1 second, 219 milliseconds)
 - hive table: lazy partition pruning with file status caching enabled (875 milliseconds)
 - datasource table: lazy partition pruning with file status caching enabled (890 milliseconds)
 - hive table: file status caching respects refresh table and refreshByPath (922 milliseconds)
 - datasource table: file status caching respects refresh table and refreshByPath (640 milliseconds)
 - hive table: file status cache respects size limit (469 milliseconds)
 - datasource table: file status cache respects size limit (453 milliseconds)
 - datasource table: table setup does not scan filesystem (328 milliseconds)
 - hive table: table setup does not scan filesystem (313 milliseconds)
 - hive table: num hive client calls does not scale with partition count (5 seconds, 431 milliseconds)
 - datasource table: num hive client calls does not scale with partition count (4 seconds, 79 milliseconds)
 - hive table: files read and cached when filesource partition management is off (656 milliseconds)
 - datasource table: all partition data cached in memory when partition management is off (484 milliseconds)
 - SPARK-18700: table loaded only once even when resolved concurrently (2 seconds, 578 milliseconds)
```

```
HiveSparkSubmitSuite:
 - temporary Hive UDF: define a UDF and use it (1 second, 745 milliseconds)
 - permanent Hive UDF: define a UDF and use it (406 milliseconds)
 - permanent Hive UDF: use a already defined permanent function (375 milliseconds)
 - SPARK-8368: includes jars passed in through --jars (391 milliseconds)
 - SPARK-8020: set sql conf in spark conf (156 milliseconds)
 - SPARK-8489: MissingRequirementError during reflection (187 milliseconds)
 - SPARK-9757 Persist Parquet relation with decimal column (157 milliseconds)
 - SPARK-11009 fix wrong result of Window function in cluster mode (156 milliseconds)
 - SPARK-14244 fix window partition size attribute binding failure (156 milliseconds)
 - set spark.sql.warehouse.dir (172 milliseconds)
 - set hive.metastore.warehouse.dir (156 milliseconds)
 - SPARK-16901: set javax.jdo.option.ConnectionURL (157 milliseconds)
 - SPARK-18360: default table path of tables in default database should depend on the location of default database (172 milliseconds)
```

```
UtilsSuite:
 - resolveURIs with multiple paths (0 milliseconds)
```

```
CheckpointSuite:
 - recovery with file input stream (4 seconds, 452 milliseconds)
```

Note: after resolving the aborted tests, there is a test failure identified as below:

```
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
  org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
```

This does not look due to this problem so this PR does not fix it here.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16451 from HyukjinKwon/all-path-resource-fixes.
2017-01-10 13:19:21 +00:00
Wenchen Fan b0319c2ecb [SPARK-19107][SQL] support creating hive table with DataFrameWriter and Catalog
## What changes were proposed in this pull request?

After unifying the CREATE TABLE syntax in https://github.com/apache/spark/pull/16296, it's pretty easy to support creating hive table with `DataFrameWriter` and `Catalog` now.

This PR basically just removes the hive provider check in `DataFrameWriter.saveAsTable` and `Catalog.createExternalTable`, and add tests.

## How was this patch tested?

new tests in `HiveDDLSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16487 from cloud-fan/hive-table.
2017-01-10 19:26:51 +08:00
Burak Yavuz faabe69cc0 [SPARK-18952] Regex strings not properly escaped in codegen for aggregations
## What changes were proposed in this pull request?

If I use the function regexp_extract, and then in my regex string, use `\`, i.e. escape character, this fails codegen, because the `\` character is not properly escaped when codegen'd.

Example stack trace:
```
/* 059 */     private int maxSteps = 2;
/* 060 */     private int numRows = 0;
/* 061 */     private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add("date_format(window#325.start, yyyy-MM-dd HH:mm)", org.apache.spark.sql.types.DataTypes.StringType)
/* 062 */     .add("regexp_extract(source#310.description, ([a-zA-Z]+)\[.*, 1)", org.apache.spark.sql.types.DataTypes.StringType);
/* 063 */     private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add("sum", org.apache.spark.sql.types.DataTypes.LongType);
/* 064 */     private Object emptyVBase;

...

org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 62, Column 58: Invalid escape sequence
	at org.codehaus.janino.Scanner.scanLiteralCharacter(Scanner.java:918)
	at org.codehaus.janino.Scanner.produce(Scanner.java:604)
	at org.codehaus.janino.Parser.peekRead(Parser.java:3239)
	at org.codehaus.janino.Parser.parseArguments(Parser.java:3055)
	at org.codehaus.janino.Parser.parseSelector(Parser.java:2914)
	at org.codehaus.janino.Parser.parseUnaryExpression(Parser.java:2617)
	at org.codehaus.janino.Parser.parseMultiplicativeExpression(Parser.java:2573)
	at org.codehaus.janino.Parser.parseAdditiveExpression(Parser.java:2552)
```

In the codegend expression, the literal should use `\\` instead of `\`

A similar problem was solved here: https://github.com/apache/spark/pull/15156.

## How was this patch tested?

Regression test in `DataFrameAggregationSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16361 from brkyvz/reg-break.
2017-01-09 14:25:38 -08:00
anabranch 19d9d4c855 [SPARK-19126][DOCS] Update Join Documentation Across Languages
## What changes were proposed in this pull request?

- [X] Make sure all join types are clearly mentioned
- [X] Make join labeling/style consistent
- [X] Make join label ordering docs the same
- [X] Improve join documentation according to above for Scala
- [X] Improve join documentation according to above for Python
- [X] Improve join documentation according to above for R

## How was this patch tested?
No tests b/c docs.

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

Author: anabranch <wac.chambers@gmail.com>

Closes #16504 from anabranch/SPARK-19126.
2017-01-08 20:37:46 -08:00
anabranch 1f6ded6455 [SPARK-19127][DOCS] Update Rank Function Documentation
## What changes were proposed in this pull request?

- [X] Fix inconsistencies in function reference for dense rank and dense
- [X] Make all languages equivalent in their reference to `dense_rank` and `rank`.

## How was this patch tested?

N/A for docs.

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

Author: anabranch <wac.chambers@gmail.com>

Closes #16505 from anabranch/SPARK-19127.
2017-01-08 17:53:53 -08:00
Dilip Biswal 4351e62207 [SPARK-19093][SQL] Cached tables are not used in SubqueryExpression
## What changes were proposed in this pull request?
Consider the plans inside subquery expressions while looking up cache manager to make
use of cached data. Currently CacheManager.useCachedData does not consider the
subquery expressions in the plan.

SQL
```
select * from rows where not exists (select * from rows)
```
Before the fix
```
== Optimized Logical Plan ==
Join LeftAnti
:- InMemoryRelation [_1#3775, _2#3776], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
:     +- *FileScan parquet [_1#3775,_2#3776] Batched: true, Format: Parquet, Location: InMemoryFileIndex[dbfs:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string>
+- Project [_1#3775 AS _1#3775#4001, _2#3776 AS _2#3776#4002]
   +- Relation[_1#3775,_2#3776] parquet
```

After
```
== Optimized Logical Plan ==
Join LeftAnti
:- InMemoryRelation [_1#256, _2#257], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
:     +- *FileScan parquet [_1#256,_2#257] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string>
+- Project [_1#256 AS _1#256#298, _2#257 AS _2#257#299]
   +- InMemoryRelation [_1#256, _2#257], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
         +- *FileScan parquet [_1#256,_2#257] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string>
```

Query2
```
 SELECT * FROM t1
 WHERE
 c1 IN (SELECT c1 FROM t2 WHERE c1 IN (SELECT c1 FROM t3 WHERE c1 = 1))
```
Before
```
== Analyzed Logical Plan ==
c1: int
Project [c1#3]
+- Filter predicate-subquery#47 [(c1#3 = c1#10)]
   :  +- Project [c1#10]
   :     +- Filter predicate-subquery#46 [(c1#10 = c1#17)]
   :        :  +- Project [c1#17]
   :        :     +- Filter (c1#17 = 1)
   :        :        +- SubqueryAlias t3, `t3`
   :        :           +- Project [value#15 AS c1#17]
   :        :              +- LocalRelation [value#15]
   :        +- SubqueryAlias t2, `t2`
   :           +- Project [value#8 AS c1#10]
   :              +- LocalRelation [value#8]
   +- SubqueryAlias t1, `t1`
      +- Project [value#1 AS c1#3]
         +- LocalRelation [value#1]

== Optimized Logical Plan ==
Join LeftSemi, (c1#3 = c1#10)
:- InMemoryRelation [c1#3], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1
:     +- LocalTableScan [c1#3]
+- Project [value#8 AS c1#10]
   +- Join LeftSemi, (value#8 = c1#17)
      :- LocalRelation [value#8]
      +- Project [value#15 AS c1#17]
         +- Filter (value#15 = 1)
            +- LocalRelation [value#15]

```
After
```
== Analyzed Logical Plan ==
c1: int
Project [c1#3]
+- Filter predicate-subquery#47 [(c1#3 = c1#10)]
   :  +- Project [c1#10]
   :     +- Filter predicate-subquery#46 [(c1#10 = c1#17)]
   :        :  +- Project [c1#17]
   :        :     +- Filter (c1#17 = 1)
   :        :        +- SubqueryAlias t3, `t3`
   :        :           +- Project [value#15 AS c1#17]
   :        :              +- LocalRelation [value#15]
   :        +- SubqueryAlias t2, `t2`
   :           +- Project [value#8 AS c1#10]
   :              +- LocalRelation [value#8]
   +- SubqueryAlias t1, `t1`
      +- Project [value#1 AS c1#3]
         +- LocalRelation [value#1]

== Optimized Logical Plan ==
Join LeftSemi, (c1#3 = c1#10)
:- InMemoryRelation [c1#3], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1
:     +- LocalTableScan [c1#3]
+- Join LeftSemi, (c1#10 = c1#17)
   :- InMemoryRelation [c1#10], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t2
   :     +- LocalTableScan [c1#10]
   +- Filter (c1#17 = 1)
      +- InMemoryRelation [c1#17], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1
            +- LocalTableScan [c1#3]
```
## How was this patch tested?
Added new tests in CachedTableSuite.

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

Closes #16493 from dilipbiswal/SPARK-19093.
2017-01-08 23:09:07 +01:00
Wenchen Fan b3d39620c5 [SPARK-19085][SQL] cleanup OutputWriterFactory and OutputWriter
## What changes were proposed in this pull request?

`OutputWriterFactory`/`OutputWriter` are internal interfaces and we can remove some unnecessary APIs:
1. `OutputWriterFactory.newWriter(path: String)`: no one calls it and no one implements it.
2. `OutputWriter.write(row: Row)`: during execution we only call `writeInternal`, which is weird as `OutputWriter` is already an internal interface. We should rename `writeInternal` to `write` and remove `def write(row: Row)` and it's related converter code. All implementations should just implement `def write(row: InternalRow)`

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16479 from cloud-fan/hive-writer.
2017-01-08 00:42:09 +08:00
Tathagata Das b59cddaba0 [SPARK-19074][SS][DOCS] Updated Structured Streaming Programming Guide for update mode and source/sink options
## What changes were proposed in this pull request?

Updates
- Updated Late Data Handling section by adding a figure for Update Mode. Its more intuitive to explain late data handling with Update Mode, so I added the new figure before the Append Mode figure.
- Updated Output Modes section with Update mode
- Added options for all the sources and sinks

---------------------------
---------------------------

![image](https://cloud.githubusercontent.com/assets/663212/21665176/f150b224-d29f-11e6-8372-14d32da21db9.png)

---------------------------
---------------------------
<img width="931" alt="screen shot 2017-01-03 at 6 09 11 pm" src="https://cloud.githubusercontent.com/assets/663212/21629740/d21c9bb8-d1df-11e6-915b-488a59589fa6.png">
<img width="933" alt="screen shot 2017-01-03 at 6 10 00 pm" src="https://cloud.githubusercontent.com/assets/663212/21629749/e22bdabe-d1df-11e6-86d3-7e51d2f28dbc.png">

---------------------------
---------------------------
![image](https://cloud.githubusercontent.com/assets/663212/21665200/108e18fc-d2a0-11e6-8640-af598cab090b.png)
![image](https://cloud.githubusercontent.com/assets/663212/21665148/cfe414fa-d29f-11e6-9baa-4124ccbab093.png)
![image](https://cloud.githubusercontent.com/assets/663212/21665226/2e8f39e4-d2a0-11e6-85b1-7657e2df5491.png)

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

Closes #16468 from tdas/SPARK-19074.
2017-01-06 11:29:01 -08:00
Michal Senkyr 903bb8e8a2 [SPARK-16792][SQL] Dataset containing a Case Class with a List type causes a CompileException (converting sequence to list)
## What changes were proposed in this pull request?

Added a `to` call at the end of the code generated by `ScalaReflection.deserializerFor` if the requested type is not a supertype of `WrappedArray[_]` that uses `CanBuildFrom[_, _, _]` to convert result into an arbitrary subtype of `Seq[_]`.

Care was taken to preserve the original deserialization where it is possible to avoid the overhead of conversion in cases where it is not needed

`ScalaReflection.serializerFor` could already be used to serialize any `Seq[_]` so it was not altered

`SQLImplicits` had to be altered and new implicit encoders added to permit serialization of other sequence types

Also fixes [SPARK-16815] Dataset[List[T]] leads to ArrayStoreException

## How was this patch tested?
```bash
./build/mvn -DskipTests clean package && ./dev/run-tests
```

Also manual execution of the following sets of commands in the Spark shell:
```scala
case class TestCC(key: Int, letters: List[String])

val ds1 = sc.makeRDD(Seq(
(List("D")),
(List("S","H")),
(List("F","H")),
(List("D","L","L"))
)).map(x=>(x.length,x)).toDF("key","letters").as[TestCC]

val test1=ds1.map{_.key}
test1.show
```

```scala
case class X(l: List[String])
spark.createDataset(Seq(List("A"))).map(X).show
```

```scala
spark.sqlContext.createDataset(sc.parallelize(List(1) :: Nil)).collect
```

After adding arbitrary sequence support also tested with the following commands:

```scala
case class QueueClass(q: scala.collection.immutable.Queue[Int])

spark.createDataset(Seq(List(1,2,3))).map(x => QueueClass(scala.collection.immutable.Queue(x: _*))).map(_.q.dequeue).collect
```

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

Closes #16240 from michalsenkyr/sql-caseclass-list-fix.
2017-01-06 15:05:20 +08:00
Wenchen Fan cca945b6aa [SPARK-18885][SQL] unify CREATE TABLE syntax for data source and hive serde tables
## What changes were proposed in this pull request?

Today we have different syntax to create data source or hive serde tables, we should unify them to not confuse users and step forward to make hive a data source.

Please read https://issues.apache.org/jira/secure/attachment/12843835/CREATE-TABLE.pdf for  details.

TODO(for follow-up PRs):
1. TBLPROPERTIES is not added to the new syntax, we should decide if we wanna add it later.
2. `SHOW CREATE TABLE` should be updated to use the new syntax.
3. we should decide if we wanna change the behavior of `SET LOCATION`.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16296 from cloud-fan/create-table.
2017-01-05 17:40:27 -08:00
Wenchen Fan 30345c43b7 [SPARK-19058][SQL] fix partition related behaviors with DataFrameWriter.saveAsTable
## What changes were proposed in this pull request?

When we append data to a partitioned table with `DataFrameWriter.saveAsTable`, there are 2 issues:
1. doesn't work when the partition has custom location.
2. will recover all partitions

This PR fixes them by moving the special partition handling code from `DataSourceAnalysis` to `InsertIntoHadoopFsRelationCommand`, so that the `DataFrameWriter.saveAsTable` code path can also benefit from it.

## How was this patch tested?

newly added regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16460 from cloud-fan/append.
2017-01-05 14:11:05 +08:00
Herman van Hovell 4262fb0d55 [SPARK-19070] Clean-up dataset actions
## What changes were proposed in this pull request?
Dataset actions currently spin off a new `Dataframe` only to track query execution. This PR simplifies this code path by using the `Dataset.queryExecution` directly. This PR also merges the typed and untyped action evaluation paths.

## How was this patch tested?
Existing tests.

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

Closes #16466 from hvanhovell/SPARK-19070.
2017-01-04 23:47:58 +08:00
Niranjan Padmanabhan a1e40b1f5d
[MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.

## How was this patch tested?
N/A since only docs or comments were updated.

Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com>

Closes #16455 from neurons/np.structure_streaming_doc.
2017-01-04 15:07:29 +00:00
Wenchen Fan 101556d0fa [SPARK-19060][SQL] remove the supportsPartial flag in AggregateFunction
## What changes were proposed in this pull request?

Now all aggregation functions support partial aggregate, we can remove the `supportsPartual` flag in `AggregateFunction`

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16461 from cloud-fan/partial.
2017-01-04 12:46:30 +01:00
gatorsmile b67b35f76b [SPARK-19048][SQL] Delete Partition Location when Dropping Managed Partitioned Tables in InMemoryCatalog
### What changes were proposed in this pull request?
The data in the managed table should be deleted after table is dropped. However, if the partition location is not under the location of the partitioned table, it is not deleted as expected. Users can specify any location for the partition when they adding a partition.

This PR is to delete partition location when dropping managed partitioned tables stored in `InMemoryCatalog`.

### How was this patch tested?
Added test cases for both HiveExternalCatalog and InMemoryCatalog

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16448 from gatorsmile/unsetSerdeProp.
2017-01-03 11:43:47 -08:00
Dongjoon Hyun 7a2b5f93bc [SPARK-18877][SQL] CSVInferSchema.inferField on DecimalType should find a common type with typeSoFar
## What changes were proposed in this pull request?

CSV type inferencing causes `IllegalArgumentException` on decimal numbers with heterogeneous precisions and scales because the current logic uses the last decimal type in a **partition**. Specifically, `inferRowType`, the **seqOp** of **aggregate**, returns the last decimal type. This PR fixes it to use `findTightestCommonType`.

**decimal.csv**
```
9.03E+12
1.19E+11
```

**BEFORE**
```scala
scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").printSchema
root
 |-- _c0: decimal(3,-9) (nullable = true)

scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").show
16/12/16 14:32:49 ERROR Executor: Exception in task 0.0 in stage 4.0 (TID 4)
java.lang.IllegalArgumentException: requirement failed: Decimal precision 4 exceeds max precision 3
```

**AFTER**
```scala
scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").printSchema
root
 |-- _c0: decimal(4,-9) (nullable = true)

scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").show
+---------+
|      _c0|
+---------+
|9.030E+12|
| 1.19E+11|
+---------+
```

## How was this patch tested?

Pass the newly add test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16320 from dongjoon-hyun/SPARK-18877.
2017-01-03 23:06:50 +08:00
Zhenhua Wang ae83c21125 [SPARK-18998][SQL] Add a cbo conf to switch between default statistics and estimated statistics
## What changes were proposed in this pull request?

We add a cbo configuration to switch between default stats and estimated stats.
We also define a new statistics method `planStats` in LogicalPlan with conf as its parameter, in order to pass the cbo switch and other estimation related configurations in the future. `planStats` is used on the caller sides (i.e. in Optimizer and Strategies) to make transformation decisions based on stats.

## How was this patch tested?

Add a test case using a dummy LogicalPlan.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16401 from wzhfy/cboSwitch.
2017-01-03 12:19:52 +08:00
Dongjoon Hyun b85e29437d [SPARK-18123][SQL] Use db column names instead of RDD column ones during JDBC Writing
## What changes were proposed in this pull request?

Apache Spark supports the following cases **by quoting RDD column names** while saving through JDBC.
- Allow reserved keyword as a column name, e.g., 'order'.
- Allow mixed-case colume names like the following, e.g., `[a: int, A: int]`.

  ``` scala
  scala> val df = sql("select 1 a, 1 A")
  df: org.apache.spark.sql.DataFrame = [a: int, A: int]
  ...
  scala> df.write.mode("overwrite").format("jdbc").options(option).save()
  scala> df.write.mode("append").format("jdbc").options(option).save()
  ```

This PR aims to use **database column names** instead of RDD column ones in order to support the following additionally.
Note that this case succeeds with `MySQL`, but fails on `Postgres`/`Oracle` before.

``` scala
val df1 = sql("select 1 a")
val df2 = sql("select 1 A")
...
df1.write.mode("overwrite").format("jdbc").options(option).save()
df2.write.mode("append").format("jdbc").options(option).save()
```
## How was this patch tested?

Pass the Jenkins test with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #15664 from dongjoon-hyun/SPARK-18123.
2016-12-30 10:27:14 -08:00
Dongjoon Hyun 752d9eeb9b [SPARK-19012][SQL] Fix createTempViewCommand to throw AnalysisException instead of ParseException
## What changes were proposed in this pull request?

Currently, `createTempView`, `createOrReplaceTempView`, and `createGlobalTempView` show `ParseExceptions` on invalid table names. We had better show better error message. Also, this PR also adds and updates the missing description on the API docs correctly.

**BEFORE**
```
scala> spark.range(10).createOrReplaceTempView("11111")
org.apache.spark.sql.catalyst.parser.ParseException:
mismatched input '11111' expecting {'SELECT', 'FROM', 'ADD', ...}(line 1, pos 0)

== SQL ==
11111
...
```

**AFTER**
```
scala> spark.range(10).createOrReplaceTempView("11111")
org.apache.spark.sql.AnalysisException: Invalid view name: 11111;
...
```

## How was this patch tested?

Pass the Jenkins with updated a test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16427 from dongjoon-hyun/SPARK-19012.
2016-12-29 21:22:13 +01:00
Wenchen Fan 7d19b6ab7d [SPARK-18567][SQL] Simplify CreateDataSourceTableAsSelectCommand
## What changes were proposed in this pull request?

The `CreateDataSourceTableAsSelectCommand` is quite complex now, as it has a lot of work to do if the table already exists:

1. throw exception if we don't want to ignore it.
2. do some check and adjust the schema if we want to append data.
3. drop the table and create it again if we want to overwrite.

The work 2 and 3 should be done by analyzer, so that we can also apply it to hive tables.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15996 from cloud-fan/append.
2016-12-28 21:50:21 -08:00
Kazuaki Ishizaki 93f35569fd [SPARK-16213][SQL] Reduce runtime overhead of a program that creates an primitive array in DataFrame
## What changes were proposed in this pull request?

This PR reduces runtime overhead of a program the creates an primitive array in DataFrame by using the similar approach to #15044. Generated code performs boxing operation in an assignment from InternalRow to an `Object[]` temporary array (at Lines 051 and 061 in the generated code before without this PR). If we know that type of array elements is primitive, we apply the following optimizations:
1. Eliminate a pair of `isNullAt()` and a null assignment
2. Allocate an primitive array instead of `Object[]` (eliminate boxing operations)
3. Create `UnsafeArrayData` by using `UnsafeArrayWriter` to keep a primitive array in a row format instead of doing non-lightweight operations in constructor of `GenericArrayData`
The PR also performs the same things for `CreateMap`.

Here are performance results of [DataFrame programs](6bf54ec5e2/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala (L83-L112)) by up to 17.9x over without this PR.

```
Without SPARK-16043
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Read a primitive array in DataFrame:     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                           3805 / 4150          0.0      507308.9       1.0X
Double                                        3593 / 3852          0.0      479056.9       1.1X

With SPARK-16043
Read a primitive array in DataFrame:     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            213 /  271          0.0       28387.5       1.0X
Double                                         204 /  223          0.0       27250.9       1.0X
```
Note : #15780 is enabled for these measurements

An motivating example

``` java
val df = sparkContext.parallelize(Seq(0.0d, 1.0d), 1).toDF
df.selectExpr("Array(value + 1.1d, value + 2.2d)").show
```

Generated code without this PR

``` java
/* 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 UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */   private Object[] project_values;
/* 013 */   private UnsafeRow project_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter;
/* 017 */
/* 018 */   public GeneratedIterator(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     inputadapter_input = inputs[0];
/* 026 */     serializefromobject_result = new UnsafeRow(1);
/* 027 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 028 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 029 */     this.project_values = null;
/* 030 */     project_result = new UnsafeRow(1);
/* 031 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32);
/* 032 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 033 */     this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 034 */
/* 035 */   }
/* 036 */
/* 037 */   protected void processNext() throws java.io.IOException {
/* 038 */     while (inputadapter_input.hasNext()) {
/* 039 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 040 */       double inputadapter_value = inputadapter_row.getDouble(0);
/* 041 */
/* 042 */       final boolean project_isNull = false;
/* 043 */       this.project_values = new Object[2];
/* 044 */       boolean project_isNull1 = false;
/* 045 */
/* 046 */       double project_value1 = -1.0;
/* 047 */       project_value1 = inputadapter_value + 1.1D;
/* 048 */       if (false) {
/* 049 */         project_values[0] = null;
/* 050 */       } else {
/* 051 */         project_values[0] = project_value1;
/* 052 */       }
/* 053 */
/* 054 */       boolean project_isNull4 = false;
/* 055 */
/* 056 */       double project_value4 = -1.0;
/* 057 */       project_value4 = inputadapter_value + 2.2D;
/* 058 */       if (false) {
/* 059 */         project_values[1] = null;
/* 060 */       } else {
/* 061 */         project_values[1] = project_value4;
/* 062 */       }
/* 063 */
/* 064 */       final ArrayData project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_values);
/* 065 */       this.project_values = null;
/* 066 */       project_holder.reset();
/* 067 */
/* 068 */       project_rowWriter.zeroOutNullBytes();
/* 069 */
/* 070 */       if (project_isNull) {
/* 071 */         project_rowWriter.setNullAt(0);
/* 072 */       } else {
/* 073 */         // Remember the current cursor so that we can calculate how many bytes are
/* 074 */         // written later.
/* 075 */         final int project_tmpCursor = project_holder.cursor;
/* 076 */
/* 077 */         if (project_value instanceof UnsafeArrayData) {
/* 078 */           final int project_sizeInBytes = ((UnsafeArrayData) project_value).getSizeInBytes();
/* 079 */           // grow the global buffer before writing data.
/* 080 */           project_holder.grow(project_sizeInBytes);
/* 081 */           ((UnsafeArrayData) project_value).writeToMemory(project_holder.buffer, project_holder.cursor);
/* 082 */           project_holder.cursor += project_sizeInBytes;
/* 083 */
/* 084 */         } else {
/* 085 */           final int project_numElements = project_value.numElements();
/* 086 */           project_arrayWriter.initialize(project_holder, project_numElements, 8);
/* 087 */
/* 088 */           for (int project_index = 0; project_index < project_numElements; project_index++) {
/* 089 */             if (project_value.isNullAt(project_index)) {
/* 090 */               project_arrayWriter.setNullDouble(project_index);
/* 091 */             } else {
/* 092 */               final double project_element = project_value.getDouble(project_index);
/* 093 */               project_arrayWriter.write(project_index, project_element);
/* 094 */             }
/* 095 */           }
/* 096 */         }
/* 097 */
/* 098 */         project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor);
/* 099 */       }
/* 100 */       project_result.setTotalSize(project_holder.totalSize());
/* 101 */       append(project_result);
/* 102 */       if (shouldStop()) return;
/* 103 */     }
/* 104 */   }
/* 105 */ }
```

Generated code with this PR

``` java
/* 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 UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */   private UnsafeArrayData project_arrayData;
/* 013 */   private UnsafeRow project_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter;
/* 017 */
/* 018 */   public GeneratedIterator(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     inputadapter_input = inputs[0];
/* 026 */     serializefromobject_result = new UnsafeRow(1);
/* 027 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 028 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 029 */
/* 030 */     project_result = new UnsafeRow(1);
/* 031 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32);
/* 032 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 033 */     this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 034 */
/* 035 */   }
/* 036 */
/* 037 */   protected void processNext() throws java.io.IOException {
/* 038 */     while (inputadapter_input.hasNext()) {
/* 039 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 040 */       double inputadapter_value = inputadapter_row.getDouble(0);
/* 041 */
/* 042 */       byte[] project_array = new byte[32];
/* 043 */       project_arrayData = new UnsafeArrayData();
/* 044 */       Platform.putLong(project_array, 16, 2);
/* 045 */       project_arrayData.pointTo(project_array, 16, 32);
/* 046 */
/* 047 */       boolean project_isNull1 = false;
/* 048 */
/* 049 */       double project_value1 = -1.0;
/* 050 */       project_value1 = inputadapter_value + 1.1D;
/* 051 */       if (false) {
/* 052 */         project_arrayData.setNullAt(0);
/* 053 */       } else {
/* 054 */         project_arrayData.setDouble(0, project_value1);
/* 055 */       }
/* 056 */
/* 057 */       boolean project_isNull4 = false;
/* 058 */
/* 059 */       double project_value4 = -1.0;
/* 060 */       project_value4 = inputadapter_value + 2.2D;
/* 061 */       if (false) {
/* 062 */         project_arrayData.setNullAt(1);
/* 063 */       } else {
/* 064 */         project_arrayData.setDouble(1, project_value4);
/* 065 */       }
/* 066 */       project_holder.reset();
/* 067 */
/* 068 */       // Remember the current cursor so that we can calculate how many bytes are
/* 069 */       // written later.
/* 070 */       final int project_tmpCursor = project_holder.cursor;
/* 071 */
/* 072 */       if (project_arrayData instanceof UnsafeArrayData) {
/* 073 */         final int project_sizeInBytes = ((UnsafeArrayData) project_arrayData).getSizeInBytes();
/* 074 */         // grow the global buffer before writing data.
/* 075 */         project_holder.grow(project_sizeInBytes);
/* 076 */         ((UnsafeArrayData) project_arrayData).writeToMemory(project_holder.buffer, project_holder.cursor);
/* 077 */         project_holder.cursor += project_sizeInBytes;
/* 078 */
/* 079 */       } else {
/* 080 */         final int project_numElements = project_arrayData.numElements();
/* 081 */         project_arrayWriter.initialize(project_holder, project_numElements, 8);
/* 082 */
/* 083 */         for (int project_index = 0; project_index < project_numElements; project_index++) {
/* 084 */           if (project_arrayData.isNullAt(project_index)) {
/* 085 */             project_arrayWriter.setNullDouble(project_index);
/* 086 */           } else {
/* 087 */             final double project_element = project_arrayData.getDouble(project_index);
/* 088 */             project_arrayWriter.write(project_index, project_element);
/* 089 */           }
/* 090 */         }
/* 091 */       }
/* 092 */
/* 093 */       project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor);
/* 094 */       project_result.setTotalSize(project_holder.totalSize());
/* 095 */       append(project_result);
/* 096 */       if (shouldStop()) return;
/* 097 */     }
/* 098 */   }
/* 099 */ }
```
## How was this patch tested?

Added unit tests into `DataFrameComplexTypeSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #13909 from kiszk/SPARK-16213.
2016-12-29 10:59:37 +08:00
Carson Wang 2a5f52a714
[MINOR][DOC] Fix doc of ForeachWriter to use writeStream
## What changes were proposed in this pull request?

Fix the document of `ForeachWriter` to use `writeStream` instead of `write` for a streaming dataset.

## How was this patch tested?
Docs only.

Author: Carson Wang <carson.wang@intel.com>

Closes #16419 from carsonwang/FixDoc.
2016-12-28 12:12:44 +00:00
uncleGen 76e9bd7488
[SPARK-18960][SQL][SS] Avoid double reading file which is being copied.
## What changes were proposed in this pull request?

In HDFS, when we copy a file into target directory, there will a temporary `._COPY_` file for a period of time. The duration depends on file size. If we do not skip this file, we will may read the same data for two times.

## How was this patch tested?
update unit test

Author: uncleGen <hustyugm@gmail.com>

Closes #16370 from uncleGen/SPARK-18960.
2016-12-28 10:42:47 +00:00
gatorsmile 5ac62043cf [SPARK-18992][SQL] Move spark.sql.hive.thriftServer.singleSession to SQLConf
### What changes were proposed in this pull request?

Since `spark.sql.hive.thriftServer.singleSession` is a configuration of SQL component, this conf can be moved from `SparkConf` to `StaticSQLConf`.

When we introduced `spark.sql.hive.thriftServer.singleSession`, all the SQL configuration are session specific. They can be modified in different sessions.

In Spark 2.1, static SQL configuration is added. It is a perfect fit for `spark.sql.hive.thriftServer.singleSession`. Previously, we did the same move for `spark.sql.warehouse.dir` from `SparkConf` to `StaticSQLConf`

### How was this patch tested?
Added test cases in HiveThriftServer2Suites.scala

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16392 from gatorsmile/hiveThriftServerSingleSession.
2016-12-28 10:16:22 +08:00
Wenchen Fan 8a7db8a608 [SPARK-18980][SQL] implement Aggregator with TypedImperativeAggregate
## What changes were proposed in this pull request?

Currently we implement `Aggregator` with `DeclarativeAggregate`, which will serialize/deserialize the buffer object every time we process an input.

This PR implements `Aggregator` with `TypedImperativeAggregate` and avoids to serialize/deserialize buffer object many times. The benchmark shows we get about 2 times speed up.

For simple buffer object that doesn't need serialization, we still go with `DeclarativeAggregate`, to avoid performance regression.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16383 from cloud-fan/aggregator.
2016-12-26 22:10:20 +08:00
hyukjinkwon d6cbec7598 [SPARK-18943][SQL] Avoid per-record type dispatch in CSV when reading
## What changes were proposed in this pull request?

`CSVRelation.csvParser` does type dispatch for each value in each row. We can prevent this because the schema is already kept in `CSVRelation`.

So, this PR proposes that converters are created first according to the schema, and then apply them to each.

I just ran some small benchmarks as below after resembling the logics in 7c33b0fd05/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala (L170-L178) to test the updated logics.

```scala
test("Benchmark for CSV converter") {
  var numMalformedRecords = 0
  val N = 500 << 12
  val schema = StructType(
    StructField("a", StringType) ::
    StructField("b", StringType) ::
    StructField("c", StringType) ::
    StructField("d", StringType) :: Nil)

  val row = Array("1.0", "test", "2015-08-20 14:57:00", "FALSE")
  val data = spark.sparkContext.parallelize(List.fill(N)(row))
  val parser = CSVRelation.csvParser(schema, schema.fieldNames, CSVOptions())

  val benchmark = new Benchmark("CSV converter", N)
  benchmark.addCase("cast CSV string tokens", 10) { _ =>
    data.flatMap { recordTokens =>
      parser(recordTokens, numMalformedRecords)
    }.collect()
  }
  benchmark.run()
}
```

**Before**

```
CSV converter:                           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
cast CSV string tokens                        1061 / 1130          1.9         517.9       1.0X
```

**After**

```
CSV converter:                           Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
cast CSV string tokens                         940 / 1011          2.2         459.2       1.0X
```

## How was this patch tested?

Tests in `CSVTypeCastSuite` and `CSVRelation`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16351 from HyukjinKwon/type-dispatch.
2016-12-24 23:28:34 +08:00
Liang-Chi Hsieh 07fcbea516
[SPARK-18800][SQL] Correct the assert in UnsafeKVExternalSorter which ensures array size
## What changes were proposed in this pull request?

`UnsafeKVExternalSorter` uses `UnsafeInMemorySorter` to sort the records of `BytesToBytesMap` if it is given a map.

Currently we use the number of keys in `BytesToBytesMap` to determine if the array used for sort is enough or not. We has an assert that ensures the size of the array is enough: `map.numKeys() <= map.getArray().size() / 2`.

However, each record in the map takes two entries in the array, one is record pointer, another is key prefix. So the correct assert should be `map.numKeys() * 2 <= map.getArray().size() / 2`.

## How was this patch tested?

N/A

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

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

Closes #16232 from viirya/SPARK-18800-fix-UnsafeKVExternalSorter.
2016-12-24 12:05:49 +00:00
wangzhenhua 3cff816157 [SPARK-18911][SQL] Define CatalogStatistics to interact with metastore and convert it to Statistics in relations
## What changes were proposed in this pull request?

Statistics in LogicalPlan should use attributes to refer to columns rather than column names, because two columns from two relations can have the same column name. But CatalogTable doesn't have the concepts of attribute or broadcast hint in Statistics. Therefore, putting Statistics in CatalogTable is confusing.

We define a different statistic structure in CatalogTable, which is only responsible for interacting with metastore, and is converted to statistics in LogicalPlan when it is used.

## How was this patch tested?

add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16323 from wzhfy/nameToAttr.
2016-12-24 15:34:44 +08:00
Shixiong Zhu 2246ce88ae [SPARK-18985][SS] Add missing @InterfaceStability.Evolving for Structured Streaming APIs
## What changes were proposed in this pull request?

Add missing InterfaceStability.Evolving for Structured Streaming APIs

## How was this patch tested?

Compiling the codes.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16385 from zsxwing/SPARK-18985.
2016-12-22 16:21:09 -08:00
Reynold Xin 2615100055 [SPARK-18973][SQL] Remove SortPartitions and RedistributeData
## What changes were proposed in this pull request?
SortPartitions and RedistributeData logical operators are not actually used and can be removed. Note that we do have a Sort operator (with global flag false) that subsumed SortPartitions.

## How was this patch tested?
Also updated test cases to reflect the removal.

Author: Reynold Xin <rxin@databricks.com>

Closes #16381 from rxin/SPARK-18973.
2016-12-22 19:35:09 +01:00
hyukjinkwon 76622c661f [SPARK-16975][SQL][FOLLOWUP] Do not duplicately check file paths in data sources implementing FileFormat
## What changes were proposed in this pull request?

This PR cleans up duplicated checking for file paths in implemented data sources and prevent to attempt to list twice in ORC data source.

https://github.com/apache/spark/pull/14585 handles a problem for the partition column name having `_` and the issue itself is resolved correctly. However, it seems the data sources implementing `FileFormat` are validating the paths duplicately. Assuming from the comment in `CSVFileFormat`, `// TODO: Move filtering.`, I guess we don't have to check this duplicately.

   Currently, this seems being filtered in `PartitioningAwareFileIndex.shouldFilterOut` and`PartitioningAwareFileIndex.isDataPath`. So, `FileFormat.inferSchema` will always receive leaf files. For example, running to codes below:

   ``` scala
   spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet")
   spark.read.parquet("/tmp/parquet")
   ```

   gives the paths below without directories but just valid data files:

   ``` bash
   /tmp/parquet/_col=0/part-r-00000-094a8efa-bece-4b50-b54c-7918d1f7b3f8.snappy.parquet
   /tmp/parquet/_col=1/part-r-00000-094a8efa-bece-4b50-b54c-7918d1f7b3f8.snappy.parquet
   /tmp/parquet/_col=2/part-r-00000-25de2b50-225a-4bcf-a2bc-9eb9ed407ef6.snappy.parquet
   ...
   ```

   to `FileFormat.inferSchema`.

## How was this patch tested?

Unit test added in `HadoopFsRelationTest` and related existing tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14627 from HyukjinKwon/SPARK-16975.
2016-12-22 10:00:20 -08:00
Reynold Xin 2e861df96e [DOC] bucketing is applicable to all file-based data sources
## What changes were proposed in this pull request?
Starting Spark 2.1.0, bucketing feature is available for all file-based data sources. This patch fixes some function docs that haven't yet been updated to reflect that.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #16349 from rxin/ds-doc.
2016-12-21 23:46:33 -08:00
Reynold Xin 7c5b7b3a2e [SQL] Minor readability improvement for partition handling code
## What changes were proposed in this pull request?
This patch includes minor changes to improve readability for partition handling code. I'm in the middle of implementing some new feature and found some naming / implicit type inference not as intuitive.

## How was this patch tested?
This patch should have no semantic change and the changes should be covered by existing test cases.

Author: Reynold Xin <rxin@databricks.com>

Closes #16378 from rxin/minor-fix.
2016-12-22 15:29:56 +08:00
Shixiong Zhu ff7d82a207 [SPARK-18908][SS] Creating StreamingQueryException should check if logicalPlan is created
## What changes were proposed in this pull request?

This PR audits places using `logicalPlan` in StreamExecution and ensures they all handles the case that `logicalPlan` cannot be created.

In addition, this PR also fixes the following issues in `StreamingQueryException`:
- `StreamingQueryException` and `StreamExecution` are cycle-dependent because in the `StreamingQueryException`'s constructor, it calls `StreamExecution`'s `toDebugString` which uses `StreamingQueryException`. Hence it will output `null` value in the error message.
- Duplicated stack trace when calling Throwable.printStackTrace because StreamingQueryException's toString contains the stack trace.

## How was this patch tested?

The updated `test("max files per trigger - incorrect values")`. I found this issue when I switched from `testStream` to the real codes to verify the failure in this test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16322 from zsxwing/SPARK-18907.
2016-12-21 22:02:57 -08:00
Takeshi YAMAMURO b41ec99778 [SPARK-18528][SQL] Fix a bug to initialise an iterator of aggregation buffer
## What changes were proposed in this pull request?
This pr is to fix an `NullPointerException` issue caused by a following `limit + aggregate` query;
```
scala> val df = Seq(("a", 1), ("b", 2), ("c", 1), ("d", 5)).toDF("id", "value")
scala> df.limit(2).groupBy("id").count().show
WARN TaskSetManager: Lost task 0.0 in stage 9.0 (TID 8204, lvsp20hdn012.stubprod.com): java.lang.NullPointerException
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.agg_doAggregateWithKeys$(Unknown Source)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
```
The root culprit is that [`$doAgg()`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L596) skips an initialization of [the buffer iterator](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L603); `BaseLimitExec` sets `stopEarly=true` and `$doAgg()` exits in the middle without the initialization.

## How was this patch tested?
Added a test to check if no exception happens for limit + aggregates in `DataFrameAggregateSuite.scala`.

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

Closes #15980 from maropu/SPARK-18528.
2016-12-22 01:53:33 +01:00
Tathagata Das 83a6ace0d1 [SPARK-18234][SS] Made update mode public
## What changes were proposed in this pull request?

Made update mode public. As part of that here are the changes.
- Update DatastreamWriter to accept "update"
- Changed package of InternalOutputModes from o.a.s.sql to o.a.s.sql.catalyst
- Added update mode state removing with watermark to StateStoreSaveExec

## How was this patch tested?

Added new tests in changed modules

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

Closes #16360 from tdas/SPARK-18234.
2016-12-21 16:43:17 -08:00
Reynold Xin 354e936187 [SPARK-18775][SQL] Limit the max number of records written per file
## What changes were proposed in this pull request?
Currently, Spark writes a single file out per task, sometimes leading to very large files. It would be great to have an option to limit the max number of records written per file in a task, to avoid humongous files.

This patch introduces a new write config option `maxRecordsPerFile` (default to a session-wide setting `spark.sql.files.maxRecordsPerFile`) that limits the max number of records written to a single file. A non-positive value indicates there is no limit (same behavior as not having this flag).

## How was this patch tested?
Added test cases in PartitionedWriteSuite for both dynamic partition insert and non-dynamic partition insert.

Author: Reynold Xin <rxin@databricks.com>

Closes #16204 from rxin/SPARK-18775.
2016-12-21 23:50:35 +01:00
Tathagata Das 607a1e63db [SPARK-18894][SS] Fix event time watermark delay threshold specified in months or years
## What changes were proposed in this pull request?

Two changes
- Fix how delays specified in months and years are translated to milliseconds
- Following up on #16258, not show watermark when there is no watermarking in the query

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

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

Closes #16304 from tdas/SPARK-18834-1.
2016-12-21 10:44:20 -08:00
Wenchen Fan b7650f11c7 [SPARK-18947][SQL] SQLContext.tableNames should not call Catalog.listTables
## What changes were proposed in this pull request?

It's a huge waste to call `Catalog.listTables` in `SQLContext.tableNames`, which only need the table names, while `Catalog.listTables` will get the table metadata for each table name.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16352 from cloud-fan/minor.
2016-12-21 19:39:00 +08:00
gatorsmile 24c0c94128 [SPARK-18949][SQL] Add recoverPartitions API to Catalog
### What changes were proposed in this pull request?

Currently, we only have a SQL interface for recovering all the partitions in the directory of a table and update the catalog. `MSCK REPAIR TABLE` or `ALTER TABLE table RECOVER PARTITIONS`. (Actually, very hard for me to remember `MSCK` and have no clue what it means)

After the new "Scalable Partition Handling", the table repair becomes much more important for making visible the data in the created data source partitioned table.

Thus, this PR is to add it into the Catalog interface. After this PR, users can repair the table by
```Scala
spark.catalog.recoverPartitions("testTable")
```

### How was this patch tested?
Modified the existing test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16356 from gatorsmile/repairTable.
2016-12-20 23:40:02 -08:00
Burak Yavuz caed89321f [SPARK-18927][SS] MemorySink for StructuredStreaming can't recover from checkpoint if location is provided in SessionConf
## What changes were proposed in this pull request?

Checkpoint Location can be defined for a StructuredStreaming on a per-query basis by the `DataStreamWriter` options, but it can also be provided through SparkSession configurations. It should be able to recover in both cases when the OutputMode is Complete for MemorySinks.

## How was this patch tested?

Unit tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16342 from brkyvz/chk-rec.
2016-12-20 14:19:35 -08:00
Reynold Xin 150d26cad4 Tiny style improvement. 2016-12-19 22:50:23 -08:00
Wenchen Fan f923c849e5 [SPARK-18899][SPARK-18912][SPARK-18913][SQL] refactor the error checking when append data to an existing table
## What changes were proposed in this pull request?

When we append data to an existing table with `DataFrameWriter.saveAsTable`, we will do various checks to make sure the appended data is consistent with the existing data.

However, we get the information of the existing table by matching the table relation, instead of looking at the table metadata. This is error-prone, e.g. we only check the number of columns for `HadoopFsRelation`, we forget to check bucketing, etc.

This PR refactors the error checking by looking at the metadata of the existing table, and fix several bugs:
* SPARK-18899: We forget to check if the specified bucketing matched the existing table, which may lead to a problematic table that has different bucketing in different data files.
* SPARK-18912: We forget to check the number of columns for non-file-based data source table
* SPARK-18913: We don't support append data to a table with special column names.

## How was this patch tested?
new regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16313 from cloud-fan/bug1.
2016-12-19 20:03:33 -08:00
Josh Rosen 5857b9ac2d [SPARK-18928] Check TaskContext.isInterrupted() in FileScanRDD, JDBCRDD & UnsafeSorter
## What changes were proposed in this pull request?

In order to respond to task cancellation, Spark tasks must periodically check `TaskContext.isInterrupted()`, but this check is missing on a few critical read paths used in Spark SQL, including `FileScanRDD`, `JDBCRDD`, and UnsafeSorter-based sorts. This can cause interrupted / cancelled tasks to continue running and become zombies (as also described in #16189).

This patch aims to fix this problem by adding `TaskContext.isInterrupted()` checks to these paths. Note that I could have used `InterruptibleIterator` to simply wrap a bunch of iterators but in some cases this would have an adverse performance penalty or might not be effective due to certain special uses of Iterators in Spark SQL. Instead, I inlined `InterruptibleIterator`-style logic into existing iterator subclasses.

## How was this patch tested?

Tested manually in `spark-shell` with two different reproductions of non-cancellable tasks, one involving scans of huge files and another involving sort-merge joins that spill to disk. Both causes of zombie tasks are fixed by the changes added here.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #16340 from JoshRosen/sql-task-interruption.
2016-12-20 01:19:38 +01:00
hyukjinkwon ed84cd0684
[MINOR][BUILD] Fix lint-check failures and javadoc8 break
## What changes were proposed in this pull request?

This PR proposes to fix lint-check failures and javadoc8 break.

Few errors were introduced as below:

**lint-check failures**

```
[ERROR] src/test/java/org/apache/spark/network/TransportClientFactorySuite.java:[45,1] (imports) RedundantImport: Duplicate import to line 43 - org.apache.spark.network.util.MapConfigProvider.
[ERROR] src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java:[255,10] (modifier) RedundantModifier: Redundant 'final' modifier.
```

**javadoc8**

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/StreamingQueryProgress.java:19: error: bad use of '>'
[error]  *                   "max" -> "2016-12-05T20:54:20.827Z"  // maximum event time seen in this trigger
[error]                             ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/StreamingQueryProgress.java:20: error: bad use of '>'
[error]  *                   "min" -> "2016-12-05T20:54:20.827Z"  // minimum event time seen in this trigger
[error]                             ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/StreamingQueryProgress.java:21: error: bad use of '>'
[error]  *                   "avg" -> "2016-12-05T20:54:20.827Z"  // average event time seen in this trigger
[error]                             ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/StreamingQueryProgress.java:22: error: bad use of '>'
[error]  *                   "watermark" -> "2016-12-05T20:54:20.827Z"  // watermark used in this trigger
[error]
```

## How was this patch tested?

Manually checked as below:

**lint-check failures**

```
./dev/lint-java
Checkstyle checks passed.
```

**javadoc8**

This seems hidden in the API doc but I manually checked after removing access modifier as below:

It looks not rendering properly (scaladoc).

![2016-12-16 3 40 34](https://cloud.githubusercontent.com/assets/6477701/21255175/8df1fe6e-c3ad-11e6-8cda-ce7f76c6677a.png)

After this PR, it renders as below:

- scaladoc
  ![2016-12-16 3 40 23](https://cloud.githubusercontent.com/assets/6477701/21255135/4a11dab6-c3ad-11e6-8ab2-b091c4f45029.png)

- javadoc
  ![2016-12-16 3 41 10](https://cloud.githubusercontent.com/assets/6477701/21255137/4bba1d9c-c3ad-11e6-9b88-62f1f697b56a.png)

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16307 from HyukjinKwon/lint-javadoc8.
2016-12-16 17:49:43 +00:00
Takeshi YAMAMURO dc2a4d4ad4 [SPARK-18108][SQL] Fix a schema inconsistent bug that makes a parquet reader fail to read data
## What changes were proposed in this pull request?
A vectorized parquet reader fails to read column data if data schema and partition schema overlap with each other and inferred types in the partition schema differ from ones in the data schema. An example code to reproduce this bug is as follows;

```
scala> case class A(a: Long, b: Int)
scala> val as = Seq(A(1, 2))
scala> spark.createDataFrame(as).write.parquet("/data/a=1/")
scala> val df = spark.read.parquet("/data/")
scala> df.printSchema
root
 |-- a: long (nullable = true)
 |-- b: integer (nullable = true)
scala> df.collect
java.lang.NullPointerException
        at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.getLong(OnHeapColumnVector.java:283)
        at org.apache.spark.sql.execution.vectorized.ColumnarBatch$Row.getLong(ColumnarBatch.java:191)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
```
The root cause is that a logical layer (`HadoopFsRelation`) and a physical layer (`VectorizedParquetRecordReader`) have a different assumption on partition schema; the logical layer trusts the data schema to infer the type the overlapped partition columns, and, on the other hand, the physical layer trusts partition schema which is inferred from path string. To fix this bug, this pr simply updates `HadoopFsRelation.schema` to respect the partition columns position in data schema and respect the partition columns type in partition schema.

## How was this patch tested?
Add tests in `ParquetPartitionDiscoverySuite`

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

Closes #16030 from maropu/SPARK-18108.
2016-12-16 22:44:42 +08:00
Shixiong Zhu d7f3058e17 [SPARK-18850][SS] Make StreamExecution and progress classes serializable
## What changes were proposed in this pull request?

This PR adds StreamingQueryWrapper to make StreamExecution and progress classes serializable because it is too easy for it to get captured with normal usage. If StreamingQueryWrapper gets captured in a closure but no place calls its methods, it should not fail the Spark tasks. However if its methods are called, then this PR will throw a better message.

## How was this patch tested?

`test("StreamingQuery should be Serializable but cannot be used in executors")`
`test("progress classes should be Serializable")`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16272 from zsxwing/SPARK-18850.
2016-12-16 00:42:39 -08:00
Shixiong Zhu 68a6dc974b [SPARK-18826][SS] Add 'latestFirst' option to FileStreamSource
## What changes were proposed in this pull request?

When starting a stream with a lot of backfill and maxFilesPerTrigger, the user could often want to start with most recent files first. This would let you keep low latency for recent data and slowly backfill historical data.

This PR adds a new option `latestFirst` to control this behavior. When it's true, `FileStreamSource` will sort the files by the modified time from latest to oldest, and take the first `maxFilesPerTrigger` files as a new batch.

## How was this patch tested?

The added test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16251 from zsxwing/newest-first.
2016-12-15 13:17:51 -08:00
jiangxingbo 01e14bf303 [SPARK-17910][SQL] Allow users to update the comment of a column
## What changes were proposed in this pull request?

Right now, once a user set the comment of a column with create table command, he/she cannot update the comment. It will be useful to provide a public interface (e.g. SQL) to do that.

This PR implements the following SQL statement:
```
ALTER TABLE table [PARTITION partition_spec]
CHANGE [COLUMN] column_old_name column_new_name column_dataType
[COMMENT column_comment]
[FIRST | AFTER column_name];
```

For further expansion, we could support alter `name`/`dataType`/`index` of a column too.

## How was this patch tested?

Add new test cases in `ExternalCatalogSuite` and `SessionCatalogSuite`.
Add sql file test for `ALTER TABLE CHANGE COLUMN` statement.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15717 from jiangxb1987/change-column.
2016-12-15 10:09:42 -08:00
Wenchen Fan d6f11a12a1 [SPARK-18856][SQL] non-empty partitioned table should not report zero size
## What changes were proposed in this pull request?

In `DataSource`, if the table is not analyzed, we will use 0 as the default value for table size. This is dangerous, we may broadcast a large table and cause OOM. We should use `defaultSizeInBytes` instead.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16280 from cloud-fan/bug.
2016-12-14 21:03:56 -08:00
Reynold Xin ffdd1fcd1e [SPARK-18854][SQL] numberedTreeString and apply(i) inconsistent for subqueries
## What changes were proposed in this pull request?
This is a bug introduced by subquery handling. numberedTreeString (which uses generateTreeString under the hood) numbers trees including innerChildren (used to print subqueries), but apply (which uses getNodeNumbered) ignores innerChildren. As a result, apply(i) would return the wrong plan node if there are subqueries.

This patch fixes the bug.

## How was this patch tested?
Added a test case in SubquerySuite.scala to test both the depth-first traversal of numbering as well as making sure the two methods are consistent.

Author: Reynold Xin <rxin@databricks.com>

Closes #16277 from rxin/SPARK-18854.
2016-12-14 16:12:14 -08:00
Shixiong Zhu 1ac6567bdb [SPARK-18852][SS] StreamingQuery.lastProgress should be null when recentProgress is empty
## What changes were proposed in this pull request?

Right now `StreamingQuery.lastProgress` throws NoSuchElementException and it's hard to be used in Python since Python user will just see Py4jError.

This PR just makes it return null instead.

## How was this patch tested?

`test("lastProgress should be null when recentProgress is empty")`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16273 from zsxwing/SPARK-18852.
2016-12-14 13:36:41 -08:00
hyukjinkwon 89ae26dcdb [SPARK-18753][SQL] Keep pushed-down null literal as a filter in Spark-side post-filter for FileFormat datasources
## What changes were proposed in this pull request?

Currently, `FileSourceStrategy` does not handle the case when the pushed-down filter is `Literal(null)` and removes it at the post-filter in Spark-side.

For example, the codes below:

```scala
val df = Seq(Tuple1(Some(true)), Tuple1(None), Tuple1(Some(false))).toDF()
df.filter($"_1" === "true").explain(true)
```

shows it keeps `null` properly.

```
== Parsed Logical Plan ==
'Filter ('_1 = true)
+- LocalRelation [_1#17]

== Analyzed Logical Plan ==
_1: boolean
Filter (cast(_1#17 as double) = cast(true as double))
+- LocalRelation [_1#17]

== Optimized Logical Plan ==
Filter (isnotnull(_1#17) && null)
+- LocalRelation [_1#17]

== Physical Plan ==
*Filter (isnotnull(_1#17) && null)       << Here `null` is there
+- LocalTableScan [_1#17]
```

However, when we read it back from Parquet,

```scala
val path = "/tmp/testfile"
df.write.parquet(path)
spark.read.parquet(path).filter($"_1" === "true").explain(true)
```

`null` is removed at the post-filter.

```
== Parsed Logical Plan ==
'Filter ('_1 = true)
+- Relation[_1#11] parquet

== Analyzed Logical Plan ==
_1: boolean
Filter (cast(_1#11 as double) = cast(true as double))
+- Relation[_1#11] parquet

== Optimized Logical Plan ==
Filter (isnotnull(_1#11) && null)
+- Relation[_1#11] parquet

== Physical Plan ==
*Project [_1#11]
+- *Filter isnotnull(_1#11)       << Here `null` is missing
   +- *FileScan parquet [_1#11] Batched: true, Format: ParquetFormat, Location: InMemoryFileIndex[file:/tmp/testfile], PartitionFilters: [null], PushedFilters: [IsNotNull(_1)], ReadSchema: struct<_1:boolean>
```

This PR fixes it to keep it properly. In more details,

```scala
val partitionKeyFilters =
  ExpressionSet(normalizedFilters.filter(_.references.subsetOf(partitionSet)))
```

This keeps this `null` in `partitionKeyFilters` as `Literal` always don't have `children` and `references` is being empty  which is always the subset of `partitionSet`.

And then in

```scala
val afterScanFilters = filterSet -- partitionKeyFilters
```

`null` is always removed from the post filter. So, if the referenced fields are empty, it should be applied into data columns too.

After this PR, it becomes as below:

```
== Parsed Logical Plan ==
'Filter ('_1 = true)
+- Relation[_1#276] parquet

== Analyzed Logical Plan ==
_1: boolean
Filter (cast(_1#276 as double) = cast(true as double))
+- Relation[_1#276] parquet

== Optimized Logical Plan ==
Filter (isnotnull(_1#276) && null)
+- Relation[_1#276] parquet

== Physical Plan ==
*Project [_1#276]
+- *Filter (isnotnull(_1#276) && null)
   +- *FileScan parquet [_1#276] Batched: true, Format: ParquetFormat, Location: InMemoryFileIndex[file:/private/var/folders/9j/gf_c342d7d150mwrxvkqnc180000gn/T/spark-a5d59bdb-5b..., PartitionFilters: [null], PushedFilters: [IsNotNull(_1)], ReadSchema: struct<_1:boolean>
```

## How was this patch tested?

Unit test in `FileSourceStrategySuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16184 from HyukjinKwon/SPARK-18753.
2016-12-14 11:29:11 -08:00
Wenchen Fan 3e307b4959 [SPARK-18566][SQL] remove OverwriteOptions
## What changes were proposed in this pull request?

`OverwriteOptions` was introduced in https://github.com/apache/spark/pull/15705, to carry the information of static partitions. However, after further refactor, this information becomes duplicated and we can remove `OverwriteOptions`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15995 from cloud-fan/overwrite.
2016-12-14 11:30:34 +08:00
Weiqing Yang ae5b2d3e46 [SPARK-18746][SQL] Add implicit encoder for BigDecimal, timestamp and date
## What changes were proposed in this pull request?
Add implicit encoders for BigDecimal, timestamp and date.

## How was this patch tested?
Add an unit test.  Pass build, unit tests, and some tests below .
Before:
```
scala> spark.createDataset(Seq(new java.math.BigDecimal(10)))
<console>:24: error: Unable to find encoder for type stored in a Dataset.  Primitive types (Int, String, etc) and Product types (case classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
       spark.createDataset(Seq(new java.math.BigDecimal(10)))
                          ^

scala>
```
After:
```
scala> spark.createDataset(Seq(new java.math.BigDecimal(10)))
res0: org.apache.spark.sql.Dataset[java.math.BigDecimal] = [value: decimal(38,18)]
```

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #16176 from weiqingy/SPARK-18746.
2016-12-14 09:48:38 +08:00
Tathagata Das c68fb426d4 [SPARK-18834][SS] Expose event time stats through StreamingQueryProgress
## What changes were proposed in this pull request?

- Changed `StreamingQueryProgress.watermark` to `StreamingQueryProgress.queryTimestamps` which is a `Map[String, String]` containing the following keys: "eventTime.max", "eventTime.min", "eventTime.avg", "processingTime", "watermark". All of them UTC formatted strings.

- Renamed `StreamingQuery.timestamp` to `StreamingQueryProgress.triggerTimestamp` to differentiate from `queryTimestamps`. It has the timestamp of when the trigger was started.

## How was this patch tested?

Updated tests

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

Closes #16258 from tdas/SPARK-18834.
2016-12-13 14:14:25 -08:00
jiangxingbo 5572ccf86b [SPARK-17932][SQL][FOLLOWUP] Change statement SHOW TABLES EXTENDED to SHOW TABLE EXTENDED
## What changes were proposed in this pull request?

Change the statement `SHOW TABLES [EXTENDED] [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards'] [PARTITION(partition_spec)]` to the following statements:

- SHOW TABLES [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards']
- SHOW TABLE EXTENDED [(IN|FROM) database_name] LIKE 'identifier_with_wildcards' [PARTITION(partition_spec)]

After this change, the statements `SHOW TABLE/SHOW TABLES` have the same syntax with that HIVE has.

## How was this patch tested?
Modified the test sql file `show-tables.sql`;
Modified the test suite `DDLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16262 from jiangxb1987/show-table-extended.
2016-12-13 19:04:34 +01:00
Marcelo Vanzin f280ccf449 [SPARK-18835][SQL] Don't expose Guava types in the JavaTypeInference API.
This avoids issues during maven tests because of shading.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16260 from vanzin/SPARK-18835.
2016-12-13 10:02:19 -08:00
Shixiong Zhu fb3081d3b3 [SPARK-13747][CORE] Fix potential ThreadLocal leaks in RPC when using ForkJoinPool
## What changes were proposed in this pull request?

Some places in SQL may call `RpcEndpointRef.askWithRetry` (e.g., ParquetFileFormat.buildReader -> SparkContext.broadcast -> ... -> BlockManagerMaster.updateBlockInfo -> RpcEndpointRef.askWithRetry), which will finally call `Await.result`. It may cause `java.lang.IllegalArgumentException: spark.sql.execution.id is already set` when running in Scala ForkJoinPool.

This PR includes the following changes to fix this issue:

- Remove `ThreadUtils.awaitResult`
- Rename `ThreadUtils. awaitResultInForkJoinSafely` to `ThreadUtils.awaitResult`
- Replace `Await.result` in RpcTimeout with `ThreadUtils.awaitResult`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16230 from zsxwing/fix-SPARK-13747.
2016-12-13 09:53:22 -08:00
Jacek Laskowski 096f868b74
[MINOR][CORE][SQL] Remove explicit RDD and Partition overrides
## What changes were proposed in this pull request?

I **believe** that I _only_ removed duplicated code (that adds nothing but noise). I'm gonna remove the comment after Jenkins has built the changes with no issues and Spark devs has agreed to include the changes.

Remove explicit `RDD` and `Partition` overrides (that turn out code duplication)

## How was this patch tested?

Local build. Awaiting Jenkins.

…cation)

Author: Jacek Laskowski <jacek@japila.pl>

Closes #16145 from jaceklaskowski/rdd-overrides-removed.
2016-12-13 09:40:16 +00:00
Shixiong Zhu 417e45c584 [SPARK-18796][SS] StreamingQueryManager should not block when starting a query
## What changes were proposed in this pull request?

Major change in this PR:
- Add `pendingQueryNames` and `pendingQueryIds` to track that are going to start but not yet put into `activeQueries` so that we don't need to hold a lock when starting a query.

Minor changes:
- Fix a potential NPE when the user sets `checkpointLocation` using SQLConf but doesn't specify a query name.
- Add missing docs in `StreamingQueryListener`

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16220 from zsxwing/SPARK-18796.
2016-12-12 22:31:22 -08:00
Marcelo Vanzin 476b34c23a [SPARK-18752][HIVE] isSrcLocal" value should be set from user query.
The value of the "isSrcLocal" parameter passed to Hive's loadTable and
loadPartition methods needs to be set according to the user query (e.g.
"LOAD DATA LOCAL"), and not the current code that tries to guess what
it should be.

For existing versions of Hive the current behavior is probably ok, but
some recent changes in the Hive code changed the semantics slightly,
making code that sets "isSrcLocal" to "true" incorrectly to do the
wrong thing. It would end up moving the parent directory of the files
into the final location, instead of the file themselves, resulting
in a table that cannot be read.

I modified HiveCommandSuite so that existing "LOAD DATA" tests are run
both in local and non-local mode, since the semantics are slightly different.
The tests include a few new checks to make sure the semantics follow
what Hive describes in its documentation.

Tested with existing unit tests and also ran some Hive integration tests
with a version of Hive containing the changes that surfaced the problem.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16179 from vanzin/SPARK-18752.
2016-12-12 14:19:42 -08:00
meknio bf42c2db57 [SPARK-16297][SQL] Fix mapping Microsoft SQLServer dialect
The problem is if it is run with no fix throws an exception and causes the following error:

  "Cannot specify a column width on data type bit."

The problem stems from the fact that the "java.sql.types.BIT" type is mapped as BIT[n] that really must be mapped as BIT.
This concerns the type Boolean.

As for the type String with maximum length of characters it must be mapped as VARCHAR (MAX) instead of TEXT which is a type deprecated in SQLServer.

Here is the list of mappings for SQL Server:
https://msdn.microsoft.com/en-us/library/ms378878(v=sql.110).aspx

Closes #13944 from meknio/master.
2016-12-12 12:54:39 -08:00
Tyson Condie 83a42897ae [SPARK-18790][SS] Keep a general offset history of stream batches
## What changes were proposed in this pull request?

Instead of only keeping the minimum number of offsets around, we should keep enough information to allow us to roll back n batches and reexecute the stream starting from a given point. In particular, we should create a config in SQLConf, spark.sql.streaming.retainedBatches that defaults to 100 and ensure that we keep enough log files in the following places to roll back the specified number of batches:
the offsets that are present in each batch
versions of the state store
the files lists stored for the FileStreamSource
the metadata log stored by the FileStreamSink

marmbrus zsxwing

## How was this patch tested?

The following tests were added.

### StreamExecution offset metadata
Test added to StreamingQuerySuite that ensures offset metadata is garbage collected according to minBatchesRetain

### CompactibleFileStreamLog
Tests added in CompactibleFileStreamLogSuite to ensure that logs are purged starting before the first compaction file that proceeds the current batch id - minBatchesToRetain.

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

Author: Tyson Condie <tcondie@gmail.com>

Closes #16219 from tcondie/offset_hist.
2016-12-11 23:38:31 -08:00
gatorsmile 422a45cf04 [SPARK-18766][SQL] Push Down Filter Through BatchEvalPython (Python UDF)
### What changes were proposed in this pull request?
Currently, when users use Python UDF in Filter, BatchEvalPython is always generated below FilterExec. However, not all the predicates need to be evaluated after Python UDF execution. Thus, this PR is to push down the determinisitc predicates through `BatchEvalPython`.
```Python
>>> df = spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"])
>>> from pyspark.sql.functions import udf, col
>>> from pyspark.sql.types import BooleanType
>>> my_filter = udf(lambda a: a < 2, BooleanType())
>>> sel = df.select(col("key"), col("value")).filter((my_filter(col("key"))) & (df.value < "2"))
>>> sel.explain(True)
```
Before the fix, the plan looks like
```
== Optimized Logical Plan ==
Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2))
+- LogicalRDD [key#0L, value#1]

== Physical Plan ==
*Project [key#0L, value#1]
+- *Filter ((isnotnull(value#1) && pythonUDF0#9) && (value#1 < 2))
   +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9]
      +- Scan ExistingRDD[key#0L,value#1]
```

After the fix, the plan looks like
```
== Optimized Logical Plan ==
Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2))
+- LogicalRDD [key#0L, value#1]

== Physical Plan ==
*Project [key#0L, value#1]
+- *Filter pythonUDF0#9: boolean
   +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9]
      +- *Filter (isnotnull(value#1) && (value#1 < 2))
         +- Scan ExistingRDD[key#0L,value#1]
```

### How was this patch tested?
Added both unit test cases for `BatchEvalPythonExec` and also add an end-to-end test case in Python test suite.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16193 from gatorsmile/pythonUDFPredicatePushDown.
2016-12-10 08:47:45 -08:00
Huaxin Gao c5172568b5 [SPARK-17460][SQL] Make sure sizeInBytes in Statistics will not overflow
## What changes were proposed in this pull request?

1. In SparkStrategies.canBroadcast, I will add the check   plan.statistics.sizeInBytes >= 0
2. In LocalRelations.statistics, when calculate the statistics, I will change the size to BigInt so it won't overflow.

## How was this patch tested?

I will add a test case to make sure the statistics.sizeInBytes won't overflow.

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #16175 from huaxingao/spark-17460.
2016-12-10 22:41:40 +08:00
Burak Yavuz 63c9159870 [SPARK-18811] StreamSource resolution should happen in stream execution thread
## What changes were proposed in this pull request?

When you start a stream, if we are trying to resolve the source of the stream, for example if we need to resolve partition columns, this could take a long time. This long execution time should not block the main thread where `query.start()` was called on. It should happen in the stream execution thread possibly before starting any triggers.

## How was this patch tested?

Unit test added. Made sure test fails with no code changes.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16238 from brkyvz/SPARK-18811.
2016-12-09 22:49:51 -08:00
Kazuaki Ishizaki d60ab5fd9b [SPARK-18745][SQL] Fix signed integer overflow due to toInt cast
## What changes were proposed in this pull request?

This PR avoids that a result of a cast `toInt` is negative due to signed integer overflow (e.g. 0x0000_0000_1???????L.toInt < 0 ). This PR performs casts after we can ensure the value is within range of signed integer (the result of `max(array.length, ???)` is always integer).

## How was this patch tested?

Manually executed query68 of TPC-DS with 100TB

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

Closes #16235 from kiszk/SPARK-18745.
2016-12-09 23:13:36 +01:00
Xiangrui Meng fd48d80a61 [SPARK-17822][R] Make JVMObjectTracker a member variable of RBackend
## What changes were proposed in this pull request?

* This PR changes `JVMObjectTracker` from `object` to `class` and let its instance associated with each RBackend. So we can manage the lifecycle of JVM objects when there are multiple `RBackend` sessions. `RBackend.close` will clear the object tracker explicitly.
* I assume that `SQLUtils` and `RRunner` do not need to track JVM instances, which could be wrong.
* Small refactor of `SerDe.sqlSerDe` to increase readability.

## How was this patch tested?

* Added unit tests for `JVMObjectTracker`.
* Wait for Jenkins to run full tests.

Author: Xiangrui Meng <meng@databricks.com>

Closes #16154 from mengxr/SPARK-17822.
2016-12-09 07:51:46 -08:00
Tathagata Das 458fa3325e [SPARK-18776][SS] Make Offset for FileStreamSource corrected formatted in json
## What changes were proposed in this pull request?

- Changed FileStreamSource to use new FileStreamSourceOffset rather than LongOffset. The field is named as `logOffset` to make it more clear that this is a offset in the file stream log.
- Fixed bug in FileStreamSourceLog, the field endId in the FileStreamSourceLog.get(startId, endId) was not being used at all. No test caught it earlier. Only my updated tests caught it.

Other minor changes
- Dont use batchId in the FileStreamSource, as calling it batch id is extremely miss leading. With multiple sources, it may happen that a new batch has no new data from a file source. So offset of FileStreamSource != batchId after that batch.

## How was this patch tested?

Updated unit test.

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

Closes #16205 from tdas/SPARK-18776.
2016-12-08 17:53:34 -08:00
Reynold Xin 5f894d23a5 [SPARK-18760][SQL] Consistent format specification for FileFormats
## What changes were proposed in this pull request?
This patch fixes the format specification in explain for file sources (Parquet and Text formats are the only two that are different from the rest):

Before:
```
scala> spark.read.text("test.text").explain()
== Physical Plan ==
*FileScan text [value#15] Batched: false, Format: org.apache.spark.sql.execution.datasources.text.TextFileFormatxyz, Location: InMemoryFileIndex[file:/scratch/rxin/spark/test.text], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<value:string>
```

After:
```
scala> spark.read.text("test.text").explain()
== Physical Plan ==
*FileScan text [value#15] Batched: false, Format: Text, Location: InMemoryFileIndex[file:/scratch/rxin/spark/test.text], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<value:string>
```

Also closes #14680.

## How was this patch tested?
Verified in spark-shell.

Author: Reynold Xin <rxin@databricks.com>

Closes #16187 from rxin/SPARK-18760.
2016-12-08 12:52:05 -08:00
Liang-Chi Hsieh 6a5a7254dc [SPARK-18667][PYSPARK][SQL] Change the way to group row in BatchEvalPythonExec so input_file_name function can work with UDF in pyspark
## What changes were proposed in this pull request?

`input_file_name` doesn't return filename when working with UDF in PySpark. An example shows the problem:

    from pyspark.sql.functions import *
    from pyspark.sql.types import *

    def filename(path):
        return path

    sourceFile = udf(filename, StringType())
    spark.read.json("tmp.json").select(sourceFile(input_file_name())).show()

    +---------------------------+
    |filename(input_file_name())|
    +---------------------------+
    |                           |
    +---------------------------+

The cause of this issue is, we group rows in `BatchEvalPythonExec` for batching processing of PythonUDF. Currently we group rows first and then evaluate expressions on the rows. If the data is less than the required number of rows for a group, the iterator will be consumed to the end before the evaluation. However, once the iterator reaches the end, we will unset input filename. So the input_file_name expression can't return correct filename.

This patch fixes the approach to group the batch of rows. We evaluate the expression first and then group evaluated results to batch.

## How was this patch tested?

Added unit test to PySpark.

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

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

Closes #16115 from viirya/fix-py-udf-input-filename.
2016-12-08 23:22:18 +08:00
Shixiong Zhu b47b892e45 [SPARK-18774][CORE][SQL] Ignore non-existing files when ignoreCorruptFiles is enabled
## What changes were proposed in this pull request?

When `ignoreCorruptFiles` is enabled, it's better to also ignore non-existing files.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16203 from zsxwing/ignore-file-not-found.
2016-12-07 22:37:04 -08:00
Tathagata Das 9ab725eabb [SPARK-18758][SS] StreamingQueryListener events from a StreamingQuery should be sent only to the listeners in the same session as the query
## What changes were proposed in this pull request?

Listeners added with `sparkSession.streams.addListener(l)` are added to a SparkSession. So events only from queries in the same session as a listener should be posted to the listener. Currently, all the events gets rerouted through the Spark's main listener bus, that is,
- StreamingQuery posts event to StreamingQueryListenerBus. Only the queries associated with the same session as the bus posts events to it.
- StreamingQueryListenerBus posts event to Spark's main LiveListenerBus as a SparkEvent.
- StreamingQueryListenerBus also subscribes to LiveListenerBus events thus getting back the posted event in a different thread.
- The received is posted to the registered listeners.

The problem is that *all StreamingQueryListenerBuses in all sessions* gets the events and posts them to their listeners. This is wrong.

In this PR, I solve it by making StreamingQueryListenerBus track active queries (by their runIds) when a query posts the QueryStarted event to the bus. This allows the rerouted events to be filtered using the tracked queries.

Note that this list needs to be maintained separately
from the `StreamingQueryManager.activeQueries` because a terminated query is cleared from
`StreamingQueryManager.activeQueries` as soon as it is stopped, but the this ListenerBus must
clear a query only after the termination event of that query has been posted lazily, much after the query has been terminated.

Credit goes to zsxwing for coming up with the initial idea.

## How was this patch tested?
Updated test harness code to use the correct session, and added new unit test.

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

Closes #16186 from tdas/SPARK-18758.
2016-12-07 19:23:27 -08:00
Michael Armbrust 70b2bf717d [SPARK-18754][SS] Rename recentProgresses to recentProgress
Based on an informal survey, users find this option easier to understand / remember.

Author: Michael Armbrust <michael@databricks.com>

Closes #16182 from marmbrus/renameRecentProgress.
2016-12-07 15:36:29 -08:00
Shixiong Zhu dbf3e298a1 [SPARK-18764][CORE] Add a warning log when skipping a corrupted file
## What changes were proposed in this pull request?

It's better to add a warning log when skipping a corrupted file. It will be helpful when we want to finish the job first, then find them in the log and fix these files.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16192 from zsxwing/SPARK-18764.
2016-12-07 10:30:05 -08:00
Tathagata Das 539bb3cf95 [SPARK-18734][SS] Represent timestamp in StreamingQueryProgress as formatted string instead of millis
## What changes were proposed in this pull request?

Easier to read while debugging as a formatted string (in ISO8601 format) than in millis

## How was this patch tested?
Updated unit tests

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

Closes #16166 from tdas/SPARK-18734.
2016-12-06 17:04:26 -08:00
Reynold Xin cb1f10b468 [SPARK-18714][SQL] Add a simple time function to SparkSession
## What changes were proposed in this pull request?
Many Spark developers often want to test the runtime of some function in interactive debugging and testing. This patch adds a simple time function to SparkSession:

```
scala> spark.time { spark.range(1000).count() }
Time taken: 77 ms
res1: Long = 1000
```

## How was this patch tested?
I tested this interactively in spark-shell.

Author: Reynold Xin <rxin@databricks.com>

Closes #16140 from rxin/SPARK-18714.
2016-12-06 11:48:11 -08:00
Shixiong Zhu 7863c62379 [SPARK-18721][SS] Fix ForeachSink with watermark + append
## What changes were proposed in this pull request?

Right now ForeachSink creates a new physical plan, so StreamExecution cannot retrieval metrics and watermark.

This PR changes ForeachSink to manually convert InternalRows to objects without creating a new plan.

## How was this patch tested?

`test("foreach with watermark: append")`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16160 from zsxwing/SPARK-18721.
2016-12-05 20:35:24 -08:00
Michael Allman 772ddbeaa6 [SPARK-18572][SQL] Add a method listPartitionNames to ExternalCatalog
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-18572)

## What changes were proposed in this pull request?

Currently Spark answers the `SHOW PARTITIONS` command by fetching all of the table's partition metadata from the external catalog and constructing partition names therefrom. The Hive client has a `getPartitionNames` method which is many times faster for this purpose, with the performance improvement scaling with the number of partitions in a table.

To test the performance impact of this PR, I ran the `SHOW PARTITIONS` command on two Hive tables with large numbers of partitions. One table has ~17,800 partitions, and the other has ~95,000 partitions. For the purposes of this PR, I'll call the former table `table1` and the latter table `table2`. I ran 5 trials for each table with before-and-after versions of this PR. The results are as follows:

Spark at bdc8153, `SHOW PARTITIONS table1`, times in seconds:
7.901
3.983
4.018
4.331
4.261

Spark at bdc8153, `SHOW PARTITIONS table2`
(Timed out after 10 minutes with a `SocketTimeoutException`.)

Spark at this PR, `SHOW PARTITIONS table1`, times in seconds:
3.801
0.449
0.395
0.348
0.336

Spark at this PR, `SHOW PARTITIONS table2`, times in seconds:
5.184
1.63
1.474
1.519
1.41

Taking the best times from each trial, we get a 12x performance improvement for a table with ~17,800 partitions and at least a 426x improvement for a table with ~95,000 partitions. More significantly, the latter command doesn't even complete with the current code in master.

This is actually a patch we've been using in-house at VideoAmp since Spark 1.1. It's made all the difference in the practical usability of our largest tables. Even with tables with about 1,000 partitions there's a performance improvement of about 2-3x.

## How was this patch tested?

I added a unit test to `VersionsSuite` which tests that the Hive client's `getPartitionNames` method returns the correct number of partitions.

Author: Michael Allman <michael@videoamp.com>

Closes #15998 from mallman/spark-18572-list_partition_names.
2016-12-06 11:33:35 +08:00
Shixiong Zhu 4af142f557 [SPARK-18722][SS] Move no data rate limit from StreamExecution to ProgressReporter
## What changes were proposed in this pull request?

Move no data rate limit from StreamExecution to ProgressReporter to make `recentProgresses` and listener events consistent.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16155 from zsxwing/SPARK-18722.
2016-12-05 18:51:07 -08:00
root 508de38c99 [SPARK-18555][SQL] DataFrameNaFunctions.fill miss up original values in long integers
## What changes were proposed in this pull request?

   DataSet.na.fill(0) used on a DataSet which has a long value column, it will change the original long value.

   The reason is that the type of the function fill's param is Double, and the numeric columns are always cast to double(`fillCol[Double](f, value)`) .
```
  def fill(value: Double, cols: Seq[String]): DataFrame = {
    val columnEquals = df.sparkSession.sessionState.analyzer.resolver
    val projections = df.schema.fields.map { f =>
      // Only fill if the column is part of the cols list.
      if (f.dataType.isInstanceOf[NumericType] && cols.exists(col => columnEquals(f.name, col))) {
        fillCol[Double](f, value)
      } else {
        df.col(f.name)
      }
    }
    df.select(projections : _*)
  }
```

 For example:
```
scala> val df = Seq[(Long, Long)]((1, 2), (-1, -2), (9123146099426677101L, 9123146560113991650L)).toDF("a", "b")
df: org.apache.spark.sql.DataFrame = [a: bigint, b: bigint]

scala> df.show
+-------------------+-------------------+
|                  a|                  b|
+-------------------+-------------------+
|                  1|                  2|
|                 -1|                 -2|
|9123146099426677101|9123146560113991650|
+-------------------+-------------------+

scala> df.na.fill(0).show
+-------------------+-------------------+
|                  a|                  b|
+-------------------+-------------------+
|                  1|                  2|
|                 -1|                 -2|
|9123146099426676736|9123146560113991680|
+-------------------+-------------------+
 ```

the original values changed [which is not we expected result]:
```
 9123146099426677101 -> 9123146099426676736
 9123146560113991650 -> 9123146560113991680
```

## How was this patch tested?

unit test added.

Author: root <root@iZbp1gsnrlfzjxh82cz80vZ.(none)>

Closes #15994 from windpiger/nafillMissupOriginalValue.
2016-12-05 18:39:56 -08:00
gatorsmile 2398fde450 [SPARK-18720][SQL][MINOR] Code Refactoring of withColumn
### What changes were proposed in this pull request?
Our existing withColumn for adding metadata can simply use the existing public withColumn API.

### How was this patch tested?
The existing test cases cover it.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16152 from gatorsmile/withColumnRefactoring.
2016-12-06 10:23:42 +08:00
Tathagata Das bb57bfe97d [SPARK-18657][SPARK-18668] Make StreamingQuery.id persists across restart and not auto-generate StreamingQuery.name
## What changes were proposed in this pull request?
Here are the major changes in this PR.
- Added the ability to recover `StreamingQuery.id` from checkpoint location, by writing the id to `checkpointLoc/metadata`.
- Added `StreamingQuery.runId` which is unique for every query started and does not persist across restarts. This is to identify each restart of a query separately (same as earlier behavior of `id`).
- Removed auto-generation of `StreamingQuery.name`. The purpose of name was to have the ability to define an identifier across restarts, but since id is precisely that, there is no need for a auto-generated name. This means name becomes purely cosmetic, and is null by default.
- Added `runId` to `StreamingQueryListener` events and `StreamingQueryProgress`.

Implementation details
- Renamed existing `StreamExecutionMetadata` to `OffsetSeqMetadata`, and moved it to the file `OffsetSeq.scala`, because that is what this metadata is tied to. Also did some refactoring to make the code cleaner (got rid of a lot of `.json` and `.getOrElse("{}")`).
- Added the `id` as the new `StreamMetadata`.
- When a StreamingQuery is created it gets or writes the `StreamMetadata` from `checkpointLoc/metadata`.
- All internal logging in `StreamExecution` uses `(name, id, runId)` instead of just `name`

TODO
- [x] Test handling of name=null in json generation of StreamingQueryProgress
- [x] Test handling of name=null in json generation of StreamingQueryListener events
- [x] Test python API of runId

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

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

Closes #16113 from tdas/SPARK-18657.
2016-12-05 18:17:38 -08:00
Shixiong Zhu 1b2785c3d0 [SPARK-18729][SS] Move DataFrame.collect out of synchronized block in MemorySink
## What changes were proposed in this pull request?

Move DataFrame.collect out of synchronized block so that we can query content in MemorySink when `DataFrame.collect` is running.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16162 from zsxwing/SPARK-18729.
2016-12-05 18:15:55 -08:00
Liang-Chi Hsieh 3ba69b6485 [SPARK-18634][PYSPARK][SQL] Corruption and Correctness issues with exploding Python UDFs
## What changes were proposed in this pull request?

As reported in the Jira, there are some weird issues with exploding Python UDFs in SparkSQL.

The following test code can reproduce it. Notice: the following test code is reported to return wrong results in the Jira. However, as I tested on master branch, it causes exception and so can't return any result.

    >>> from pyspark.sql.functions import *
    >>> from pyspark.sql.types import *
    >>>
    >>> df = spark.range(10)
    >>>
    >>> def return_range(value):
    ...   return [(i, str(i)) for i in range(value - 1, value + 1)]
    ...
    >>> range_udf = udf(return_range, ArrayType(StructType([StructField("integer_val", IntegerType()),
    ...                                                     StructField("string_val", StringType())])))
    >>>
    >>> df.select("id", explode(range_udf(df.id))).show()
    Traceback (most recent call last):
      File "<stdin>", line 1, in <module>
      File "/spark/python/pyspark/sql/dataframe.py", line 318, in show
        print(self._jdf.showString(n, 20))
      File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 1133, in __call__
      File "/spark/python/pyspark/sql/utils.py", line 63, in deco
        return f(*a, **kw)
      File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/protocol.py", line 319, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o126.showString.: java.lang.AssertionError: assertion failed
        at scala.Predef$.assert(Predef.scala:156)
        at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:120)
        at org.apache.spark.sql.execution.GenerateExec.consume(GenerateExec.scala:57)

The cause of this issue is, in `ExtractPythonUDFs` we insert `BatchEvalPythonExec` to run PythonUDFs in batch. `BatchEvalPythonExec` will add extra outputs (e.g., `pythonUDF0`) to original plan. In above case, the original `Range` only has one output `id`. After `ExtractPythonUDFs`, the added `BatchEvalPythonExec` has two outputs `id` and `pythonUDF0`.

Because the output of `GenerateExec` is given after analysis phase, in above case, it is the combination of `id`, i.e., the output of `Range`, and `col`. But in planning phase, we change `GenerateExec`'s child plan to `BatchEvalPythonExec` with additional output attributes.

It will cause no problem in non wholestage codegen. Because when evaluating the additional attributes are projected out the final output of `GenerateExec`.

However, as `GenerateExec` now supports wholestage codegen, the framework will input all the outputs of the child plan to `GenerateExec`. Then when consuming `GenerateExec`'s output data (i.e., calling `consume`), the number of output attributes is different to the output variables in wholestage codegen.

To solve this issue, this patch only gives the generator's output to `GenerateExec` after analysis phase. `GenerateExec`'s output is the combination of its child plan's output and the generator's output. So when we change `GenerateExec`'s child, its output is still correct.

## How was this patch tested?

Added test cases to PySpark.

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

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

Closes #16120 from viirya/fix-py-udf-with-generator.
2016-12-05 17:50:43 -08:00
Shixiong Zhu 246012859f [SPARK-18694][SS] Add StreamingQuery.explain and exception to Python and fix StreamingQueryException
## What changes were proposed in this pull request?

- Add StreamingQuery.explain and exception to Python.
- Fix StreamingQueryException to not expose `OffsetSeq`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16125 from zsxwing/py-streaming-explain.
2016-12-05 11:36:11 -08:00