Commit graph

2349 commits

Author SHA1 Message Date
petermaxlee 9812f7d538 [SPARK-17165][SQL] FileStreamSource should not track the list of seen files indefinitely
## What changes were proposed in this pull request?
Before this change, FileStreamSource uses an in-memory hash set to track the list of files processed by the engine. The list can grow indefinitely, leading to OOM or overflow of the hash set.

This patch introduces a new user-defined option called "maxFileAge", default to 24 hours. If a file is older than this age, FileStreamSource will purge it from the in-memory map that was used to track the list of files that have been processed.

## How was this patch tested?
Added unit tests for the underlying utility, and also added an end-to-end test to validate the purge in FileStreamSourceSuite. Also verified the new test cases would fail when the timeout was set to a very large number.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14728 from petermaxlee/SPARK-17165.
2016-08-26 11:30:23 -07:00
gatorsmile fd4ba3f626 [SPARK-17192][SQL] Issue Exception when Users Specify the Partitioning Columns without a Given Schema
### What changes were proposed in this pull request?
Address the comments by yhuai in the original PR: https://github.com/apache/spark/pull/14207

First, issue an exception instead of logging a warning when users specify the partitioning columns without a given schema.

Second, refactor the codes a little.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14572 from gatorsmile/followup16552.
2016-08-26 11:13:38 -07:00
hyukjinkwon 6063d5963f [SPARK-16216][SQL][FOLLOWUP] Enable timestamp type tests for JSON and verify all unsupported types in CSV
## What changes were proposed in this pull request?

This PR enables the tests for `TimestampType` for JSON and unifies the logics for verifying schema when writing in CSV.

In more details, this PR,

- Enables the tests for `TimestampType` for JSON and

  This was disabled due to an issue in `DatatypeConverter.parseDateTime` which parses dates incorrectly, for example as below:

  ```scala
   val d = javax.xml.bind.DatatypeConverter.parseDateTime("0900-01-01T00:00:00.000").getTime
  println(d.toString)
  ```
  ```
  Fri Dec 28 00:00:00 KST 899
  ```

  However, since we use `FastDateFormat`, it seems we are safe now.

  ```scala
  val d = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSS").parse("0900-01-01T00:00:00.000")
  println(d)
  ```
  ```
  Tue Jan 01 00:00:00 PST 900
  ```

- Verifies all unsupported types in CSV

  There is a separate logics to verify the schemas in `CSVFileFormat`. This is actually not quite correct enough because we don't support `NullType` and `CalanderIntervalType` as well `StructType`, `ArrayType`, `MapType`. So, this PR adds both types.

## How was this patch tested?

Tests in `JsonHadoopFsRelation` and `CSVSuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14829 from HyukjinKwon/SPARK-16216-followup.
2016-08-26 17:29:37 +02:00
Sean Zhong d96d151563 [SPARK-17187][SQL] Supports using arbitrary Java object as internal aggregation buffer object
## What changes were proposed in this pull request?

This PR introduces an abstract class `TypedImperativeAggregate` so that an aggregation function of TypedImperativeAggregate can use  **arbitrary** user-defined Java object as intermediate aggregation buffer object.

**This has advantages like:**
1. It now can support larger category of aggregation functions. For example, it will be much easier to implement aggregation function `percentile_approx`, which has a complex aggregation buffer definition.
2. It can be used to avoid doing serialization/de-serialization for every call of `update` or `merge` when converting domain specific aggregation object to internal Spark-Sql storage format.
3. It is easier to integrate with other existing monoid libraries like algebird, and supports more aggregation functions with high performance.

Please see `org.apache.spark.sql.TypedImperativeAggregateSuite.TypedMaxAggregate` to find an example of how to defined a `TypedImperativeAggregate` aggregation function.
Please see Java doc of `TypedImperativeAggregate` and Jira ticket SPARK-17187 for more information.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzhong@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14753 from clockfly/object_aggregation_buffer_try_2.
2016-08-25 16:36:16 -07:00
Josh Rosen a133057ce5 [SPARK-17229][SQL] PostgresDialect shouldn't widen float and short types during reads
## What changes were proposed in this pull request?

When reading float4 and smallint columns from PostgreSQL, Spark's `PostgresDialect` widens these types to Decimal and Integer rather than using the narrower Float and Short types. According to https://www.postgresql.org/docs/7.1/static/datatype.html#DATATYPE-TABLE, Postgres maps the `smallint` type to a signed two-byte integer and the `real` / `float4` types to single precision floating point numbers.

This patch fixes this by adding more special-cases to `getCatalystType`, similar to what was done for the Derby JDBC dialect. I also fixed a similar problem in the write path which causes Spark to create integer columns in Postgres for what should have been ShortType columns.

## How was this patch tested?

New test cases in `PostgresIntegrationSuite` (which I ran manually because Jenkins can't run it right now).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14796 from JoshRosen/postgres-jdbc-type-fixes.
2016-08-25 23:22:40 +02:00
Takeshi YAMAMURO 2b0cc4e0df [SPARK-12978][SQL] Skip unnecessary final group-by when input data already clustered with group-by keys
This ticket targets the optimization to skip an unnecessary group-by operation below;

Without opt.:
```
== Physical Plan ==
TungstenAggregate(key=[col0#159], functions=[(sum(col1#160),mode=Final,isDistinct=false),(avg(col2#161),mode=Final,isDistinct=false)], output=[col0#159,sum(col1)#177,avg(col2)#178])
+- TungstenAggregate(key=[col0#159], functions=[(sum(col1#160),mode=Partial,isDistinct=false),(avg(col2#161),mode=Partial,isDistinct=false)], output=[col0#159,sum#200,sum#201,count#202L])
   +- TungstenExchange hashpartitioning(col0#159,200), None
      +- InMemoryColumnarTableScan [col0#159,col1#160,col2#161], InMemoryRelation [col0#159,col1#160,col2#161], true, 10000, StorageLevel(true, true, false, true, 1), ConvertToUnsafe, None
```

With opt.:
```
== Physical Plan ==
TungstenAggregate(key=[col0#159], functions=[(sum(col1#160),mode=Complete,isDistinct=false),(avg(col2#161),mode=Final,isDistinct=false)], output=[col0#159,sum(col1)#177,avg(col2)#178])
+- TungstenExchange hashpartitioning(col0#159,200), None
  +- InMemoryColumnarTableScan [col0#159,col1#160,col2#161], InMemoryRelation [col0#159,col1#160,col2#161], true, 10000, StorageLevel(true, true, false, true, 1), ConvertToUnsafe, None
```

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

Closes #10896 from maropu/SkipGroupbySpike.
2016-08-25 12:39:58 +02:00
jiangxingbo 5f02d2e5b4 [SPARK-17215][SQL] Method SQLContext.parseDataType(dataTypeString: String) could be removed.
## What changes were proposed in this pull request?

Method `SQLContext.parseDataType(dataTypeString: String)` could be removed, we should use `SparkSession.parseDataType(dataTypeString: String)` instead.
This require updating PySpark.

## How was this patch tested?

Existing test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #14790 from jiangxb1987/parseDataType.
2016-08-24 23:36:04 -07:00
gatorsmile 4d0706d616 [SPARK-17190][SQL] Removal of HiveSharedState
### What changes were proposed in this pull request?
Since `HiveClient` is used to interact with the Hive metastore, it should be hidden in `HiveExternalCatalog`. After moving `HiveClient` into `HiveExternalCatalog`, `HiveSharedState` becomes a wrapper of `HiveExternalCatalog`. Thus, removal of `HiveSharedState` becomes straightforward. After removal of `HiveSharedState`, the reflection logic is directly applied on the choice of `ExternalCatalog` types, based on the configuration of `CATALOG_IMPLEMENTATION`.

~~`HiveClient` is also used/invoked by the other entities besides HiveExternalCatalog, we defines the following two APIs: getClient and getNewClient~~

### How was this patch tested?
The existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14757 from gatorsmile/removeHiveClient.
2016-08-25 12:50:03 +08:00
hyukjinkwon 29952ed096 [SPARK-16216][SQL] Read/write timestamps and dates in ISO 8601 and dateFormat/timestampFormat option for CSV and JSON
## What changes were proposed in this pull request?

### Default - ISO 8601

Currently, CSV datasource is writing `Timestamp` and `Date` as numeric form and JSON datasource is writing both as below:

- CSV
  ```
  // TimestampType
  1414459800000000
  // DateType
  16673
  ```

- Json

  ```
  // TimestampType
  1970-01-01 11:46:40.0
  // DateType
  1970-01-01
  ```

So, for CSV we can't read back what we write and for JSON it becomes ambiguous because the timezone is being missed.

So, this PR make both **write** `Timestamp` and `Date` in ISO 8601 formatted string (please refer the [ISO 8601 specification](https://www.w3.org/TR/NOTE-datetime)).

- For `Timestamp` it becomes as below: (`yyyy-MM-dd'T'HH:mm:ss.SSSZZ`)

  ```
  1970-01-01T02:00:01.000-01:00
  ```

- For `Date` it becomes as below (`yyyy-MM-dd`)

  ```
  1970-01-01
  ```

### Custom date format option - `dateFormat`

This PR also adds the support to write and read dates and timestamps in a formatted string as below:

- **DateType**

  - With `dateFormat` option (e.g. `yyyy/MM/dd`)

    ```
    +----------+
    |      date|
    +----------+
    |2015/08/26|
    |2014/10/27|
    |2016/01/28|
    +----------+
    ```

### Custom date format option - `timestampFormat`

- **TimestampType**

  - With `dateFormat` option (e.g. `dd/MM/yyyy HH:mm`)

    ```
    +----------------+
    |            date|
    +----------------+
    |2015/08/26 18:00|
    |2014/10/27 18:30|
    |2016/01/28 20:00|
    +----------------+
    ```

## How was this patch tested?

Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14279 from HyukjinKwon/SPARK-16216-json-csv.
2016-08-24 22:16:20 +02:00
Wenchen Fan 52fa45d62a [SPARK-17186][SQL] remove catalog table type INDEX
## What changes were proposed in this pull request?

Actually Spark SQL doesn't support index, the catalog table type `INDEX` is from Hive. However, most operations in Spark SQL can't handle index table, e.g. create table, alter table, etc.

Logically index table should be invisible to end users, and Hive also generates special table name for index table to avoid users accessing it directly. Hive has special SQL syntax to create/show/drop index tables.

At Spark SQL side, although we can describe index table directly, but the result is unreadable, we should use the dedicated SQL syntax to do it(e.g. `SHOW INDEX ON tbl`). Spark SQL can also read index table directly, but the result is always empty.(Can hive read index table directly?)

This PR remove the table type `INDEX`, to make it clear that Spark SQL doesn't support index currently.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14752 from cloud-fan/minor2.
2016-08-23 23:46:09 -07:00
Davies Liu 9afdfc94f4 [SPARK-13286] [SQL] add the next expression of SQLException as cause
## What changes were proposed in this pull request?

Some JDBC driver (for example PostgreSQL) does not use the underlying exception as cause, but have another APIs (getNextException) to access that, so it it's included in the error logging, making us hard to find the root cause, especially in batch mode.

This PR will pull out the next exception and add it as cause (if it's different) or suppressed (if there is another different cause).

## How was this patch tested?

Can't reproduce this on the default JDBC driver, so did not add a regression test.

Author: Davies Liu <davies@databricks.com>

Closes #14722 from davies/keep_cause.
2016-08-23 09:45:13 -07:00
Jacek Laskowski 9d376ad76c [SPARK-17199] Use CatalystConf.resolver for case-sensitivity comparison
## What changes were proposed in this pull request?

Use `CatalystConf.resolver` consistently for case-sensitivity comparison (removed dups).

## How was this patch tested?

Local build. Waiting for Jenkins to ensure clean build and test.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #14771 from jaceklaskowski/17199-catalystconf-resolver.
2016-08-23 12:59:25 +02:00
Sean Zhong cc33460a51 [SPARK-17188][SQL] Moves class QuantileSummaries to project catalyst for implementing percentile_approx
## What changes were proposed in this pull request?

This is a sub-task of [SPARK-16283](https://issues.apache.org/jira/browse/SPARK-16283) (Implement percentile_approx SQL function), which moves class QuantileSummaries to project catalyst so that it can be reused when implementing aggregation function `percentile_approx`.

## How was this patch tested?

This PR only does class relocation, class implementation is not changed.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14754 from clockfly/move_QuantileSummaries_to_catalyst.
2016-08-23 14:57:00 +08:00
gatorsmile 6d93f9e023 [SPARK-17144][SQL] Removal of useless CreateHiveTableAsSelectLogicalPlan
## What changes were proposed in this pull request?
`CreateHiveTableAsSelectLogicalPlan` is a dead code after refactoring.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14707 from gatorsmile/removeCreateHiveTable.
2016-08-23 08:03:08 +08:00
Eric Liang 84770b59f7 [SPARK-17162] Range does not support SQL generation
## What changes were proposed in this pull request?

The range operator previously didn't support SQL generation, which made it not possible to use in views.

## How was this patch tested?

Unit tests.

cc hvanhovell

Author: Eric Liang <ekl@databricks.com>

Closes #14724 from ericl/spark-17162.
2016-08-22 15:48:35 -07:00
Sean Zhong 929cb8beed [MINOR][SQL] Fix some typos in comments and test hints
## What changes were proposed in this pull request?

Fix some typos in comments and test hints

## How was this patch tested?

N/A.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14755 from clockfly/fix_minor_typo.
2016-08-22 13:31:38 -07:00
Davies Liu 8d35a6f68d [SPARK-17115][SQL] decrease the threshold when split expressions
## What changes were proposed in this pull request?

In 2.0, we change the threshold of splitting expressions from 16K to 64K, which cause very bad performance on wide table, because the generated method can't be JIT compiled by default (above the limit of 8K bytecode).

This PR will decrease it to 1K, based on the benchmark results for a wide table with 400 columns of LongType.

It also fix a bug around splitting expression in whole-stage codegen (it should not split them).

## How was this patch tested?

Added benchmark suite.

Author: Davies Liu <davies@databricks.com>

Closes #14692 from davies/split_exprs.
2016-08-22 16:16:03 +08:00
Wenchen Fan b2074b664a [SPARK-16498][SQL] move hive hack for data source table into HiveExternalCatalog
## What changes were proposed in this pull request?

Spark SQL doesn't have its own meta store yet, and use hive's currently. However, hive's meta store has some limitations(e.g. columns can't be too many, not case-preserving, bad decimal type support, etc.), so we have some hacks to successfully store data source table metadata into hive meta store, i.e. put all the information in table properties.

This PR moves these hacks to `HiveExternalCatalog`, tries to isolate hive specific logic in one place.

changes overview:

1.  **before this PR**: we need to put metadata(schema, partition columns, etc.) of data source tables to table properties before saving it to external catalog, even the external catalog doesn't use hive metastore(e.g. `InMemoryCatalog`)
**after this PR**: the table properties tricks are only in `HiveExternalCatalog`, the caller side doesn't need to take care of it anymore.

2. **before this PR**: because the table properties tricks are done outside of external catalog, so we also need to revert these tricks when we read the table metadata from external catalog and use it. e.g. in `DescribeTableCommand` we will read schema and partition columns from table properties.
**after this PR**: The table metadata read from external catalog is exactly the same with what we saved to it.

bonus: now we can create data source table using `SessionCatalog`, if schema is specified.
breaks: `schemaStringLengthThreshold` is not configurable anymore. `hive.default.rcfile.serde` is not configurable anymore.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14155 from cloud-fan/catalog-table.
2016-08-21 22:23:14 -07:00
petermaxlee 9560c8d295 [SPARK-17124][SQL] RelationalGroupedDataset.agg should preserve order and allow multiple aggregates per column
## What changes were proposed in this pull request?
This patch fixes a longstanding issue with one of the RelationalGroupedDataset.agg function. Even though the signature accepts vararg of pairs, the underlying implementation turns the seq into a map, and thus not order preserving nor allowing multiple aggregates per column.

This change also allows users to use this function to run multiple different aggregations for a single column, e.g.
```
agg("age" -> "max", "age" -> "count")
```

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

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14697 from petermaxlee/SPARK-17124.
2016-08-21 00:25:55 +08:00
Liang-Chi Hsieh 31a0155720 [SPARK-17104][SQL] LogicalRelation.newInstance should follow the semantics of MultiInstanceRelation
## What changes were proposed in this pull request?

Currently `LogicalRelation.newInstance()` simply creates another `LogicalRelation` object with the same parameters. However, the `newInstance()` method inherited from `MultiInstanceRelation` should return a copy of object with unique expression ids. Current `LogicalRelation.newInstance()` can cause failure when doing self-join.

## How was this patch tested?

Jenkins tests.

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

Closes #14682 from viirya/fix-localrelation.
2016-08-20 23:29:48 +08:00
petermaxlee 45d40d9f66 [SPARK-17150][SQL] Support SQL generation for inline tables
## What changes were proposed in this pull request?
This patch adds support for SQL generation for inline tables. With this, it would be possible to create a view that depends on inline tables.

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

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14709 from petermaxlee/SPARK-17150.
2016-08-20 13:19:38 +08:00
Reynold Xin 1748f82410 [SPARK-16391][SQL] Support partial aggregation for reduceGroups
## What changes were proposed in this pull request?
This patch introduces a new private ReduceAggregator interface that is a subclass of Aggregator. ReduceAggregator only requires a single associative and commutative reduce function. ReduceAggregator is also used to implement KeyValueGroupedDataset.reduceGroups in order to support partial aggregation.

Note that the pull request was initially done by viirya.

## How was this patch tested?
Covered by original tests for reduceGroups, as well as a new test suite for ReduceAggregator.

Author: Reynold Xin <rxin@databricks.com>
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14576 from rxin/reduceAggregator.
2016-08-18 16:37:25 +08:00
Tathagata Das d60af8f6aa [SPARK-17096][SQL][STREAMING] Improve exception string reported through the StreamingQueryListener
## What changes were proposed in this pull request?

Currently, the stackTrace (as `Array[StackTraceElements]`) reported through StreamingQueryListener.onQueryTerminated is useless as it has the stack trace of where StreamingQueryException is defined, not the stack trace of underlying exception.  For example, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.stackTrace` will have
```
org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:211)
org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124)
```
This is basically useless, as it is location where the StreamingQueryException was defined. What we want is

Here is the right way to reason about what should be posted as through StreamingQueryListener.onQueryTerminated
- The actual exception could either be a SparkException, or an arbitrary exception.
  - SparkException reports the relevant executor stack trace of a failed task as a string in the the exception message. The `Array[StackTraceElements]` returned by `SparkException.stackTrace()` is mostly irrelevant.
  - For any arbitrary exception, the `Array[StackTraceElements]` returned by `exception.stackTrace()` may be relevant.
- When there is an error in a streaming query, it's hard to reason whether the `Array[StackTraceElements]` is useful or not. In fact, it is not clear whether it is even useful to report the stack trace as this array of Java objects. It may be sufficient to report the strack trace as a string, along with the message. This is how Spark reported executor stra
- Hence, this PR simplifies the API by removing the array `stackTrace` from `QueryTerminated`. Instead the `exception` returns a string containing the message and the stack trace of the actual underlying exception that failed the streaming query (i.e. not that of the StreamingQueryException). If anyone is interested in the actual stack trace as an array, can always access them through `streamingQuery.exception` which returns the exception object.

With this change, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.exception` will be
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent failure: Lost task 1.0 in stage 0.0 (TID 1, localhost): java.lang.ArithmeticException: / by zero
	at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply$mcII$sp(StreamingQueryListenerSuite.scala:153)
	at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153)
	at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:232)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:226)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
	at org.apache.spark.scheduler.Task.run(Task.scala:86)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:744)

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1429)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1417)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1416)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1416)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
...
```
It contains the relevant executor stack trace. In a case non-SparkException, if the streaming source MemoryStream throws an exception, exception message will have the relevant stack trace.
```
java.lang.RuntimeException: this is the exception message
	at org.apache.spark.sql.execution.streaming.MemoryStream.getBatch(memory.scala:103)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:316)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:313)
	at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
	at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
	at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25)
	at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
	at org.apache.spark.sql.execution.streaming.StreamProgress.flatMap(StreamProgress.scala:25)
	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:313)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:197)
	at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43)
	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:187)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124)
```

Note that this change in the public `QueryTerminated` class is okay as the APIs are still experimental.

## How was this patch tested?
Unit tests that test whether the right information is present in the exception message reported through QueryTerminated object.

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

Closes #14675 from tdas/SPARK-17096.
2016-08-17 13:31:34 -07:00
Steve Loughran cc97ea188e [SPARK-16736][CORE][SQL] purge superfluous fs calls
A review of the code, working back from Hadoop's `FileSystem.exists()` and `FileSystem.isDirectory()` code, then removing uses of the calls when superfluous.

1. delete is harmless if called on a nonexistent path, so don't do any checks before deletes
1. any `FileSystem.exists()`  check before `getFileStatus()` or `open()` is superfluous as the operation itself does the check. Instead the `FileNotFoundException` is caught and triggers the downgraded path. When a `FileNotFoundException` was thrown before, the code still creates a new FNFE with the error messages. Though now the inner exceptions are nested, for easier diagnostics.

Initially, relying on Jenkins test runs.

One troublespot here is that some of the codepaths are clearly error situations; it's not clear that they have coverage anyway. Trying to create the failure conditions in tests would be ideal, but it will also be hard.

Author: Steve Loughran <stevel@apache.org>

Closes #14371 from steveloughran/cloud/SPARK-16736-superfluous-fs-calls.
2016-08-17 11:43:01 -07:00
Herman van Hovell 0b0c8b95e3 [SPARK-17106] [SQL] Simplify the SubqueryExpression interface
## What changes were proposed in this pull request?
The current subquery expression interface contains a little bit of technical debt in the form of a few different access paths to get and set the query contained by the expression. This is confusing to anyone who goes over this code.

This PR unifies these access paths.

## How was this patch tested?
(Existing tests)

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

Closes #14685 from hvanhovell/SPARK-17106.
2016-08-17 07:03:24 -07:00
Herman van Hovell f7c9ff57c1 [SPARK-17068][SQL] Make view-usage visible during analysis
## What changes were proposed in this pull request?
This PR adds a field to subquery alias in order to make the usage of views in a resolved `LogicalPlan` more visible (and more understandable).

For example, the following view and query:
```sql
create view constants as select 1 as id union all select 1 union all select 42
select * from constants;
```
...now yields the following analyzed plan:
```
Project [id#39]
+- SubqueryAlias c, `default`.`constants`
   +- Project [gen_attr_0#36 AS id#39]
      +- SubqueryAlias gen_subquery_0
         +- Union
            :- Union
            :  :- Project [1 AS gen_attr_0#36]
            :  :  +- OneRowRelation$
            :  +- Project [1 AS gen_attr_1#37]
            :     +- OneRowRelation$
            +- Project [42 AS gen_attr_2#38]
               +- OneRowRelation$
```
## How was this patch tested?
Added tests for the two code paths in `SessionCatalogSuite` (sql/core) and `HiveMetastoreCatalogSuite` (sql/hive)

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

Closes #14657 from hvanhovell/SPARK-17068.
2016-08-16 23:09:53 -07:00
Herman van Hovell 4a2c375be2 [SPARK-17084][SQL] Rename ParserUtils.assert to validate
## What changes were proposed in this pull request?
This PR renames `ParserUtils.assert` to `ParserUtils.validate`. This is done because this method is used to check requirements, and not to check if the program is in an invalid state.

## How was this patch tested?
Simple rename. Compilation should do.

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

Closes #14665 from hvanhovell/SPARK-17084.
2016-08-16 21:35:39 -07:00
Wenchen Fan 7de30d6e9e [SPARK-16916][SQL] serde/storage properties should not have limitations
## What changes were proposed in this pull request?

`CatalogStorageFormat.properties` can be used in 2 ways:

1. for hive tables, it stores the serde properties.
2. for data source tables, it stores the data source options, e.g. `path`, `skipHiveMetadata`, etc.

however, both of them have nothing to do with data source properties, e.g. `spark.sql.sources.provider`, so they should not have limitations about data source properties.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14506 from cloud-fan/table-prop.
2016-08-15 21:43:41 -07:00
Shixiong Zhu 268b71d0d7 [SPARK-17065][SQL] Improve the error message when encountering an incompatible DataSourceRegister
## What changes were proposed in this pull request?

Add an instruction to ask the user to remove or upgrade the incompatible DataSourceRegister in the error message.

## How was this patch tested?

Test command:
```
build/sbt -Dscala-2.10 package
SPARK_SCALA_VERSION=2.10 bin/spark-shell --packages ai.h2o:sparkling-water-core_2.10:1.6.5

scala> Seq(1).toDS().write.format("parquet").save("foo")
```

Before:
```
java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated
	at java.util.ServiceLoader.fail(ServiceLoader.java:232)
	at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
	at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
	at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
...
Caused by: java.lang.NoClassDefFoundError: org/apache/spark/Logging
	at java.lang.ClassLoader.defineClass1(Native Method)
	at java.lang.ClassLoader.defineClass(ClassLoader.java:760)
	at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
	at java.net.URLClassLoader.defineClass(URLClassLoader.java:467)
	at java.net.URLClassLoader.access$100(URLClassLoader.java:73)
	at java.net.URLClassLoader$1.run(URLClassLoader.java:368)
	at java.net.URLClassLoader$1.run(URLClassLoader.java:362)
	at java.security.AccessController.doPrivileged(Native Method)
...
```

After:

```
java.lang.ClassNotFoundException: Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated
	at org.apache.spark.sql.execution.datasources.DataSource.lookupDataSource(DataSource.scala:178)
	at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:79)
	at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:79)
	at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:441)
	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:213)
	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:196)
...
```

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #14651 from zsxwing/SPARK-17065.
2016-08-15 15:55:32 -07:00
Marcelo Vanzin 5da6c4b24f [SPARK-16671][CORE][SQL] Consolidate code to do variable substitution.
Both core and sql have slightly different code that does variable substitution
of config values. This change refactors that code and encapsulates the logic
of reading config values and expading variables in a new helper class, which
can be configured so that both core and sql can use it without losing existing
functionality, and allows for easier testing and makes it easier to add more
features in the future.

Tested with existing and new unit tests, and by running spark-shell with
some configs referencing variables and making sure it behaved as expected.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14468 from vanzin/SPARK-16671.
2016-08-15 11:09:54 -07:00
Sean Owen cdaa562c9a [SPARK-16966][SQL][CORE] App Name is a randomUUID even when "spark.app.name" exists
## What changes were proposed in this pull request?

Don't override app name specified in `SparkConf` with a random app name. Only set it if the conf has no app name even after options have been applied.

See also https://github.com/apache/spark/pull/14602
This is similar to Sherry302 's original proposal in https://github.com/apache/spark/pull/14556

## How was this patch tested?

Jenkins test, with new case reproducing the bug

Author: Sean Owen <sowen@cloudera.com>

Closes #14630 from srowen/SPARK-16966.2.
2016-08-13 15:40:43 -07:00
GraceH 8c8acdec93 [SPARK-16968] Add additional options in jdbc when creating a new table
## What changes were proposed in this pull request?

In the PR, we just allow the user to add additional options when create a new table in JDBC writer.
The options can be table_options or partition_options.
E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8"

Here is the usage example:
```
df.write.option("createTableOptions", "ENGINE=InnoDB DEFAULT CHARSET=utf8").jdbc(...)
```
## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
will apply test result soon.

Author: GraceH <93113783@qq.com>

Closes #14559 from GraceH/jdbc_options.
2016-08-13 11:39:58 +01:00
Dongjoon Hyun abff92bfdc [SPARK-16975][SQL] Column-partition path starting '_' should be handled correctly
## What changes were proposed in this pull request?

Currently, Spark ignores path names starting with underscore `_` and `.`. This causes read-failures for the column-partitioned file data sources whose partition column names starts from '_', e.g. `_col`.

**Before**
```scala
scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet")
scala> spark.read.parquet("/tmp/parquet")
org.apache.spark.sql.AnalysisException: Unable to infer schema for ParquetFormat at /tmp/parquet20. It must be specified manually;
```

**After**
```scala
scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet")
scala> spark.read.parquet("/tmp/parquet")
res2: org.apache.spark.sql.DataFrame = [id: bigint, _locality_code: int]
```

## How was this patch tested?

Pass the Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14585 from dongjoon-hyun/SPARK-16975-PARQUET.
2016-08-12 14:40:12 +08:00
hyukjinkwon ac84fb64dd [SPARK-16434][SQL] Avoid per-record type dispatch in JSON when reading
## What changes were proposed in this pull request?

Currently, `JacksonParser.parse` is doing type-based dispatch for each row to convert the tokens to appropriate values for Spark.
It might not have to be done like this because the schema is already kept.

So, appropriate converters can be created first according to the schema once, and then apply them to each row.

This PR corrects `JacksonParser` so that it creates all converters for the schema once and then applies them to each row rather than type dispatching for every row.

Benchmark was proceeded with the codes below:

#### Parser tests

**Before**

```scala
test("Benchmark for JSON converter") {
  val N = 500 << 8
  val row =
    """{"struct":{"field1": true, "field2": 92233720368547758070},
    "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]},
    "arrayOfString":["str1", "str2"],
    "arrayOfInteger":[1, 2147483647, -2147483648],
    "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808],
    "arrayOfBigInteger":[922337203685477580700, -922337203685477580800],
    "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308],
    "arrayOfBoolean":[true, false, true],
    "arrayOfNull":[null, null, null, null],
    "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}],
    "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]],
    "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]]
   }"""
  val data = List.fill(N)(row)
  val dummyOption = new JSONOptions(Map.empty[String, String])
  val schema =
    InferSchema.infer(spark.sparkContext.parallelize(Seq(row)), "", dummyOption)
  val factory = new JsonFactory()

  val benchmark = new Benchmark("JSON converter", N)
  benchmark.addCase("convert JSON file", 10) { _ =>
    data.foreach { input =>
      val parser = factory.createParser(input)
      parser.nextToken()
      JacksonParser.convertRootField(factory, parser, schema)
    }
  }
  benchmark.run()
}
```

```
JSON converter:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
convert JSON file                             1697 / 1807          0.1       13256.9       1.0X
```

**After**

```scala
test("Benchmark for JSON converter") {
  val N = 500 << 8
  val row =
    """{"struct":{"field1": true, "field2": 92233720368547758070},
    "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]},
    "arrayOfString":["str1", "str2"],
    "arrayOfInteger":[1, 2147483647, -2147483648],
    "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808],
    "arrayOfBigInteger":[922337203685477580700, -922337203685477580800],
    "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308],
    "arrayOfBoolean":[true, false, true],
    "arrayOfNull":[null, null, null, null],
    "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}],
    "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]],
    "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]]
   }"""
  val data = List.fill(N)(row)
  val dummyOption = new JSONOptions(Map.empty[String, String], new SQLConf())
  val schema =
    InferSchema.infer(spark.sparkContext.parallelize(Seq(row)), dummyOption)

  val benchmark = new Benchmark("JSON converter", N)
  benchmark.addCase("convert JSON file", 10) { _ =>
    val parser = new JacksonParser(schema, dummyOption)
    data.foreach { input =>
      parser.parse(input)
    }
  }
  benchmark.run()
}
```

```
JSON converter:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
convert JSON file                             1401 / 1461          0.1       10947.4       1.0X
```

It seems parsing time is improved by roughly ~20%

#### End-to-End test

```scala
test("Benchmark for JSON reader") {
  val N = 500 << 8
  val row =
    """{"struct":{"field1": true, "field2": 92233720368547758070},
    "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]},
    "arrayOfString":["str1", "str2"],
    "arrayOfInteger":[1, 2147483647, -2147483648],
    "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808],
    "arrayOfBigInteger":[922337203685477580700, -922337203685477580800],
    "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308],
    "arrayOfBoolean":[true, false, true],
    "arrayOfNull":[null, null, null, null],
    "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}],
    "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]],
    "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]]
   }"""
  val df = spark.sqlContext.read.json(spark.sparkContext.parallelize(List.fill(N)(row)))
  withTempPath { path =>
    df.write.format("json").save(path.getCanonicalPath)

    val benchmark = new Benchmark("JSON reader", N)
    benchmark.addCase("reading JSON file", 10) { _ =>
      spark.read.format("json").load(path.getCanonicalPath).collect()
    }
    benchmark.run()
  }
}
```

**Before**

```
JSON reader:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
reading JSON file                             6485 / 6924          0.0       50665.0       1.0X
```

**After**

```
JSON reader:                             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
reading JSON file                             6350 / 6529          0.0       49609.3       1.0X
```

## How was this patch tested?

Existing test cases should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14102 from HyukjinKwon/SPARK-16434.
2016-08-12 11:09:42 +08:00
Wenchen Fan acaf2a81ad [SPARK-17021][SQL] simplify the constructor parameters of QuantileSummaries
## What changes were proposed in this pull request?

1. `sampled` doesn't need to be `ArrayBuffer`, we never update it, but assign new value
2. `count` doesn't need to be `var`, we never mutate it.
3. `headSampled` doesn't need to be in constructor, we never pass a non-empty `headSampled` to constructor

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14603 from cloud-fan/simply.
2016-08-11 11:02:11 -07:00
Davies Liu 0f72e4f04b [SPARK-16958] [SQL] Reuse subqueries within the same query
## What changes were proposed in this pull request?

There could be multiple subqueries that generate same results, we could re-use the result instead of running it multiple times.

This PR also cleanup up how we run subqueries.

For SQL query
```sql
select id,(select avg(id) from t) from t where id > (select avg(id) from t)
```
The explain is
```
== Physical Plan ==
*Project [id#15L, Subquery subquery29 AS scalarsubquery()#35]
:  +- Subquery subquery29
:     +- *HashAggregate(keys=[], functions=[avg(id#15L)])
:        +- Exchange SinglePartition
:           +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)])
:              +- *Range (0, 1000, splits=4)
+- *Filter (cast(id#15L as double) > Subquery subquery29)
   :  +- Subquery subquery29
   :     +- *HashAggregate(keys=[], functions=[avg(id#15L)])
   :        +- Exchange SinglePartition
   :           +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)])
   :              +- *Range (0, 1000, splits=4)
   +- *Range (0, 1000, splits=4)
```
The visualized plan:

![reuse-subquery](https://cloud.githubusercontent.com/assets/40902/17573229/e578d93c-5f0d-11e6-8a3c-0150d81d3aed.png)

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #14548 from davies/subq.
2016-08-11 09:47:19 -07:00
Qifan Pu bf5cb8af4a [SPARK-16928] [SQL] Recursive call of ColumnVector::getInt() breaks JIT inlining
## What changes were proposed in this pull request?

In both `OnHeapColumnVector` and `OffHeapColumnVector`, we implemented `getInt()` with the following code pattern:
```
public int getInt(int rowId) {
if (dictionary == null)
{ return intData[rowId]; }
else
{ return dictionary.decodeToInt(dictionaryIds.getInt(rowId)); }
}
```
As `dictionaryIds` is also a `ColumnVector`, this results in a recursive call of `getInt()` and breaks JIT inlining. As a result, `getInt()` will not get inlined.

We fix this by adding a separate method `getDictId()` specific for `dictionaryIds` to use.

## How was this patch tested?

We tested the difference with the following aggregate query on a TPCDS dataset (with scale factor = 5):
```
select
  max(ss_sold_date_sk) as max_ss_sold_date_sk,
from store_sales
```
The query runtime is improved, from 202ms (before) to 159ms (after).

Author: Qifan Pu <qifan.pu@gmail.com>

Closes #14513 from ooq/SPARK-16928.
2016-08-10 14:45:13 -07:00
Liang-Chi Hsieh 19af298bb6 [SPARK-15639] [SPARK-16321] [SQL] Push down filter at RowGroups level for parquet reader
## What changes were proposed in this pull request?

The base class `SpecificParquetRecordReaderBase` used for vectorized parquet reader will try to get pushed-down filters from the given configuration. This pushed-down filters are used for RowGroups-level filtering. However, we don't set up the filters to push down into the configuration. In other words, the filters are not actually pushed down to do RowGroups-level filtering. This patch is to fix this and tries to set up the filters for pushing down to configuration for the reader.

The benchmark that excludes the time of writing Parquet file:

    test("Benchmark for Parquet") {
      val N = 500 << 12
        withParquetTable((0 until N).map(i => (101, i)), "t") {
          val benchmark = new Benchmark("Parquet reader", N)
          benchmark.addCase("reading Parquet file", 10) { iter =>
            sql("SELECT _1 FROM t where t._1 < 100").collect()
          }
          benchmark.run()
      }
    }

`withParquetTable` in default will run tests for vectorized reader non-vectorized readers. I only let it run vectorized reader.

When we set the block size of parquet as 1024 to have multiple row groups. The benchmark is:

Before this patch:

The retrieved row groups: 8063

    Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic
    Intel(R) Core(TM) i7-5557U CPU  3.10GHz
    Parquet reader:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    ------------------------------------------------------------------------------------------------
    reading Parquet file                           825 / 1233          2.5         402.6       1.0X

After this patch:

The retrieved row groups: 0

    Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic
    Intel(R) Core(TM) i7-5557U CPU  3.10GHz
    Parquet reader:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    ------------------------------------------------------------------------------------------------
    reading Parquet file                           306 /  503          6.7         149.6       1.0X

Next, I run the benchmark for non-pushdown case using the same benchmark code but with disabled pushdown configuration. This time the parquet block size is default value.

Before this patch:

    Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic
    Intel(R) Core(TM) i7-5557U CPU  3.10GHz
    Parquet reader:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    ------------------------------------------------------------------------------------------------
    reading Parquet file                           136 /  238         15.0          66.5       1.0X

After this patch:

    Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic
    Intel(R) Core(TM) i7-5557U CPU  3.10GHz
    Parquet reader:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
    ------------------------------------------------------------------------------------------------
    reading Parquet file                           124 /  193         16.5          60.7       1.0X

For non-pushdown case, from the results, I think this patch doesn't affect normal code path.

I've manually output the `totalRowCount` in `SpecificParquetRecordReaderBase` to see if this patch actually filter the row-groups. When running the above benchmark:

After this patch:
    `totalRowCount = 0`

Before this patch:
    `totalRowCount = 1024000`

## How was this patch tested?
Existing tests should be passed.

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

Closes #13701 from viirya/vectorized-reader-push-down-filter2.
2016-08-10 10:03:55 -07:00
avulanov 11a6844beb [SPARK-15899][SQL] Fix the construction of the file path with hadoop Path
## What changes were proposed in this pull request?

Fix the construction of the file path. Previous way of construction caused the creation of incorrect path on Windows.

## How was this patch tested?

Run SQL unit tests on Windows

Author: avulanov <nashb@yandex.ru>

Closes #13868 from avulanov/SPARK-15899-file.
2016-08-10 10:25:00 +01:00
Sean Owen 0578ff9681 [SPARK-16324][SQL] regexp_extract should doc that it returns empty string when match fails
## What changes were proposed in this pull request?

Doc that regexp_extract returns empty string when regex or group does not match

## How was this patch tested?

Jenkins test, with a few new test cases

Author: Sean Owen <sowen@cloudera.com>

Closes #14525 from srowen/SPARK-16324.
2016-08-10 10:14:43 +01:00
Andrew Ash bfda53f63a Typo: Fow -> For
Author: Andrew Ash <andrew@andrewash.com>

Closes #14563 from ash211/patch-8.
2016-08-10 10:09:35 +01:00
gatorsmile 2b10ebe6ac [SPARK-16185][SQL] Better Error Messages When Creating Table As Select Without Enabling Hive Support
#### What changes were proposed in this pull request?
When we do not turn on the Hive Support, the following query generates a confusing error message by Planner:
```Scala
sql("CREATE TABLE t2 SELECT a, b from t1")
```

```
assertion failed: No plan for CreateTable CatalogTable(
	Table: `t2`
	Created: Tue Aug 09 23:45:32 PDT 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Provider: hive
	Storage(InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat)), ErrorIfExists
+- Relation[a#19L,b#20L] parquet

java.lang.AssertionError: assertion failed: No plan for CreateTable CatalogTable(
	Table: `t2`
	Created: Tue Aug 09 23:45:32 PDT 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Provider: hive
	Storage(InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat)), ErrorIfExists
+- Relation[a#19L,b#20L] parquet
```

This PR is to issue a better error message:
```
Hive support is required to use CREATE Hive TABLE AS SELECT
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13886 from gatorsmile/createCatalogedTableAsSelect.
2016-08-10 17:05:50 +08:00
Davies Liu 92da22878b [SPARK-16905] SQL DDL: MSCK REPAIR TABLE
## What changes were proposed in this pull request?

MSCK REPAIR TABLE could be used to recover the partitions in external catalog based on partitions in file system.

Another syntax is: ALTER TABLE table RECOVER PARTITIONS

The implementation in this PR will only list partitions (not the files with a partition) in driver (in parallel if needed).

## How was this patch tested?

Added unit tests for it and Hive compatibility test suite.

Author: Davies Liu <davies@databricks.com>

Closes #14500 from davies/repair_table.
2016-08-09 10:04:36 -07:00
Reynold Xin 511f52f842 [SPARK-16964][SQL] Remove private[sql] and private[spark] from sql.execution package
## What changes were proposed in this pull request?
This package is meant to be internal, and as a result it does not make sense to mark things as private[sql] or private[spark]. It simply makes debugging harder when Spark developers need to inspect the plans at runtime.

This patch removes all private[sql] and private[spark] visibility modifiers in org.apache.spark.sql.execution.

## How was this patch tested?
N/A - just visibility changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #14554 from rxin/remote-private.
2016-08-09 18:22:14 +08:00
Sean Zhong bca43cd635 [SPARK-16898][SQL] Adds argument type information for typed logical plan like MapElements, TypedFilter, and AppendColumn
## What changes were proposed in this pull request?

This PR adds argument type information for typed logical plan like MapElements, TypedFilter, and AppendColumn, so that we can use these info in customized optimizer rule.

## How was this patch tested?

Existing test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14494 from clockfly/add_more_info_for_typed_operator.
2016-08-09 08:36:50 +08:00
Herman van Hovell df10658831 [SPARK-16749][SQL] Simplify processing logic in LEAD/LAG processing.
## What changes were proposed in this pull request?
The logic for LEAD/LAG processing is more complex that it needs to be. This PR fixes that.

## How was this patch tested?
Existing tests.

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

Closes #14376 from hvanhovell/SPARK-16749.
2016-08-08 16:34:57 -07:00
gatorsmile 5959df217d [SPARK-16936][SQL] Case Sensitivity Support for Refresh Temp Table
### What changes were proposed in this pull request?
Currently, the `refreshTable` API is always case sensitive.

When users use the view name without the exact case match, the API silently ignores the call. Users might expect the command has been successfully completed. However, when users run the subsequent SQL commands, they might still get the exception, like
```
Job aborted due to stage failure:
Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 7, localhost):
java.io.FileNotFoundException:
File file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-bd4b9ea6-9aec-49c5-8f05-01cff426211e/part-r-00000-0c84b915-c032-4f2e-abf5-1d48fdbddf38.snappy.parquet does not exist
```

This PR is to fix the issue.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14523 from gatorsmile/refreshTempTable.
2016-08-08 22:34:28 +08:00
gatorsmile ab126909ce [SPARK-16457][SQL] Fix Wrong Messages when CTAS with a Partition By Clause
#### What changes were proposed in this pull request?
When doing a CTAS with a Partition By clause, we got a wrong error message.

For example,
```SQL
CREATE TABLE gen__tmp
PARTITIONED BY (key string)
AS SELECT key, value FROM mytable1
```
The error message we get now is like
```
Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement(line 2, pos 0)
```

However, based on the code, the message we should get is like
```
Operation not allowed: A Create Table As Select (CTAS) statement is not allowed to create a partitioned table using Hive's file formats. Please use the syntax of "CREATE TABLE tableName USING dataSource OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a CTAS statement.(line 2, pos 0)
```

Currently, partitioning columns is part of the schema. This PR fixes the bug by changing the detection orders.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14113 from gatorsmile/ctas.
2016-08-08 22:26:44 +08:00
Sean Zhong 94a9d11ed1 [SPARK-16906][SQL] Adds auxiliary info like input class and input schema in TypedAggregateExpression
## What changes were proposed in this pull request?

This PR adds auxiliary info like input class and input schema in TypedAggregateExpression

## How was this patch tested?

Manual test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14501 from clockfly/typed_aggregation.
2016-08-08 22:20:54 +08:00
hyukjinkwon 55d6dad6f2 [SPARK-16847][SQL] Prevent to potentially read corrupt statstics on binary in Parquet vectorized reader
## What changes were proposed in this pull request?

This problem was found in [PARQUET-251](https://issues.apache.org/jira/browse/PARQUET-251) and we disabled filter pushdown on binary columns in Spark before. We enabled this after upgrading Parquet but it seems there is potential incompatibility for Parquet files written in lower Spark versions.

Currently, this does not happen in normal Parquet reader. However, In Spark, we implemented a vectorized reader, separately with Parquet's standard API. For normal Parquet reader this is being handled but not in the vectorized reader.

It is okay to just pass `FileMetaData`. This is being handled in parquet-mr (See e3b95020f7). This will prevent loading corrupt statistics in each page in Parquet.

This PR replaces the deprecated usage of constructor.

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14450 from HyukjinKwon/SPARK-16847.
2016-08-06 04:40:24 +01:00