Commit graph

4513 commits

Author SHA1 Message Date
Takuya UESHIN 0c66fe4f22 [SPARK-22002][SQL][FOLLOWUP][TEST] Add a test to check if the original schema doesn't have metadata.
## What changes were proposed in this pull request?

This is a follow-up pr of #19231 which modified the behavior to remove metadata from JDBC table schema.
This pr adds a test to check if the schema doesn't have metadata.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20585 from ueshin/issues/SPARK-22002/fup1.
2018-02-12 12:20:29 -08:00
James Thompson 5bb11411ae [SPARK-23388][SQL] Support for Parquet Binary DecimalType in VectorizedColumnReader
## What changes were proposed in this pull request?

Re-add support for parquet binary DecimalType in VectorizedColumnReader

## How was this patch tested?

Existing test suite

Author: James Thompson <jamesthomp@users.noreply.github.com>

Closes #20580 from jamesthomp/jt/add-back-binary-decimal.
2018-02-12 11:34:56 -08:00
liuxian 4a4dd4f36f [SPARK-23391][CORE] It may lead to overflow for some integer multiplication
## What changes were proposed in this pull request?
In the `getBlockData`,`blockId.reduceId` is the `Int` type, when it is greater than 2^28, `blockId.reduceId*8` will overflow
In the `decompress0`, `len` and  `unitSize` are  Int type, so `len * unitSize` may lead to  overflow
## How was this patch tested?
N/A

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

Closes #20581 from 10110346/overflow2.
2018-02-12 08:49:45 -06:00
Wenchen Fan 0e2c266de7 [SPARK-22977][SQL] fix web UI SQL tab for CTAS
## What changes were proposed in this pull request?

This is a regression in Spark 2.3.

In Spark 2.2, we have a fragile UI support for SQL data writing commands. We only track the input query plan of `FileFormatWriter` and display its metrics. This is not ideal because we don't know who triggered the writing(can be table insertion, CTAS, etc.), but it's still useful to see the metrics of the input query.

In Spark 2.3, we introduced a new mechanism: `DataWritigCommand`, to fix the UI issue entirely. Now these writing commands have real children, and we don't need to hack into the `FileFormatWriter` for the UI. This also helps with `explain`, now `explain` can show the physical plan of the input query, while in 2.2 the physical writing plan is simply `ExecutedCommandExec` and it has no child.

However there is a regression in CTAS. CTAS commands don't extend `DataWritigCommand`, and we don't have the UI hack in `FileFormatWriter` anymore, so the UI for CTAS is just an empty node. See https://issues.apache.org/jira/browse/SPARK-22977 for more information about this UI issue.

To fix it, we should apply the `DataWritigCommand` mechanism to CTAS commands.

TODO: In the future, we should refactor this part and create some physical layer code pieces for data writing, and reuse them in different writing commands. We should have different logical nodes for different operators, even some of them share some same logic, e.g. CTAS, CREATE TABLE, INSERT TABLE. Internally we can share the same physical logic.

## How was this patch tested?

manually tested.
For data source table
<img width="644" alt="1" src="https://user-images.githubusercontent.com/3182036/35874155-bdffab28-0ba6-11e8-94a8-e32e106ba069.png">
For hive table
<img width="666" alt="2" src="https://user-images.githubusercontent.com/3182036/35874161-c437e2a8-0ba6-11e8-98ed-7930f01432c5.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20521 from cloud-fan/UI.
2018-02-12 22:07:59 +08:00
Wenchen Fan 6efd5d117e [SPARK-23390][SQL] Flaky Test Suite: FileBasedDataSourceSuite in Spark 2.3/hadoop 2.7
## What changes were proposed in this pull request?

This test only fails with sbt on Hadoop 2.7, I can't reproduce it locally, but here is my speculation by looking at the code:
1. FileSystem.delete doesn't delete the directory entirely, somehow we can still open the file as a 0-length empty file.(just speculation)
2. ORC intentionally allow empty files, and the reader fails during reading without closing the file stream.

This PR improves the test to make sure all files are deleted and can't be opened.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20584 from cloud-fan/flaky-test.
2018-02-11 23:46:23 -08:00
Wenchen Fan 4bbd7443eb [SPARK-23376][SQL] creating UnsafeKVExternalSorter with BytesToBytesMap may fail
## What changes were proposed in this pull request?

This is a long-standing bug in `UnsafeKVExternalSorter` and was reported in the dev list multiple times.

When creating `UnsafeKVExternalSorter` with `BytesToBytesMap`, we need to create a `UnsafeInMemorySorter` to sort the data in `BytesToBytesMap`. The data format of the sorter and the map is same, so no data movement is required. However, both the sorter and the map need a point array for some bookkeeping work.

There is an optimization in `UnsafeKVExternalSorter`: reuse the point array between the sorter and the map, to avoid an extra memory allocation. This sounds like a reasonable optimization, the length of the `BytesToBytesMap` point array is at least 4 times larger than the number of keys(to avoid hash collision, the hash table size should be at least 2 times larger than the number of keys, and each key occupies 2 slots). `UnsafeInMemorySorter` needs the pointer array size to be 4 times of the number of entries, so we are safe to reuse the point array.

However, the number of keys of the map doesn't equal to the number of entries in the map, because `BytesToBytesMap` supports duplicated keys. This breaks the assumption of the above optimization and we may run out of space when inserting data into the sorter, and hit error
```
java.lang.IllegalStateException: There is no space for new record
   at org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.insertRecord(UnsafeInMemorySorter.java:239)
   at org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:149)
...
```

This PR fixes this bug by creating a new point array if the existing one is not big enough.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20561 from cloud-fan/bug.
2018-02-12 00:03:49 +08:00
Dongjoon Hyun 8cbcc33876 [SPARK-23186][SQL] Initialize DriverManager first before loading JDBC Drivers
## What changes were proposed in this pull request?

Since some JDBC Drivers have class initialization code to call `DriverManager`, we need to initialize `DriverManager` first in order to avoid potential executor-side **deadlock** situations like the following (or [STORM-2527](https://issues.apache.org/jira/browse/STORM-2527)).

```
Thread 9587: (state = BLOCKED)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance0(java.lang.reflect.Constructor, java.lang.Object[]) bci=0 (Compiled frame; information may be imprecise)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=85, line=62 (Compiled frame)
 - sun.reflect.DelegatingConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=5, line=45 (Compiled frame)
 - java.lang.reflect.Constructor.newInstance(java.lang.Object[]) bci=79, line=423 (Compiled frame)
 - java.lang.Class.newInstance() bci=138, line=442 (Compiled frame)
 - java.util.ServiceLoader$LazyIterator.nextService() bci=119, line=380 (Interpreted frame)
 - java.util.ServiceLoader$LazyIterator.next() bci=11, line=404 (Interpreted frame)
 - java.util.ServiceLoader$1.next() bci=37, line=480 (Interpreted frame)
 - java.sql.DriverManager$2.run() bci=21, line=603 (Interpreted frame)
 - java.sql.DriverManager$2.run() bci=1, line=583 (Interpreted frame)
 - java.security.AccessController.doPrivileged(java.security.PrivilegedAction) bci=0 (Compiled frame)
 - java.sql.DriverManager.loadInitialDrivers() bci=27, line=583 (Interpreted frame)
 - java.sql.DriverManager.<clinit>() bci=32, line=101 (Interpreted frame)
 - org.apache.phoenix.mapreduce.util.ConnectionUtil.getConnection(java.lang.String, java.lang.Integer, java.lang.String, java.util.Properties) bci=12, line=98 (Interpreted frame)
 - org.apache.phoenix.mapreduce.util.ConnectionUtil.getInputConnection(org.apache.hadoop.conf.Configuration, java.util.Properties) bci=22, line=57 (Interpreted frame)
 - org.apache.phoenix.mapreduce.PhoenixInputFormat.getQueryPlan(org.apache.hadoop.mapreduce.JobContext, org.apache.hadoop.conf.Configuration) bci=61, line=116 (Interpreted frame)
 - org.apache.phoenix.mapreduce.PhoenixInputFormat.createRecordReader(org.apache.hadoop.mapreduce.InputSplit, org.apache.hadoop.mapreduce.TaskAttemptContext) bci=10, line=71 (Interpreted frame)
 - org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(org.apache.spark.rdd.NewHadoopRDD, org.apache.spark.Partition, org.apache.spark.TaskContext) bci=233, line=156 (Interpreted frame)

Thread 9170: (state = BLOCKED)
 - org.apache.phoenix.jdbc.PhoenixDriver.<clinit>() bci=35, line=125 (Interpreted frame)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance0(java.lang.reflect.Constructor, java.lang.Object[]) bci=0 (Compiled frame)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=85, line=62 (Compiled frame)
 - sun.reflect.DelegatingConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=5, line=45 (Compiled frame)
 - java.lang.reflect.Constructor.newInstance(java.lang.Object[]) bci=79, line=423 (Compiled frame)
 - java.lang.Class.newInstance() bci=138, line=442 (Compiled frame)
 - org.apache.spark.sql.execution.datasources.jdbc.DriverRegistry$.register(java.lang.String) bci=89, line=46 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$createConnectionFactory$2.apply() bci=7, line=53 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$createConnectionFactory$2.apply() bci=1, line=52 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD$$anon$1.<init>(org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD, org.apache.spark.Partition, org.apache.spark.TaskContext) bci=81, line=347 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD.compute(org.apache.spark.Partition, org.apache.spark.TaskContext) bci=7, line=339 (Interpreted frame)
```

## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20359 from dongjoon-hyun/SPARK-23186.
2018-02-09 12:54:57 +08:00
Wenchen Fan a75f927173 [SPARK-23268][SQL][FOLLOWUP] Reorganize packages in data source V2
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/20435.

While reorganizing the packages for streaming data source v2, the top level stream read/write support interfaces should not be in the reader/writer package, but should be in the `sources.v2` package, to follow the `ReadSupport`, `WriteSupport`, etc.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20509 from cloud-fan/followup.
2018-02-08 19:20:11 +08:00
Wenchen Fan 7f5f5fb129 [SPARK-23348][SQL] append data using saveAsTable should adjust the data types
## What changes were proposed in this pull request?

For inserting/appending data to an existing table, Spark should adjust the data types of the input query according to the table schema, or fail fast if it's uncastable.

There are several ways to insert/append data: SQL API, `DataFrameWriter.insertInto`, `DataFrameWriter.saveAsTable`. The first 2 ways create `InsertIntoTable` plan, and the last way creates `CreateTable` plan. However, we only adjust input query data types for `InsertIntoTable`, and users may hit weird errors when appending data using `saveAsTable`. See the JIRA for the error case.

This PR fixes this bug by adjusting data types for `CreateTable` too.

## How was this patch tested?

new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20527 from cloud-fan/saveAsTable.
2018-02-08 00:08:54 -08:00
Tathagata Das 30295bf5a6 [SPARK-23092][SQL] Migrate MemoryStream to DataSourceV2 APIs
## What changes were proposed in this pull request?

This PR migrates the MemoryStream to DataSourceV2 APIs.

One additional change is in the reported keys in StreamingQueryProgress.durationMs. "getOffset" and "getBatch" replaced with "setOffsetRange" and "getEndOffset" as tracking these make more sense. Unit tests changed accordingly.

## How was this patch tested?
Existing unit tests, few updated unit tests.

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

Closes #20445 from tdas/SPARK-23092.
2018-02-07 15:22:53 -08:00
Liang-Chi Hsieh 9841ae0313 [SPARK-23345][SQL] Remove open stream record even closing it fails
## What changes were proposed in this pull request?

When `DebugFilesystem` closes opened stream, if any exception occurs, we still need to remove the open stream record from `DebugFilesystem`. Otherwise, it goes to report leaked filesystem connection.

## How was this patch tested?

Existing tests.

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

Closes #20524 from viirya/SPARK-23345.
2018-02-07 09:48:49 -08:00
gatorsmile 9775df67f9 [SPARK-23122][PYSPARK][FOLLOWUP] Replace registerTempTable by createOrReplaceTempView
## What changes were proposed in this pull request?
Replace `registerTempTable` by `createOrReplaceTempView`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20523 from gatorsmile/updateExamples.
2018-02-07 23:24:16 +09:00
gatorsmile c36fecc3b4 [SPARK-23327][SQL] Update the description and tests of three external API or functions
## What changes were proposed in this pull request?
Update the description and tests of three external API or functions `createFunction `, `length` and `repartitionByRange `

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20495 from gatorsmile/updateFunc.
2018-02-06 16:46:43 -08:00
Wenchen Fan b96a083b1c [SPARK-23315][SQL] failed to get output from canonicalized data source v2 related plans
## What changes were proposed in this pull request?

`DataSourceV2Relation`  keeps a `fullOutput` and resolves the real output on demand by column name lookup. i.e.
```
lazy val output: Seq[Attribute] = reader.readSchema().map(_.name).map { name =>
  fullOutput.find(_.name == name).get
}
```

This will be broken after we canonicalize the plan, because all attribute names become "None", see https://github.com/apache/spark/blob/v2.3.0-rc1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala#L42

To fix this, `DataSourceV2Relation` should just keep `output`, and update the `output` when doing column pruning.

## How was this patch tested?

a new test case

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20485 from cloud-fan/canonicalize.
2018-02-06 12:43:45 -08:00
Wenchen Fan ac7454cac0 [SPARK-23312][SQL][FOLLOWUP] add a config to turn off vectorized cache reader
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/20483 tried to provide a way to turn off the new columnar cache reader, to restore the behavior in 2.2. However even we turn off that config, the behavior is still different than 2.2.

If the output data are rows, we still enable whole stage codegen for the scan node, which is different with 2.2, we should also fix it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20513 from cloud-fan/cache.
2018-02-06 12:27:37 -08:00
Xingbo Jiang c2766b07b4 [SPARK-23330][WEBUI] Spark UI SQL executions page throws NPE
## What changes were proposed in this pull request?

Spark SQL executions page throws the following error and the page crashes:
```
HTTP ERROR 500
Problem accessing /SQL/. Reason:

Server Error
Caused by:
java.lang.NullPointerException
at scala.collection.immutable.StringOps$.length$extension(StringOps.scala:47)
at scala.collection.immutable.StringOps.length(StringOps.scala:47)
at scala.collection.IndexedSeqOptimized$class.isEmpty(IndexedSeqOptimized.scala:27)
at scala.collection.immutable.StringOps.isEmpty(StringOps.scala:29)
at scala.collection.TraversableOnce$class.nonEmpty(TraversableOnce.scala:111)
at scala.collection.immutable.StringOps.nonEmpty(StringOps.scala:29)
at org.apache.spark.sql.execution.ui.ExecutionTable.descriptionCell(AllExecutionsPage.scala:182)
at org.apache.spark.sql.execution.ui.ExecutionTable.row(AllExecutionsPage.scala:155)
at org.apache.spark.sql.execution.ui.ExecutionTable$$anonfun$8.apply(AllExecutionsPage.scala:204)
at org.apache.spark.sql.execution.ui.ExecutionTable$$anonfun$8.apply(AllExecutionsPage.scala:204)
at org.apache.spark.ui.UIUtils$$anonfun$listingTable$2.apply(UIUtils.scala:339)
at org.apache.spark.ui.UIUtils$$anonfun$listingTable$2.apply(UIUtils.scala:339)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at org.apache.spark.ui.UIUtils$.listingTable(UIUtils.scala:339)
at org.apache.spark.sql.execution.ui.ExecutionTable.toNodeSeq(AllExecutionsPage.scala:203)
at org.apache.spark.sql.execution.ui.AllExecutionsPage.render(AllExecutionsPage.scala:67)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)
at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584)
at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)
at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512)
at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)
at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
at org.eclipse.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:213)
at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
at org.eclipse.jetty.server.Server.handle(Server.java:534)
at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:320)
at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108)
at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)
at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
at java.lang.Thread.run(Thread.java:748)
```

One of the possible reason that this page fails may be the `SparkListenerSQLExecutionStart` event get dropped before processed, so the execution description and details don't get updated.
This was not a issue in 2.2 because it would ignore any job start event that arrives before the corresponding execution start event, which doesn't sound like a good decision.

We shall try to handle the null values in the front page side, that is, try to give a default value when `execution.details` or `execution.description` is null.
Another possible approach is not to spill the `LiveExecutionData` in `SQLAppStatusListener.update(exec: LiveExecutionData)` if `exec.details` is null. This is not ideal because this way you will not see the execution if `SparkListenerSQLExecutionStart` event is lost, because `AllExecutionsPage` only read executions from KVStore.

## How was this patch tested?

After the change, the page shows the following:
![image](https://user-images.githubusercontent.com/4784782/35775480-28cc5fde-093e-11e8-8ccc-f58c2ef4a514.png)

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

Closes #20502 from jiangxb1987/executionPage.
2018-02-05 14:17:11 -08:00
Shixiong Zhu a6bf3db207 [SPARK-23307][WEBUI] Sort jobs/stages/tasks/queries with the completed timestamp before cleaning up them
## What changes were proposed in this pull request?

Sort jobs/stages/tasks/queries with the completed timestamp before cleaning up them to make the behavior consistent with 2.2.

## How was this patch tested?

- Jenkins.
- Manually ran the following codes and checked the UI for jobs/stages/tasks/queries.

```
spark.ui.retainedJobs 10
spark.ui.retainedStages 10
spark.sql.ui.retainedExecutions 10
spark.ui.retainedTasks 10
```

```
new Thread() {
  override def run() {
    spark.range(1, 2).foreach { i =>
        Thread.sleep(10000)
    }
  }
}.start()

Thread.sleep(5000)

for (_ <- 1 to 20) {
    new Thread() {
      override def run() {
        spark.range(1, 2).foreach { i =>
        }
      }
    }.start()
}

Thread.sleep(15000)
  spark.range(1, 2).foreach { i =>
}

sc.makeRDD(1 to 100, 100).foreach { i =>
}
```

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20481 from zsxwing/SPARK-23307.
2018-02-05 18:41:49 +08:00
Yuming Wang 6fb3fd1536 [SPARK-22036][SQL][FOLLOWUP] Fix decimalArithmeticOperations.sql
## What changes were proposed in this pull request?

Fix decimalArithmeticOperations.sql test

## How was this patch tested?

N/A

Author: Yuming Wang <wgyumg@gmail.com>
Author: wangyum <wgyumg@gmail.com>
Author: Yuming Wang <yumwang@ebay.com>

Closes #20498 from wangyum/SPARK-22036.
2018-02-04 09:15:48 -08:00
Dongjoon Hyun 522e0b1866 [SPARK-23305][SQL][TEST] Test spark.sql.files.ignoreMissingFiles for all file-based data sources
## What changes were proposed in this pull request?

Like Parquet, all file-based data source handles `spark.sql.files.ignoreMissingFiles` correctly. We had better have a test coverage for feature parity and in order to prevent future accidental regression for all data sources.

## How was this patch tested?

Pass Jenkins with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20479 from dongjoon-hyun/SPARK-23305.
2018-02-03 00:04:00 -08:00
Wenchen Fan fe73cb4b43 [SPARK-23317][SQL] rename ContinuousReader.setOffset to setStartOffset
## What changes were proposed in this pull request?

In the document of `ContinuousReader.setOffset`, we say this method is used to specify the start offset. We also have a `ContinuousReader.getStartOffset` to get the value back. I think it makes more sense to rename `ContinuousReader.setOffset` to `setStartOffset`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20486 from cloud-fan/rename.
2018-02-02 20:49:08 -08:00
Reynold Xin 3ff83ad43a [SQL] Minor doc update: Add an example in DataFrameReader.schema
## What changes were proposed in this pull request?
This patch adds a small example to the schema string definition of schema function. It isn't obvious how to use it, so an example would be useful.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #20491 from rxin/schema-doc.
2018-02-02 20:36:27 -08:00
Wenchen Fan b9503fcbb3 [SPARK-23312][SQL] add a config to turn off vectorized cache reader
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-23309 reported a performance regression about cached table in Spark 2.3. While the investigating is still going on, this PR adds a conf to turn off the vectorized cache reader, to unblock the 2.3 release.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20483 from cloud-fan/cache.
2018-02-02 22:43:28 +08:00
Wenchen Fan 19c7c7ebde [SPARK-23301][SQL] data source column pruning should work for arbitrary expressions
## What changes were proposed in this pull request?

This PR fixes a mistake in the `PushDownOperatorsToDataSource` rule, the column pruning logic is incorrect about `Project`.

## How was this patch tested?

a new test case for column pruning with arbitrary expressions, and improve the existing tests to make sure the `PushDownOperatorsToDataSource` really works.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20476 from cloud-fan/push-down.
2018-02-01 20:44:46 -08:00
Liang-Chi Hsieh 90848d5074 [SPARK-23284][SQL] Document the behavior of several ColumnVector's get APIs when accessing null slot
## What changes were proposed in this pull request?

For some ColumnVector get APIs such as getDecimal, getBinary, getStruct, getArray, getInterval, getUTF8String, we should clearly document their behaviors when accessing null slot. They should return null in this case. Then we can remove null checks from the places using above APIs.

For the APIs of primitive values like getInt, getInts, etc., this also documents their behaviors when accessing null slots. Their returning values are undefined and can be anything.

## How was this patch tested?

Added tests into `ColumnarBatchSuite`.

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

Closes #20455 from viirya/SPARK-23272-followup.
2018-02-02 10:18:32 +08:00
Wenchen Fan 73da3b6968 [SPARK-23293][SQL] fix data source v2 self join
## What changes were proposed in this pull request?

`DataSourceV2Relation` should extend `MultiInstanceRelation`, to take care of self-join.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20466 from cloud-fan/dsv2-selfjoin.
2018-02-01 10:48:34 -08:00
Wang Gengliang ffbca84519 [SPARK-23202][SQL] Add new API in DataSourceWriter: onDataWriterCommit
## What changes were proposed in this pull request?

The current DataSourceWriter API makes it hard to implement `onTaskCommit(taskCommit: TaskCommitMessage)` in `FileCommitProtocol`.
In general, on receiving commit message, driver can start processing messages(e.g. persist messages into files) before all the messages are collected.

The proposal to add a new API:
`add(WriterCommitMessage message)`:  Handles a commit message on receiving from a successful data writer.

This should make the whole API of DataSourceWriter compatible with `FileCommitProtocol`, and more flexible.

There was another radical attempt in #20386.  This one should be more reasonable.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20454 from gengliangwang/write_api.
2018-02-01 20:39:15 +08:00
Takuya UESHIN 89e8d556b9 [SPARK-23280][SQL][FOLLOWUP] Enable MutableColumnarRow.getMap().
## What changes were proposed in this pull request?

This is a followup pr of #20450.
We should've enabled `MutableColumnarRow.getMap()` as well.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20471 from ueshin/issues/SPARK-23280/fup2.
2018-02-01 21:28:53 +09:00
Takuya UESHIN 8bb70b068e [SPARK-23280][SQL][FOLLOWUP] Fix Java style check issues.
## What changes were proposed in this pull request?

This is a follow-up of #20450 which broke lint-java checks.
This pr fixes the lint-java issues.

```
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java:[20,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java:[21,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
[ERROR] src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.catalyst.util.MapData.
```

## How was this patch tested?

Checked manually in my local environment.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20468 from ueshin/issues/SPARK-23280/fup1.
2018-02-01 21:25:02 +09:00
Xingbo Jiang cc41245fa3 [SPARK-23188][SQL] Make vectorized columar reader batch size configurable
## What changes were proposed in this pull request?

This PR include the following changes:
- Make the capacity of `VectorizedParquetRecordReader` configurable;
- Make the capacity of `OrcColumnarBatchReader` configurable;
- Update the error message when required capacity in writable columnar vector cannot be fulfilled.

## How was this patch tested?

N/A

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

Closes #20361 from jiangxb1987/vectorCapacity.
2018-02-01 12:56:07 +08:00
Wang Gengliang 56ae32657e [SPARK-23268][SQL] Reorganize packages in data source V2
## What changes were proposed in this pull request?
1. create a new package for partitioning/distribution related classes.
    As Spark will add new concrete implementations of `Distribution` in new releases, it is good to
    have a new package for partitioning/distribution related classes.

2. move streaming related class to package `org.apache.spark.sql.sources.v2.reader/writer.streaming`, instead of `org.apache.spark.sql.sources.v2.streaming.reader/writer`.
So that the there won't be package reader/writer inside package streaming, which is quite confusing.
Before change:
```
v2
├── reader
├── streaming
│   ├── reader
│   └── writer
└── writer
```

After change:
```
v2
├── reader
│   └── streaming
└── writer
    └── streaming
```
## How was this patch tested?
Unit test.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20435 from gengliangwang/new_pkg.
2018-01-31 20:33:51 -08:00
caoxuewen 2ac895be90 [SPARK-23247][SQL] combines Unsafe operations and statistics operations in Scan Data Source
## What changes were proposed in this pull request?

Currently, we scan the execution plan of the data source, first the unsafe operation of each row of data, and then re traverse the data for the count of rows. In terms of performance, this is not necessary. this PR combines the two operations and makes statistics on the number of rows while performing the unsafe operation.

Before modified,

```
val unsafeRow = rdd.mapPartitionsWithIndexInternal { (index, iter) =>
val proj = UnsafeProjection.create(schema)
 proj.initialize(index)
iter.map(proj)
}

val numOutputRows = longMetric("numOutputRows")
unsafeRow.map { r =>
numOutputRows += 1
 r
}
```
After modified,

    val numOutputRows = longMetric("numOutputRows")

    rdd.mapPartitionsWithIndexInternal { (index, iter) =>
      val proj = UnsafeProjection.create(schema)
      proj.initialize(index)
      iter.map( r => {
        numOutputRows += 1
        proj(r)
      })
    }

## How was this patch tested?

the existed test cases.

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

Closes #20415 from heary-cao/DataSourceScanExec.
2018-02-01 12:05:12 +08:00
Wenchen Fan 52e00f7066 [SPARK-23280][SQL] add map type support to ColumnVector
## What changes were proposed in this pull request?

Fill the last missing piece of `ColumnVector`: the map type support.

The idea is similar to the array type support. A map is basically 2 arrays: keys and values. We ask the implementations to provide a key array, a value array, and an offset and length to specify the range of this map in the key/value array.

In `WritableColumnVector`, we put the key array in first child vector, and value array in second child vector, and offsets and lengths in the current vector, which is very similar to how array type is implemented here.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20450 from cloud-fan/map.
2018-02-01 11:56:06 +08:00
Dilip Biswal 9ff1d96f01 [SPARK-23281][SQL] Query produces results in incorrect order when a composite order by clause refers to both original columns and aliases
## What changes were proposed in this pull request?
Here is the test snippet.
``` SQL
scala> Seq[(Integer, Integer)](
     |         (1, 1),
     |         (1, 3),
     |         (2, 3),
     |         (3, 3),
     |         (4, null),
     |         (5, null)
     |       ).toDF("key", "value").createOrReplaceTempView("src")

scala> sql(
     |         """
     |           |SELECT MAX(value) as value, key as col2
     |           |FROM src
     |           |GROUP BY key
     |           |ORDER BY value desc, key
     |         """.stripMargin).show
+-----+----+
|value|col2|
+-----+----+
|    3|   3|
|    3|   2|
|    3|   1|
| null|   5|
| null|   4|
+-----+----+
```SQL
Here is the explain output :

```SQL
== Parsed Logical Plan ==
'Sort ['value DESC NULLS LAST, 'key ASC NULLS FIRST], true
+- 'Aggregate ['key], ['MAX('value) AS value#9, 'key AS col2#10]
   +- 'UnresolvedRelation `src`

== Analyzed Logical Plan ==
value: int, col2: int
Project [value#9, col2#10]
+- Sort [value#9 DESC NULLS LAST, col2#10 DESC NULLS LAST], true
   +- Aggregate [key#5], [max(value#6) AS value#9, key#5 AS col2#10]
      +- SubqueryAlias src
         +- Project [_1#2 AS key#5, _2#3 AS value#6]
            +- LocalRelation [_1#2, _2#3]
``` SQL
The sort direction is being wrongly changed from ASC to DSC while resolving ```Sort``` in
resolveAggregateFunctions.

The above testcase models TPCDS-Q71 and thus we have the same issue in Q71 as well.

## How was this patch tested?
A few tests are added in SQLQuerySuite.

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

Closes #20453 from dilipbiswal/local_spark.
2018-01-31 13:52:47 -08:00
Glen Takahashi 8c21170dec [SPARK-23249][SQL] Improved block merging logic for partitions
## What changes were proposed in this pull request?

Change DataSourceScanExec so that when grouping blocks together into partitions, also checks the end of the sorted list of splits to more efficiently fill out partitions.

## How was this patch tested?

Updated old test to reflect the new logic, which causes the # of partitions to drop from 4 -> 3
Also, a current test exists to test large non-splittable files at c575977a59/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala (L346)

## Rationale

The current bin-packing method of next-fit descending for blocks into partitions is sub-optimal in a lot of cases and will result in extra partitions, un-even distribution of block-counts across partitions, and un-even distribution of partition sizes.

As an example, 128 files ranging from 1MB, 2MB,...127MB,128MB. will result in 82 partitions with the current algorithm, but only 64 using this algorithm. Also in this example, the max # of blocks per partition in NFD is 13, while in this algorithm is is 2.

More generally, running a simulation of 1000 runs using 128MB blocksize, between 1-1000 normally distributed file sizes between 1-500Mb, you can see an improvement of approx 5% reduction of partition counts, and a large reduction in standard deviation of blocks per partition.

This algorithm also runs in O(n) time as NFD does, and in every case is strictly better results than NFD.

Overall, the more even distribution of blocks across partitions and therefore reduced partition counts should result in a small but significant performance increase across the board

Author: Glen Takahashi <gtakahashi@palantir.com>

Closes #20372 from glentakahashi/feature/improved-block-merging.
2018-02-01 01:14:01 +08:00
Wenchen Fan 48dd6a4c79 revert [SPARK-22785][SQL] remove ColumnVector.anyNullsSet
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/19980 , we thought `anyNullsSet` can be simply implemented by `numNulls() > 0`. This is logically true, but may have performance problems.

`OrcColumnVector` is an example. It doesn't have the `numNulls` property, only has a `noNulls` property. We will lose a lot of performance if we use `numNulls() > 0` to check null.

This PR simply revert #19980, with a renaming to call it `hasNull`. Better name suggestions are welcome, e.g. `nullable`?

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20452 from cloud-fan/null.
2018-02-01 00:24:42 +08:00
Wenchen Fan 695f7146bc [SPARK-23272][SQL] add calendar interval type support to ColumnVector
## What changes were proposed in this pull request?

`ColumnVector` is aimed to support all the data types, but `CalendarIntervalType` is missing. Actually we do support interval type for inner fields, e.g. `ColumnarRow`, `ColumnarArray` both support interval type. It's weird if we don't support interval type at the top level.

This PR adds the interval type support.

This PR also makes `ColumnVector.getChild` protect. We need it public because `MutableColumnaRow.getInterval` needs it. Now the interval implementation is in `ColumnVector.getInterval`.

## How was this patch tested?

a new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20438 from cloud-fan/interval.
2018-01-31 15:13:15 +08:00
jerryshao 8c6a9c90a3 [SPARK-23279][SS] Avoid triggering distributed job for Console sink
## What changes were proposed in this pull request?

Console sink will redistribute collected local data and trigger a distributed job in each batch, this is not necessary, so here change to local job.

## How was this patch tested?

Existing UT and manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #20447 from jerryshao/console-minor.
2018-01-31 13:59:21 +08:00
gatorsmile ca04c3ff23 [SPARK-23274][SQL] Fix ReplaceExceptWithFilter when the right's Filter contains the references that are not in the left output
## What changes were proposed in this pull request?
This PR is to fix the `ReplaceExceptWithFilter` rule when the right's Filter contains the references that are not in the left output.

Before this PR, we got the error like
```
java.util.NoSuchElementException: key not found: a
  at scala.collection.MapLike$class.default(MapLike.scala:228)
  at scala.collection.AbstractMap.default(Map.scala:59)
  at scala.collection.MapLike$class.apply(MapLike.scala:141)
  at scala.collection.AbstractMap.apply(Map.scala:59)
```

After this PR, `ReplaceExceptWithFilter ` will not take an effect in this case.

## How was this patch tested?
Added tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20444 from gatorsmile/fixReplaceExceptWithFilter.
2018-01-30 20:05:57 -08:00
gatorsmile 31c00ad8b0 [SPARK-23267][SQL] Increase spark.sql.codegen.hugeMethodLimit to 65535
## What changes were proposed in this pull request?
Still saw the performance regression introduced by `spark.sql.codegen.hugeMethodLimit` in our internal workloads. There are two major issues in the current solution.
- The size of the complied byte code is not identical to the bytecode size of the method. The detection is still not accurate.
- The bytecode size of a single operator (e.g., `SerializeFromObject`) could still exceed 8K limit. We saw the performance regression in such scenario.

Since it is close to the release of 2.3, we decide to increase it to 64K for avoiding the perf regression.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20434 from gatorsmile/revertConf.
2018-01-30 11:33:30 -08:00
Liang-Chi Hsieh 84bcf9dc88 [SPARK-23222][SQL] Make DataFrameRangeSuite not flaky
## What changes were proposed in this pull request?

It is reported that the test `Cancelling stage in a query with Range` in `DataFrameRangeSuite` fails a few times in unrelated PRs. I personally also saw it too in my PR.

This test is not very flaky actually but only fails occasionally. Based on how the test works, I guess that is because `range` finishes before the listener calls `cancelStage`.

I increase the range number from `1000000000L` to `100000000000L` and count the range in one partition. I also reduce the `interval` of checking stage id. Hopefully it can make the test not flaky anymore.

## How was this patch tested?

The modified tests.

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

Closes #20431 from viirya/SPARK-23222.
2018-01-30 21:00:29 +08:00
gatorsmile 7a2ada223e [SPARK-23261][PYSPARK] Rename Pandas UDFs
## What changes were proposed in this pull request?
Rename the public APIs and names of pandas udfs.

- `PANDAS SCALAR UDF` -> `SCALAR PANDAS UDF`
- `PANDAS GROUP MAP UDF` -> `GROUPED MAP PANDAS UDF`
- `PANDAS GROUP AGG UDF` -> `GROUPED AGG PANDAS UDF`

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20428 from gatorsmile/renamePandasUDFs.
2018-01-30 21:55:55 +09:00
Wenchen Fan 0a9ac0248b [SPARK-23260][SPARK-23262][SQL] several data source v2 naming cleanup
## What changes were proposed in this pull request?

All other classes in the reader/writer package doesn't have `V2` in their names, and the streaming reader/writer don't have `V2` either. It's more consistent to remove `V2` from `DataSourceV2Reader` and `DataSourceVWriter`.

Also rename `DataSourceV2Option` to remote the `V2`, we should only have `V2` in the root interface: `DataSourceV2`.

This PR also fixes some places that the mix-in interface doesn't extend the interface it aimed to mix in.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20427 from cloud-fan/ds-v2.
2018-01-30 19:43:17 +08:00
Henry Robinson 8b983243e4 [SPARK-23157][SQL] Explain restriction on column expression in withColumn()
## What changes were proposed in this pull request?

It's not obvious from the comments that any added column must be a
function of the dataset that we are adding it to. Add a comment to
that effect to Scala, Python and R Data* methods.

Author: Henry Robinson <henry@cloudera.com>

Closes #20429 from henryr/SPARK-23157.
2018-01-29 22:19:59 -08:00
Xingbo Jiang b375397b16 [SPARK-23207][SQL][FOLLOW-UP] Don't perform local sort for DataFrame.repartition(1)
## What changes were proposed in this pull request?

In `ShuffleExchangeExec`, we don't need to insert extra local sort before round-robin partitioning, if the new partitioning has only 1 partition, because under that case all output rows go to the same partition.

## How was this patch tested?

The existing test cases.

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

Closes #20426 from jiangxb1987/repartition1.
2018-01-30 11:40:42 +08:00
gatorsmile e30b34f7bd [SPARK-22916][SQL][FOLLOW-UP] Update the Description of Join Selection
## What changes were proposed in this pull request?
This PR is to update the description of the join algorithm changes.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20420 from gatorsmile/followUp22916.
2018-01-29 10:29:42 -08:00
Herman van Hovell 2d903cf9d3 [SPARK-23223][SQL] Make stacking dataset transforms more performant
## What changes were proposed in this pull request?
It is a common pattern to apply multiple transforms to a `Dataset` (using `Dataset.withColumn` for example. This is currently quite expensive because we run `CheckAnalysis` on the full plan and create an encoder for each intermediate `Dataset`.

This PR extends the usage of the `AnalysisBarrier` to include `CheckAnalysis`. By doing this we hide the already analyzed plan  from `CheckAnalysis` because barrier is a `LeafNode`. The `AnalysisBarrier` is in the `FinishAnalysis` phase of the optimizer.

We also make binding the `Dataset` encoder lazy. The bound encoder is only needed when we materialize the dataset.

## How was this patch tested?
Existing test should cover this.

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

Closes #20402 from hvanhovell/SPARK-23223.
2018-01-29 09:00:54 -08:00
xubo245 fbce2ed0fa [SPARK-23059][SQL][TEST] Correct some improper with view related method usage
## What changes were proposed in this pull request?

Correct some improper with view related method usage
Only change test cases

like:

```
 test("list global temp views") {
    try {
      sql("CREATE GLOBAL TEMP VIEW v1 AS SELECT 3, 4")
      sql("CREATE TEMP VIEW v2 AS SELECT 1, 2")

      checkAnswer(sql(s"SHOW TABLES IN $globalTempDB"),
        Row(globalTempDB, "v1", true) ::
        Row("", "v2", true) :: Nil)

      assert(spark.catalog.listTables(globalTempDB).collect().toSeq.map(_.name) == Seq("v1", "v2"))
    } finally {
      spark.catalog.dropTempView("v1")
      spark.catalog.dropGlobalTempView("v2")
    }
  }
```

other change please review the code.
## How was this patch tested?

See test case.

Author: xubo245 <601450868@qq.com>

Closes #20250 from xubo245/DropTempViewError.
2018-01-29 08:58:14 -08:00
Wang Gengliang badf0d0e0d [SPARK-23219][SQL] Rename ReadTask to DataReaderFactory in data source v2
## What changes were proposed in this pull request?

Currently we have `ReadTask` in data source v2 reader, while in writer we have `DataWriterFactory`.
To make the naming consistent and better, renaming `ReadTask` to `DataReaderFactory`.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20397 from gengliangwang/rename.
2018-01-30 00:50:49 +08:00
Jose Torres 49b0207dc9 [SPARK-23196] Unify continuous and microbatch V2 sinks
## What changes were proposed in this pull request?

Replace streaming V2 sinks with a unified StreamWriteSupport interface, with a shim to use it with microbatch execution.

Add a new SQL config to use for disabling V2 sinks, falling back to the V1 sink implementation.

## How was this patch tested?

Existing tests, which in the case of Kafka (the only existing continuous V2 sink) now use V2 for microbatch.

Author: Jose Torres <jose@databricks.com>

Closes #20369 from jose-torres/streaming-sink.
2018-01-29 13:10:38 +08:00
CCInCharge 686a622c93 [SPARK-23250][DOCS] Typo in JavaDoc/ScalaDoc for DataFrameWriter
## What changes were proposed in this pull request?

Fix typo in ScalaDoc for DataFrameWriter - originally stated "This is applicable for all file-based data sources (e.g. Parquet, JSON) staring Spark 2.1.0", should be "starting with Spark 2.1.0".

## How was this patch tested?

Check of correct spelling in ScalaDoc

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

Author: CCInCharge <charles.l.chen.clc@gmail.com>

Closes #20417 from CCInCharge/master.
2018-01-28 14:55:43 -06:00
Jose Torres 6328868e52 [SPARK-23245][SS][TESTS] Don't access lastExecution.executedPlan in StreamTest
## What changes were proposed in this pull request?

`lastExecution.executedPlan` is lazy val so accessing it in StreamTest may need to acquire the lock of `lastExecution`. It may be waiting forever when the streaming thread is holding it and running a continuous Spark job.

This PR changes to check if `s.lastExecution` is null to avoid accessing `lastExecution.executedPlan`.

## How was this patch tested?

Jenkins

Author: Jose Torres <jose@databricks.com>

Closes #20413 from zsxwing/SPARK-23245.
2018-01-26 23:06:03 -08:00
Dongjoon Hyun e7bc9f0524 [MINOR][SS][DOC] Fix Trigger Scala/Java doc examples
## What changes were proposed in this pull request?

This PR fixes Scala/Java doc examples in `Trigger.java`.

## How was this patch tested?

N/A.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20401 from dongjoon-hyun/SPARK-TRIGGER.
2018-01-26 18:57:32 -06:00
Wenchen Fan 5b5447c68a [SPARK-23214][SQL] cached data should not carry extra hint info
## What changes were proposed in this pull request?

This is a regression introduced by https://github.com/apache/spark/pull/19864

When we lookup cache, we should not carry the hint info, as this cache entry might be added by a plan having hint info, while the input plan for this lookup may not have hint info, or have different hint info.

## How was this patch tested?

a new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20394 from cloud-fan/cache.
2018-01-26 16:46:51 -08:00
Xingbo Jiang 94c67a76ec [SPARK-23207][SQL] Shuffle+Repartition on a DataFrame could lead to incorrect answers
## What changes were proposed in this pull request?

Currently shuffle repartition uses RoundRobinPartitioning, the generated result is nondeterministic since the sequence of input rows are not determined.

The bug can be triggered when there is a repartition call following a shuffle (which would lead to non-deterministic row ordering), as the pattern shows below:
upstream stage -> repartition stage -> result stage
(-> indicate a shuffle)
When one of the executors process goes down, some tasks on the repartition stage will be retried and generate inconsistent ordering, and some tasks of the result stage will be retried generating different data.

The following code returns 931532, instead of 1000000:
```
import scala.sys.process._

import org.apache.spark.TaskContext
val res = spark.range(0, 1000 * 1000, 1).repartition(200).map { x =>
  x
}.repartition(200).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 2) {
    throw new Exception("pkill -f java".!!)
  }
  x
}
res.distinct().count()
```

In this PR, we propose a most straight-forward way to fix this problem by performing a local sort before partitioning, after we make the input row ordering deterministic, the function from rows to partitions is fully deterministic too.

The downside of the approach is that with extra local sort inserted, the performance of repartition() will go down, so we add a new config named `spark.sql.execution.sortBeforeRepartition` to control whether this patch is applied. The patch is default enabled to be safe-by-default, but user may choose to manually turn it off to avoid performance regression.

This patch also changes the output rows ordering of repartition(), that leads to a bunch of test cases failure because they are comparing the results directly.

## How was this patch tested?

Add unit test in ExchangeSuite.

With this patch(and `spark.sql.execution.sortBeforeRepartition` set to true), the following query returns 1000000:
```
import scala.sys.process._

import org.apache.spark.TaskContext

spark.conf.set("spark.sql.execution.sortBeforeRepartition", "true")

val res = spark.range(0, 1000 * 1000, 1).repartition(200).map { x =>
  x
}.repartition(200).map { x =>
  if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 2) {
    throw new Exception("pkill -f java".!!)
  }
  x
}
res.distinct().count()

res7: Long = 1000000
```

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

Closes #20393 from jiangxb1987/shuffle-repartition.
2018-01-26 15:01:03 -08:00
Wenchen Fan dd8e257d1c [SPARK-23218][SQL] simplify ColumnVector.getArray
## What changes were proposed in this pull request?

`ColumnVector` is very flexible about how to implement array type. As a result `ColumnVector` has 3 abstract methods for array type: `arrayData`, `getArrayOffset`, `getArrayLength`. For example, in `WritableColumnVector` we use the first child vector as the array data vector, and store offsets and lengths in 2 arrays in the parent vector. `ArrowColumnVector` has a different implementation.

This PR simplifies `ColumnVector` by using only one abstract method for array type: `getArray`.

## How was this patch tested?

existing tests.

rerun `ColumnarBatchBenchmark`, there is no performance regression.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20395 from cloud-fan/vector.
2018-01-26 09:17:05 -08:00
Kris Mok e57f394818 [SPARK-23032][SQL] Add a per-query codegenStageId to WholeStageCodegenExec
## What changes were proposed in this pull request?

**Proposal**

Add a per-query ID to the codegen stages as represented by `WholeStageCodegenExec` operators. This ID will be used in
-  the explain output of the physical plan, and in
- the generated class name.

Specifically, this ID will be stable within a query, counting up from 1 in depth-first post-order for all the `WholeStageCodegenExec` inserted into a plan.
The ID value 0 is reserved for "free-floating" `WholeStageCodegenExec` objects, which may have been created for one-off purposes, e.g. for fallback handling of codegen stages that failed to codegen the whole stage and wishes to codegen a subset of the children operators (as seen in `org.apache.spark.sql.execution.FileSourceScanExec#doExecute`).

Example: for the following query:
```scala
scala> spark.conf.set("spark.sql.autoBroadcastJoinThreshold", 1)

scala> val df1 = spark.range(10).select('id as 'x, 'id + 1 as 'y).orderBy('x).select('x + 1 as 'z, 'y)
df1: org.apache.spark.sql.DataFrame = [z: bigint, y: bigint]

scala> val df2 = spark.range(5)
df2: org.apache.spark.sql.Dataset[Long] = [id: bigint]

scala> val query = df1.join(df2, 'z === 'id)
query: org.apache.spark.sql.DataFrame = [z: bigint, y: bigint ... 1 more field]
```

The explain output before the change is:
```scala
scala> query.explain
== Physical Plan ==
*SortMergeJoin [z#9L], [id#13L], Inner
:- *Sort [z#9L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(z#9L, 200)
:     +- *Project [(x#3L + 1) AS z#9L, y#4L]
:        +- *Sort [x#3L ASC NULLS FIRST], true, 0
:           +- Exchange rangepartitioning(x#3L ASC NULLS FIRST, 200)
:              +- *Project [id#0L AS x#3L, (id#0L + 1) AS y#4L]
:                 +- *Range (0, 10, step=1, splits=8)
+- *Sort [id#13L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#13L, 200)
      +- *Range (0, 5, step=1, splits=8)
```
Note how codegen'd operators are annotated with a prefix `"*"`. See how the `SortMergeJoin` operator and its direct children `Sort` operators are adjacent and all annotated with the `"*"`, so it's hard to tell they're actually in separate codegen stages.

and after this change it'll be:
```scala
scala> query.explain
== Physical Plan ==
*(6) SortMergeJoin [z#9L], [id#13L], Inner
:- *(3) Sort [z#9L ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(z#9L, 200)
:     +- *(2) Project [(x#3L + 1) AS z#9L, y#4L]
:        +- *(2) Sort [x#3L ASC NULLS FIRST], true, 0
:           +- Exchange rangepartitioning(x#3L ASC NULLS FIRST, 200)
:              +- *(1) Project [id#0L AS x#3L, (id#0L + 1) AS y#4L]
:                 +- *(1) Range (0, 10, step=1, splits=8)
+- *(5) Sort [id#13L ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#13L, 200)
      +- *(4) Range (0, 5, step=1, splits=8)
```
Note that the annotated prefix becomes `"*(id) "`. See how the `SortMergeJoin` operator and its direct children `Sort` operators have different codegen stage IDs.

It'll also show up in the name of the generated class, as a suffix in the format of `GeneratedClass$GeneratedIterator$id`.

For example, note how `GeneratedClass$GeneratedIteratorForCodegenStage3` and `GeneratedClass$GeneratedIteratorForCodegenStage6` in the following stack trace corresponds to the IDs shown in the explain output above:
```
"Executor task launch worker for task 42412957" daemon prio=5 tid=0x58 nid=NA runnable
  java.lang.Thread.State: RUNNABLE
	  at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:109)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage3.sort_addToSorter$(generated.java:32)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage3.processNext(generated.java:41)
	  at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	  at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$9$$anon$1.hasNext(WholeStageCodegenExec.scala:494)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage6.findNextInnerJoinRows$(generated.java:42)
	  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage6.processNext(generated.java:101)
	  at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	  at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$2.hasNext(WholeStageCodegenExec.scala:513)
	  at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:253)
	  at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
	  at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:828)
	  at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:828)
	  at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	  at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	  at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	  at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	  at org.apache.spark.scheduler.Task.run(Task.scala:109)
	  at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
	  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	  at java.lang.Thread.run(Thread.java:748)
```

**Rationale**

Right now, the codegen from Spark SQL lacks the means to differentiate between a couple of things:

1. It's hard to tell which physical operators are in the same WholeStageCodegen stage. Note that this "stage" is a separate notion from Spark's RDD execution stages; this one is only to delineate codegen units.
There can be adjacent physical operators that are both codegen'd but are in separate codegen stages. Some of this is due to hacky implementation details, such as the case with `SortMergeJoin` and its `Sort` inputs -- they're hard coded to be split into separate stages although both are codegen'd.
When printing out the explain output of the physical plan, you'd only see the codegen'd physical operators annotated with a preceding star (`'*'`) but would have no way to figure out if they're in the same stage.

2. Performance/error diagnosis
The generated code has class/method names that are hard to differentiate between queries or even between codegen stages within the same query. If we use a Java-level profiler to collect profiles, or if we encounter a Java-level exception with a stack trace in it, it's really hard to tell which part of a query it's at.
By introducing a per-query codegen stage ID, we'd at least be able to know which codegen stage (and in turn, which group of physical operators) was a profile tick or an exception happened.

The reason why this proposal uses a per-query ID is because it's stable within a query, so that multiple runs of the same query will see the same resulting IDs. This both benefits understandability for users, and also it plays well with the codegen cache in Spark SQL which uses the generated source code as the key.

The downside to using per-query IDs as opposed to a per-session or globally incrementing ID is of course we can't tell apart different query runs with this ID alone. But for now I believe this is a good enough tradeoff.

## How was this patch tested?

Existing tests. This PR does not involve any runtime behavior changes other than some name changes.
The SQL query test suites that compares explain outputs have been updates to ignore the newly added `codegenStageId`.

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

Closes #20224 from rednaxelafx/wsc-codegenstageid.
2018-01-25 16:11:33 -08:00
Huaxin Gao 8480c0c576 [SPARK-23081][PYTHON] Add colRegex API to PySpark
## What changes were proposed in this pull request?

Add colRegex API to PySpark

## How was this patch tested?

add a test in sql/tests.py

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #20390 from huaxingao/spark-23081.
2018-01-26 07:50:48 +09:00
Liang-Chi Hsieh d20bbc2d87 [SPARK-21717][SQL] Decouple consume functions of physical operators in whole-stage codegen
## What changes were proposed in this pull request?

It has been observed in SPARK-21603 that whole-stage codegen suffers performance degradation, if the generated functions are too long to be optimized by JIT.

We basically produce a single function to incorporate generated codes from all physical operators in whole-stage. Thus, it is possibly to grow the size of generated function over a threshold that we can't have JIT optimization for it anymore.

This patch is trying to decouple the logic of consuming rows in physical operators to avoid a giant function processing rows.

## How was this patch tested?

Added tests.

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

Closes #18931 from viirya/SPARK-21717.
2018-01-25 19:49:58 +08:00
Henry Robinson de36f65d3a [SPARK-23148][SQL] Allow pathnames with special characters for CSV / JSON / text
…JSON / text

## What changes were proposed in this pull request?

Fix for JSON and CSV data sources when file names include characters
that would be changed by URL encoding.

## How was this patch tested?

New unit tests for JSON, CSV and text suites

Author: Henry Robinson <henry@cloudera.com>

Closes #20355 from henryr/spark-23148.
2018-01-24 21:19:09 +09:00
gatorsmile 4e7b49041a Revert "[SPARK-23195][SQL] Keep the Hint of Cached Data"
This reverts commit 44cc4daf3a.
2018-01-23 22:38:20 -08:00
Liang-Chi Hsieh a3911cf896 [SPARK-23177][SQL][PYSPARK] Extract zero-parameter UDFs from aggregate
## What changes were proposed in this pull request?

We extract Python UDFs in logical aggregate which depends on aggregate expression or grouping key in ExtractPythonUDFFromAggregate rule. But Python UDFs which don't depend on above expressions should also be extracted to avoid the issue reported in the JIRA.

A small code snippet to reproduce that issue looks like:
```python
import pyspark.sql.functions as f

df = spark.createDataFrame([(1,2), (3,4)])
f_udf = f.udf(lambda: str("const_str"))
df2 = df.distinct().withColumn("a", f_udf())
df2.show()
```

Error exception is raised as:
```
: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF0#50
        at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:91)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:90)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
        at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:90)
        at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$38.apply(HashAggregateExec.scala:514)
        at org.apache.spark.sql.execution.aggregate.HashAggregateExec$$anonfun$38.apply(HashAggregateExec.scala:513)
```

This exception raises because `HashAggregateExec` tries to bind the aliased Python UDF expression (e.g., `pythonUDF0#50 AS a#44`) to grouping key.

## How was this patch tested?

Added test.

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

Closes #20360 from viirya/SPARK-23177.
2018-01-24 11:43:48 +09:00
gatorsmile 44cc4daf3a [SPARK-23195][SQL] Keep the Hint of Cached Data
## What changes were proposed in this pull request?
The broadcast hint of the cached plan is lost if we cache the plan. This PR is to correct it.

```Scala
  val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value")
  val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value")
  broadcast(df2).cache()
  df2.collect()
  val df3 = df1.join(df2, Seq("key"), "inner")
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20368 from gatorsmile/cachedBroadcastHint.
2018-01-23 16:17:09 -08:00
gatorsmile 613c290336 [SPARK-23192][SQL] Keep the Hint after Using Cached Data
## What changes were proposed in this pull request?

The hint of the plan segment is lost, if the plan segment is replaced by the cached data.

```Scala
      val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value")
      val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value")
      df2.cache()
      val df3 = df1.join(broadcast(df2), Seq("key"), "inner")
```

This PR is to fix it.

## How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20365 from gatorsmile/fixBroadcastHintloss.
2018-01-23 14:56:28 -08:00
gatorsmile ee572ba8c1 [SPARK-20749][SQL][FOLLOW-UP] Override prettyName for bit_length and octet_length
## What changes were proposed in this pull request?
We need to override the prettyName for bit_length and octet_length for getting the expected auto-generated alias name.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20358 from gatorsmile/test2.3More.
2018-01-23 21:36:20 +09:00
Li Jin b2ce17b4c9 [SPARK-22274][PYTHON][SQL] User-defined aggregation functions with pandas udf (full shuffle)
## What changes were proposed in this pull request?

Add support for using pandas UDFs with groupby().agg().

This PR introduces a new type of pandas UDF - group aggregate pandas UDF. This type of UDF defines a transformation of multiple pandas Series -> a scalar value. Group aggregate pandas UDFs can be used with groupby().agg(). Note group aggregate pandas UDF doesn't support partial aggregation, i.e., a full shuffle is required.

This PR doesn't support group aggregate pandas UDFs that return ArrayType, StructType or MapType. Support for these types is left for future PR.

## How was this patch tested?

GroupbyAggPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19872 from icexelloss/SPARK-22274-groupby-agg.
2018-01-23 14:11:30 +09:00
Wenchen Fan 51eb750263 [SPARK-22389][SQL] data source v2 partitioning reporting interface
## What changes were proposed in this pull request?

a new interface which allows data source to report partitioning and avoid shuffle at Spark side.

The design is pretty like the internal distribution/partitioing framework. Spark defines a `Distribution` interfaces and several concrete implementations, and ask the data source to report a `Partitioning`, the `Partitioning` should tell Spark if it can satisfy a `Distribution` or not.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20201 from cloud-fan/partition-reporting.
2018-01-22 15:21:09 -08:00
Jacek Laskowski 76b8b840dd [MINOR] Typo fixes
## What changes were proposed in this pull request?

Typo fixes

## How was this patch tested?

Local build / Doc-only changes

Author: Jacek Laskowski <jacek@japila.pl>

Closes #20344 from jaceklaskowski/typo-fixes.
2018-01-22 13:55:14 -06:00
Wenchen Fan 5d680cae48 [SPARK-23090][SQL] polish ColumnVector
## What changes were proposed in this pull request?

Several improvements:
* provide a default implementation for the batch get methods
* rename `getChildColumn` to `getChild`, which is more concise
* remove `getStruct(int, int)`, it's only used to simplify the codegen, which is an internal thing, we should not add a public API for this purpose.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20277 from cloud-fan/column-vector.
2018-01-22 20:56:38 +08:00
gatorsmile 896e45af5f [MINOR][SQL][TEST] Test case cleanups for recent PRs
## What changes were proposed in this pull request?
Revert the unneeded test case changes we made in SPARK-23000

Also fixes the test suites that do not call `super.afterAll()` in the local `afterAll`. The `afterAll()` of `TestHiveSingleton` actually reset the environments.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20341 from gatorsmile/testRelated.
2018-01-22 04:32:59 -08:00
gatorsmile 78801881c4 [SPARK-23170][SQL] Dump the statistics of effective runs of analyzer and optimizer rules
## What changes were proposed in this pull request?

Dump the statistics of effective runs of analyzer and optimizer rules.

## How was this patch tested?

Do a manual run of TPCDSQuerySuite

```
=== Metrics of Analyzer/Optimizer Rules ===
Total number of runs: 175899
Total time: 25.486559948 seconds

Rule                                                                                               Effective Time / Total Time                     Effective Runs / Total Runs

org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              1603280450 / 2868461549                         761 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$CTESubstitution                                    2045860009 / 2056602674                         37 / 788
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggregateFunctions                          440719059 / 1693110949                          38 / 1982
org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries                               1429834919 / 1446016225                         39 / 285
org.apache.spark.sql.catalyst.optimizer.PruneFilters                                               33273083 / 1389586938                           3 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  821183615 / 1266668754                          616 / 1982
org.apache.spark.sql.catalyst.optimizer.ReorderJoin                                                775837028 / 866238225                           132 / 1592
org.apache.spark.sql.catalyst.analysis.DecimalPrecision                                            550683593 / 748854507                           211 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubquery                                    513075345 / 634370596                           49 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$FixNullability                                     33475731 / 606406532                            12 / 742
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ImplicitTypeCasts                              193144298 / 545403925                           86 / 1982
org.apache.spark.sql.catalyst.optimizer.BooleanSimplification                                      18651497 / 495725004                            7 / 1592
org.apache.spark.sql.catalyst.optimizer.PushPredicateThroughJoin                                   369257217 / 489934378                           709 / 1592
org.apache.spark.sql.catalyst.optimizer.RemoveRedundantAliases                                     3707000 / 468291609                             9 / 1592
org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints                                410155900 / 435254175                           192 / 285
org.apache.spark.sql.execution.datasources.FindDataSourceTable                                     348885539 / 371855866                           233 / 1982
org.apache.spark.sql.catalyst.optimizer.NullPropagation                                            11307645 / 307531225                            26 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions                                   120324545 / 304948785                           294 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$FunctionArgumentConversion                     92323199 / 286695007                            38 / 1982
org.apache.spark.sql.catalyst.optimizer.PushDownPredicate                                          230084193 / 265845972                           785 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$PromoteStrings                                 45938401 / 265144009                            40 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$InConversion                                   14888776 / 261499450                            1 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$CaseWhenCoercion                               113796384 / 244913861                           29 / 1982
org.apache.spark.sql.catalyst.optimizer.ConstantFolding                                            65008069 / 236548480                            126 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractGenerator                                   0 / 226338929                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.ResolveTimeZone                                             98134906 / 221323770                            417 / 1982
org.apache.spark.sql.catalyst.optimizer.ReorderAssociativeOperator                                 0 / 208421703                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.OptimizeIn                                                 8762534 / 199351958                             16 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$DateTimeOperations                             11980016 / 190779046                            27 / 1982
org.apache.spark.sql.catalyst.optimizer.SimplifyBinaryComparison                                   0 / 188887385                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyConditionals                                       0 / 186812106                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCaseConversionExpressions                          0 / 183885230                                   0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCasts                                              17128295 / 182901910                            69 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$Division                                       14579110 / 180309340                            8 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$BooleanEquality                                0 / 176740516                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$IfCoercion                                     0 / 170781986                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.LikeSimplification                                         771605 / 164136736                              1 / 1592
org.apache.spark.sql.catalyst.optimizer.RemoveDispensableExpressions                               0 / 155958962                                   0 / 1592
org.apache.spark.sql.catalyst.analysis.ResolveCreateNamedStruct                                    0 / 151222943                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveWindowOrder                                 7534632 / 146596355                             14 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$EltCoercion                                    0 / 144488654                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$ConcatCoercion                                 0 / 142403338                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveWindowFrame                                 12067635 / 141500665                            21 / 1982
org.apache.spark.sql.catalyst.analysis.TimeWindowing                                               0 / 140431958                                   0 / 1982
org.apache.spark.sql.catalyst.analysis.TypeCoercion$WindowFrameCoercion                            0 / 125471960                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin                                         14226972 / 124922019                            11 / 1592
org.apache.spark.sql.catalyst.analysis.TypeCoercion$StackCoercion                                  0 / 123613887                                   0 / 1982
org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery                            8491071 / 121179056                             7 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGroupingAnalytics                           55526073 / 120290529                            11 / 1982
org.apache.spark.sql.catalyst.optimizer.ConstantPropagation                                        0 / 113886790                                   0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveDeserializer                                52383759 / 107160222                            148 / 1982
org.apache.spark.sql.catalyst.analysis.CleanupAliases                                              52543524 / 102091518                            344 / 1086
org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject                                     40682895 / 94403652                             342 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$ExtractWindowExpressions                           38473816 / 89740578                             23 / 1982
org.apache.spark.sql.catalyst.optimizer.CollapseProject                                            46806090 / 83315506                             281 / 1877
org.apache.spark.sql.catalyst.optimizer.FoldablePropagation                                        0 / 78750087                                    0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases                                     13742765 / 77227258                             47 / 1982
org.apache.spark.sql.catalyst.optimizer.CombineFilters                                             53386729 / 76960344                             448 / 1592
org.apache.spark.sql.execution.datasources.DataSourceAnalysis                                      68034341 / 75724186                             24 / 742
org.apache.spark.sql.catalyst.analysis.Analyzer$LookupFunctions                                    0 / 71151084                                    0 / 750
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveMissingReferences                           12139848 / 67599140                             8 / 1982
org.apache.spark.sql.catalyst.optimizer.PullupCorrelatedPredicates                                 45017938 / 65968777                             23 / 285
org.apache.spark.sql.execution.datasources.v2.PushDownOperatorsToDataSource                        0 / 60937767                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CollapseRepartition                                        0 / 59897237                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.PushProjectionThroughUnion                                 8547262 / 53941370                              10 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$HandleNullInputsForUDF                             0 / 52735976                                    0 / 742
org.apache.spark.sql.catalyst.analysis.TypeCoercion$WidenSetOperationTypes                         9797713 / 52401665                              9 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$PullOutNondeterministic                            0 / 51741500                                    0 / 742
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations                                   28614911 / 51061186                             233 / 1990
org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions                               0 / 50621510                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CombineUnions                                              2777800 / 50262112                              17 / 1877
org.apache.spark.sql.catalyst.analysis.Analyzer$GlobalAggregates                                   1640641 / 49633909                              46 / 1982
org.apache.spark.sql.catalyst.optimizer.DecimalAggregates                                          20198374 / 48488419                             100 / 385
org.apache.spark.sql.catalyst.optimizer.LimitPushDown                                              0 / 45052523                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.CombineLimits                                              0 / 44719443                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.EliminateSorts                                             0 / 44216930                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.RewritePredicateSubquery                                   36235699 / 44165786                             148 / 285
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNewInstance                                 0 / 42750307                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast                                      0 / 41811748                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveOrdinalInOrderByAndGroupBy                  3819476 / 41776562                              4 / 1982
org.apache.spark.sql.catalyst.optimizer.ComputeCurrentTime                                         0 / 40527808                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CollapseWindow                                             0 / 36832538                                    0 / 1592
org.apache.spark.sql.catalyst.optimizer.EliminateSerialization                                     0 / 36120667                                    0 / 1592
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAggAliasInGroupBy                           0 / 32435826                                    0 / 1982
org.apache.spark.sql.execution.datasources.PreprocessTableCreation                                 0 / 32145218                                    0 / 742
org.apache.spark.sql.execution.datasources.ResolveSQLOnFile                                        0 / 30295614                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolvePivot                                       0 / 30111655                                    0 / 1982
org.apache.spark.sql.catalyst.expressions.codegen.package$ExpressionCanonicalizer$CleanExpressions 59930 / 28038201                                26 / 8280
org.apache.spark.sql.catalyst.analysis.ResolveInlineTables                                         0 / 27808108                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubqueryColumnAliases                       0 / 27066690                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate                                    0 / 26660210                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNaturalAndUsingJoin                         0 / 25255184                                    0 / 1982
org.apache.spark.sql.catalyst.analysis.ResolveTableValuedFunctions                                 0 / 24663088                                    0 / 1990
org.apache.spark.sql.catalyst.analysis.SubstituteUnresolvedOrdinals                                9709079 / 24450670                              4 / 788
org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveBroadcastHints                          0 / 23776535                                    0 / 750
org.apache.spark.sql.catalyst.optimizer.ReplaceExpressions                                         0 / 22697895                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.CheckCartesianProducts                                     0 / 22523798                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.ReplaceDistinctWithAggregate                               988593 / 21535410                               15 / 300
org.apache.spark.sql.catalyst.optimizer.EliminateMapObjects                                        0 / 20269996                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates                                  0 / 19388592                                    0 / 285
org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases                                    17675532 / 18971185                             215 / 285
org.apache.spark.sql.catalyst.optimizer.GetCurrentDatabase                                         0 / 18271152                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.PropagateEmptyRelation                                     2077097 / 17190855                              3 / 288
org.apache.spark.sql.catalyst.analysis.EliminateBarriers                                           0 / 16736359                                    0 / 1086
org.apache.spark.sql.execution.OptimizeMetadataOnlyQuery                                           0 / 16669341                                    0 / 285
org.apache.spark.sql.catalyst.analysis.UpdateOuterReferences                                       0 / 14470235                                    0 / 742
org.apache.spark.sql.catalyst.optimizer.ReplaceExceptWithAntiJoin                                  6715625 / 12190561                              1 / 300
org.apache.spark.sql.catalyst.optimizer.ReplaceIntersectWithSemiJoin                               3451793 / 11431432                              7 / 300
org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate                                0 / 10810568                                    0 / 285
org.apache.spark.sql.catalyst.optimizer.RemoveRepetitionFromGroupExpressions                       344198 / 10475276                               1 / 286
org.apache.spark.sql.catalyst.analysis.Analyzer$WindowsSubstitution                                0 / 10386630                                    0 / 788
org.apache.spark.sql.catalyst.analysis.EliminateUnions                                             0 / 10096526                                    0 / 788
org.apache.spark.sql.catalyst.analysis.AliasViewChild                                              0 / 9991706                                     0 / 742
org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation                                     0 / 9649334                                     0 / 288
org.apache.spark.sql.catalyst.analysis.ResolveHints$RemoveAllHints                                 0 / 8739109                                     0 / 750
org.apache.spark.sql.execution.datasources.PreprocessTableInsertion                                0 / 8420889                                     0 / 742
org.apache.spark.sql.catalyst.analysis.EliminateView                                               0 / 8319134                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.RemoveLiteralFromGroupExpressions                          0 / 7392627                                     0 / 286
org.apache.spark.sql.catalyst.optimizer.ReplaceExceptWithFilter                                    0 / 7170516                                     0 / 300
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateArrayOps                                     0 / 7109643                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateStructOps                                    0 / 6837590                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.SimplifyCreateMapOps                                       0 / 6617848                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.CombineConcats                                             0 / 5768406                                     0 / 1592
org.apache.spark.sql.catalyst.optimizer.ReplaceDeduplicateWithAggregate                            0 / 5349831                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.CombineTypedFilters                                        0 / 5186642                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.EliminateDistinct                                          0 / 2427686                                     0 / 285
org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder                                       0 / 2420436                                     0 / 285

```

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20342 from gatorsmile/reportExecution.
2018-01-22 04:31:24 -08:00
Dongjoon Hyun 8142a3b883 [MINOR][SQL] Fix wrong comments on org.apache.spark.sql.parquet.row.attributes
## What changes were proposed in this pull request?

This PR fixes the wrong comment on `org.apache.spark.sql.parquet.row.attributes`
which is useful for UDTs like Vector/Matrix. Please see [SPARK-22320](https://issues.apache.org/jira/browse/SPARK-22320) for the usage.

Originally, [SPARK-19411](bf493686eb (diff-ee26d4c4be21e92e92a02e9f16dbc285L314)) left this behind during removing optional column metadatas. In the same PR, the same comment was removed at line 310-311.

## How was this patch tested?

N/A (This is about comments).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20346 from dongjoon-hyun/minor_comment_parquet.
2018-01-22 15:18:57 +09:00
Marco Gaido 121dc96f08 [SPARK-23087][SQL] CheckCartesianProduct too restrictive when condition is false/null
## What changes were proposed in this pull request?

CheckCartesianProduct raises an AnalysisException also when the join condition is always false/null. In this case, we shouldn't raise it, since the result will not be a cartesian product.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20333 from mgaido91/SPARK-23087.
2018-01-20 22:39:49 -08:00
fjh100456 00d169156d [SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing
[SPARK-21786][SQL] The 'spark.sql.parquet.compression.codec' and 'spark.sql.orc.compression.codec' configuration doesn't take effect on hive table writing

What changes were proposed in this pull request?

Pass ‘spark.sql.parquet.compression.codec’ value to ‘parquet.compression’.
Pass ‘spark.sql.orc.compression.codec’ value to ‘orc.compress’.

How was this patch tested?

Add test.

Note:
This is the same issue mentioned in #19218 . That branch was deleted mistakenly, so make a new pr instead.

gatorsmile maropu dongjoon-hyun discipleforteen

Author: fjh100456 <fu.jinhua6@zte.com.cn>
Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Wenchen Fan <wenchen@databricks.com>
Author: gatorsmile <gatorsmile@gmail.com>
Author: Yinan Li <liyinan926@gmail.com>
Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Juliusz Sompolski <julek@databricks.com>
Author: Felix Cheung <felixcheung_m@hotmail.com>
Author: jerryshao <sshao@hortonworks.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Gera Shegalov <gera@apache.org>
Author: chetkhatri <ckhatrimanjal@gmail.com>
Author: Joseph K. Bradley <joseph@databricks.com>
Author: Bago Amirbekian <bago@databricks.com>
Author: Xianjin YE <advancedxy@gmail.com>
Author: Bruce Robbins <bersprockets@gmail.com>
Author: zuotingbing <zuo.tingbing9@zte.com.cn>
Author: Kent Yao <yaooqinn@hotmail.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Adrian Ionescu <adrian@databricks.com>

Closes #20087 from fjh100456/HiveTableWriting.
2018-01-20 14:49:49 -08:00
Sean Owen 396cdfbea4 [SPARK-23091][ML] Incorrect unit test for approxQuantile
## What changes were proposed in this pull request?

Narrow bound on approx quantile test to epsilon from 2*epsilon to match paper

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #20324 from srowen/SPARK-23091.
2018-01-19 22:46:34 -08:00
Wenchen Fan d8aaa771e2 [SPARK-23149][SQL] polish ColumnarBatch
## What changes were proposed in this pull request?

Several cleanups in `ColumnarBatch`
* remove `schema`. The `ColumnVector`s inside `ColumnarBatch` already have the data type information, we don't need this `schema`.
* remove `capacity`. `ColumnarBatch` is just a wrapper of `ColumnVector`s, not builders, it doesn't need a capacity property.
* remove `DEFAULT_BATCH_SIZE`. As a wrapper, `ColumnarBatch` can't decide the batch size, it should be decided by the reader, e.g. parquet reader, orc reader, cached table reader. The default batch size should also be defined by the reader.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20316 from cloud-fan/columnar-batch.
2018-01-19 08:58:21 -08:00
gatorsmile 6c39654efc [SPARK-23000][TEST] Keep Derby DB Location Unchanged After Session Cloning
## What changes were proposed in this pull request?
After session cloning in `TestHive`, the conf of the singleton SparkContext for derby DB location is changed to a new directory. The new directory is created in `HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false)`.

This PR is to keep the conf value of `ConfVars.METASTORECONNECTURLKEY.varname` unchanged during the session clone.

## How was this patch tested?
The issue can be reproduced by the command:
> build/sbt -Phive "hive/test-only org.apache.spark.sql.hive.HiveSessionStateSuite org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite"

Also added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20328 from gatorsmile/fixTestFailure.
2018-01-19 22:47:18 +08:00
Sameer Agarwal 9c4b99861c [BUILD][MINOR] Fix java style check issues
## What changes were proposed in this pull request?

This patch fixes a few recently introduced java style check errors in master and release branch.

As an aside, given that [java linting currently fails](https://github.com/apache/spark/pull/10763
) on machines with a clean maven cache, it'd be great to find another workaround to [re-enable the java style checks](3a07eff5af/dev/run-tests.py (L577)) as part of Spark PRB.

/cc zsxwing JoshRosen srowen for any suggestions

## How was this patch tested?

Manual Check

Author: Sameer Agarwal <sameerag@apache.org>

Closes #20323 from sameeragarwal/java.
2018-01-19 01:38:08 -08:00
Takuya UESHIN 568055da93 [SPARK-23054][SQL][PYSPARK][FOLLOWUP] Use sqlType casting when casting PythonUserDefinedType to String.
## What changes were proposed in this pull request?

This is a follow-up of #20246.

If a UDT in Python doesn't have its corresponding Scala UDT, cast to string will be the raw string of the internal value, e.g. `"org.apache.spark.sql.catalyst.expressions.UnsafeArrayDataxxxxxxxx"` if the internal type is `ArrayType`.

This pr fixes it by using its `sqlType` casting.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20306 from ueshin/issues/SPARK-23054/fup1.
2018-01-19 11:37:08 +08:00
Tathagata Das bf34d665b9 [SPARK-23144][SS] Added console sink for continuous processing
## What changes were proposed in this pull request?
Refactored ConsoleWriter into ConsoleMicrobatchWriter and ConsoleContinuousWriter.

## How was this patch tested?
new unit test

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

Closes #20311 from tdas/SPARK-23144.
2018-01-18 12:33:39 -08:00
Marco Gaido e28eb43114 [SPARK-22036][SQL] Decimal multiplication with high precision/scale often returns NULL
## What changes were proposed in this pull request?

When there is an operation between Decimals and the result is a number which is not representable exactly with the result's precision and scale, Spark is returning `NULL`. This was done to reflect Hive's behavior, but it is against SQL ANSI 2011, which states that "If the result cannot be represented exactly in the result type, then whether it is rounded or truncated is implementation-defined". Moreover, Hive now changed its behavior in order to respect the standard, thanks to HIVE-15331.

Therefore, the PR propose to:
 - update the rules to determine the result precision and scale according to the new Hive's ones introduces in HIVE-15331;
 - round the result of the operations, when it is not representable exactly with the result's precision and scale, instead of returning `NULL`
 - introduce a new config `spark.sql.decimalOperations.allowPrecisionLoss` which default to `true` (ie. the new behavior) in order to allow users to switch back to the previous one.

Hive behavior reflects SQLServer's one. The only difference is that the precision and scale are adjusted for all the arithmetic operations in Hive, while SQL Server is said to do so only for multiplications and divisions in the documentation. This PR follows Hive's behavior.

A more detailed explanation is available here: https://mail-archives.apache.org/mod_mbox/spark-dev/201712.mbox/%3CCAEorWNAJ4TxJR9NBcgSFMD_VxTg8qVxusjP%2BAJP-x%2BJV9zH-yA%40mail.gmail.com%3E.

## How was this patch tested?

modified and added UTs. Comparisons with results of Hive and SQLServer.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20023 from mgaido91/SPARK-22036.
2018-01-18 21:24:39 +08:00
Jose Torres 1c76a91e5f [SPARK-23052][SS] Migrate ConsoleSink to data source V2 api.
## What changes were proposed in this pull request?

Migrate ConsoleSink to data source V2 api.

Note that this includes a missing piece in DataStreamWriter required to specify a data source V2 writer.

Note also that I've removed the "Rerun batch" part of the sink, because as far as I can tell this would never have actually happened. A MicroBatchExecution object will only commit each batch once for its lifetime, and a new MicroBatchExecution object would have a new ConsoleSink object which doesn't know it's retrying a batch. So I think this represents an anti-feature rather than a weakness in the V2 API.

## How was this patch tested?

new unit test

Author: Jose Torres <jose@databricks.com>

Closes #20243 from jose-torres/console-sink.
2018-01-17 22:36:29 -08:00
Xiayun Sun 0219470206 [SPARK-21996][SQL] read files with space in name for streaming
## What changes were proposed in this pull request?

Structured streaming is now able to read files with space in file name (previously it would skip the file and output a warning)

## How was this patch tested?

Added new unit test.

Author: Xiayun Sun <xiayunsun@gmail.com>

Closes #19247 from xysun/SPARK-21996.
2018-01-17 16:42:38 -08:00
Tathagata Das bac0d661af [SPARK-23119][SS] Minor fixes to V2 streaming APIs
## What changes were proposed in this pull request?

- Added `InterfaceStability.Evolving` annotations
- Improved docs.

## How was this patch tested?
Existing tests.

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

Closes #20286 from tdas/SPARK-23119.
2018-01-17 16:40:02 -08:00
Li Jin 4e6f8fb150 [SPARK-23047][PYTHON][SQL] Change MapVector to NullableMapVector in ArrowColumnVector
## What changes were proposed in this pull request?
This PR changes usage of `MapVector` in Spark codebase to use `NullableMapVector`.

`MapVector` is an internal Arrow class that is not supposed to be used directly. We should use `NullableMapVector` instead.

## How was this patch tested?

Existing test.

Author: Li Jin <ice.xelloss@gmail.com>

Closes #20239 from icexelloss/arrow-map-vector.
2018-01-18 07:26:43 +09:00
Jose Torres e946c63dd5 [SPARK-23093][SS] Don't change run id when reconfiguring a continuous processing query.
## What changes were proposed in this pull request?

Keep the run ID static, using a different ID for the epoch coordinator to avoid cross-execution message contamination.

## How was this patch tested?

new and existing unit tests

Author: Jose Torres <jose@databricks.com>

Closes #20282 from jose-torres/fix-runid.
2018-01-17 13:58:44 -08:00
Jose Torres 86a8450318 [SPARK-23033][SS] Don't use task level retry for continuous processing
## What changes were proposed in this pull request?

Continuous processing tasks will fail on any attempt number greater than 0. ContinuousExecution will catch these failures and restart globally from the last recorded checkpoints.
## How was this patch tested?
unit test

Author: Jose Torres <jose@databricks.com>

Closes #20225 from jose-torres/no-retry.
2018-01-17 13:52:51 -08:00
Wang Gengliang 8598a982b4 [SPARK-23079][SQL] Fix query constraints propagation with aliases
## What changes were proposed in this pull request?

Previously, PR #19201 fix the problem of non-converging constraints.
After that PR #19149 improve the loop and constraints is inferred only once.
So the problem of non-converging constraints is gone.

However, the case below will fail.

```

spark.range(5).write.saveAsTable("t")
val t = spark.read.table("t")
val left = t.withColumn("xid", $"id" + lit(1)).as("x")
val right = t.withColumnRenamed("id", "xid").as("y")
val df = left.join(right, "xid").filter("id = 3").toDF()
checkAnswer(df, Row(4, 3))

```

Because `aliasMap` replace all the aliased child. See the test case in PR for details.

This PR is to fix this bug by removing useless code for preventing non-converging constraints.
It can be also fixed with #20270, but this is much simpler and clean up the code.

## How was this patch tested?

Unit test

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #20278 from gengliangwang/FixConstraintSimple.
2018-01-18 00:05:26 +08:00
Dongjoon Hyun 0f8a28617a [SPARK-21783][SQL] Turn on ORC filter push-down by default
## What changes were proposed in this pull request?

ORC filter push-down is disabled by default from the beginning, [SPARK-2883](aa31e431fc (diff-41ef65b9ef5b518f77e2a03559893f4dR149)
).

Now, Apache Spark starts to depend on Apache ORC 1.4.1. For Apache Spark 2.3, this PR turns on ORC filter push-down by default like Parquet ([SPARK-9207](https://issues.apache.org/jira/browse/SPARK-21783)) as a part of [SPARK-20901](https://issues.apache.org/jira/browse/SPARK-20901), "Feature parity for ORC with Parquet".

## How was this patch tested?

Pass the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20265 from dongjoon-hyun/SPARK-21783.
2018-01-17 21:53:36 +08:00
Henry Robinson 1f3d933e0b [SPARK-23062][SQL] Improve EXCEPT documentation
## What changes were proposed in this pull request?

Make the default behavior of EXCEPT (i.e. EXCEPT DISTINCT) more
explicit in the documentation, and call out the change in behavior
from 1.x.

Author: Henry Robinson <henry@cloudera.com>

Closes #20254 from henryr/spark-23062.
2018-01-17 16:01:41 +08:00
Dongjoon Hyun a0aedb0ded [SPARK-23072][SQL][TEST] Add a Unicode schema test for file-based data sources
## What changes were proposed in this pull request?

After [SPARK-20682](https://github.com/apache/spark/pull/19651), Apache Spark 2.3 is able to read ORC files with Unicode schema. Previously, it raises `org.apache.spark.sql.catalyst.parser.ParseException`.

This PR adds a Unicode schema test for CSV/JSON/ORC/Parquet file-based data sources. Note that TEXT data source only has [a single column with a fixed name 'value'](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala#L71).

## How was this patch tested?

Pass the newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20266 from dongjoon-hyun/SPARK-23072.
2018-01-17 14:32:18 +08:00
Jose Torres 1667057851 [SPARK-22908][SS] Roll forward continuous processing Kafka support with fix to continuous Kafka data reader
## What changes were proposed in this pull request?

The Kafka reader is now interruptible and can close itself.
## How was this patch tested?

I locally ran one of the ContinuousKafkaSourceSuite tests in a tight loop. Before the fix, my machine ran out of open file descriptors a few iterations in; now it works fine.

Author: Jose Torres <jose@databricks.com>

Closes #20253 from jose-torres/fix-data-reader.
2018-01-16 18:11:27 -08:00
Gabor Somogyi a9b845ebb5 [SPARK-22361][SQL][TEST] Add unit test for Window Frames
## What changes were proposed in this pull request?

There are already quite a few integration tests using window frames, but the unit tests coverage is not ideal.

In this PR the already existing tests are reorganized, extended and where gaps found additional cases added.

## How was this patch tested?

Automated: Pass the Jenkins.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20019 from gaborgsomogyi/SPARK-22361.
2018-01-17 10:03:25 +08:00
Dilip Biswal 0c2ba427bc [SPARK-23095][SQL] Decorrelation of scalar subquery fails with java.util.NoSuchElementException
## What changes were proposed in this pull request?
The following SQL involving scalar correlated query returns a map exception.
``` SQL
SELECT t1a
FROM   t1
WHERE  t1a = (SELECT   count(*)
              FROM     t2
              WHERE    t2c = t1c
              HAVING   count(*) >= 1)
```
``` SQL
key not found: ExprId(278,786682bb-41f9-4bd5-a397-928272cc8e4e)
java.util.NoSuchElementException: key not found: ExprId(278,786682bb-41f9-4bd5-a397-928272cc8e4e)
        at scala.collection.MapLike$class.default(MapLike.scala:228)
        at scala.collection.AbstractMap.default(Map.scala:59)
        at scala.collection.MapLike$class.apply(MapLike.scala:141)
        at scala.collection.AbstractMap.apply(Map.scala:59)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$.org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$evalSubqueryOnZeroTups(subquery.scala:378)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$$anonfun$org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$constructLeftJoins$1.apply(subquery.scala:430)
        at org.apache.spark.sql.catalyst.optimizer.RewriteCorrelatedScalarSubquery$$anonfun$org$apache$spark$sql$catalyst$optimizer$RewriteCorrelatedScalarSubquery$$constructLeftJoins$1.apply(subquery.scala:426)
```

In this case, after evaluating the HAVING clause "count(*) > 1" statically
against the binding of aggregtation result on empty input, we determine
that this query will not have a the count bug. We should simply return
the evalSubqueryOnZeroTups with empty value.
(Please fill in changes proposed in this fix)

## How was this patch tested?
A new test was added in the Subquery bucket.

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

Closes #20283 from dilipbiswal/scalar-count-defect.
2018-01-17 09:57:30 +08:00
Gabor Somogyi 12db365b4f [SPARK-16139][TEST] Add logging functionality for leaked threads in tests
## What changes were proposed in this pull request?

Lots of our tests don't properly shutdown everything they create, and end up leaking lots of threads. For example, `TaskSetManagerSuite` doesn't stop the extra `TaskScheduler` and `DAGScheduler` it creates. There are a couple more instances, eg. in `DAGSchedulerSuite`.

This PR adds the possibility to print out the not properly stopped thread list after a test suite executed. The format is the following:

```
===== FINISHED o.a.s.scheduler.DAGSchedulerSuite: 'task end event should have updated accumulators (SPARK-20342)' =====

...

===== Global thread whitelist loaded with name /thread_whitelist from classpath: rpc-client.*, rpc-server.*, shuffle-client.*, shuffle-server.*' =====

ScalaTest-run:

===== THREADS NOT STOPPED PROPERLY =====

ScalaTest-run: dag-scheduler-event-loop
ScalaTest-run: globalEventExecutor-2-5
ScalaTest-run:

===== END OF THREAD DUMP =====

ScalaTest-run:

===== EITHER PUT THREAD NAME INTO THE WHITELIST FILE OR SHUT IT DOWN PROPERLY =====
```

With the help of this leaking threads has been identified in TaskSetManagerSuite. My intention is to hunt down and fix such bugs in later PRs.

## How was this patch tested?

Manual: TaskSetManagerSuite test executed and found out where are the leaking threads.
Automated: Pass the Jenkins.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #19893 from gaborgsomogyi/SPARK-16139.
2018-01-16 11:41:08 -08:00
Wenchen Fan 75db14864d [SPARK-22392][SQL] data source v2 columnar batch reader
## What changes were proposed in this pull request?

a new Data Source V2 interface to allow the data source to return `ColumnarBatch` during the scan.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20153 from cloud-fan/columnar-reader.
2018-01-16 22:41:30 +08:00
Yuanjian Li 07ae39d0ec [SPARK-22956][SS] Bug fix for 2 streams union failover scenario
## What changes were proposed in this pull request?

This problem reported by yanlin-Lynn ivoson and LiangchangZ. Thanks!

When we union 2 streams from kafka or other sources, while one of them have no continues data coming and in the same time task restart, this will cause an `IllegalStateException`. This mainly cause because the code in [MicroBatchExecution](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala#L190) , while one stream has no continues data, its comittedOffset same with availableOffset during `populateStartOffsets`, and `currentPartitionOffsets` not properly handled in KafkaSource. Also, maybe we should also consider this scenario in other Source.

## How was this patch tested?

Add a UT in KafkaSourceSuite.scala

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #20150 from xuanyuanking/SPARK-22956.
2018-01-15 22:01:14 -08:00
Marco Gaido 8ab2d7ea99 [SPARK-23080][SQL] Improve error message for built-in functions
## What changes were proposed in this pull request?

When a user puts the wrong number of parameters in a function, an AnalysisException is thrown. If the function is a UDF, he user is told how many parameters the function expected and how many he/she put. If the function, instead, is a built-in one, no information about the number of parameters expected and the actual one is provided. This can help in some cases, to debug the errors (eg. bad quotes escaping may lead to a different number of parameters than expected, etc. etc.)

The PR adds the information about the number of parameters passed and the expected one, analogously to what happens for UDF.

## How was this patch tested?

modified existing UT + manual test

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20271 from mgaido91/SPARK-23080.
2018-01-16 11:47:42 +09:00
xubo245 6c81fe227a [SPARK-23035][SQL] Fix improper information of TempTableAlreadyExistsException
## What changes were proposed in this pull request?

Problem: it throw TempTableAlreadyExistsException and output "Temporary table '$table' already exists" when we create temp view by using org.apache.spark.sql.catalyst.catalog.GlobalTempViewManager#create, it's improper.

So fix improper information about TempTableAlreadyExistsException when create temp view:

change "Temporary table"  to  "Temporary view"

## How was this patch tested?

test("rename temporary view - destination table already exists, with: CREATE TEMPORARY view")

test("rename temporary view - destination table with database name,with:CREATE TEMPORARY view")

Author: xubo245 <601450868@qq.com>

Closes #20227 from xubo245/fixDeprecated.
2018-01-15 23:13:15 +08:00
Yuming Wang a38c887ac0 [SPARK-19550][BUILD][FOLLOW-UP] Remove MaxPermSize for sql module
## What changes were proposed in this pull request?

Remove `MaxPermSize` for `sql` module

## How was this patch tested?

Manually tested.

Author: Yuming Wang <yumwang@ebay.com>

Closes #20268 from wangyum/SPARK-19550-MaxPermSize.
2018-01-15 07:49:34 -06:00
Takeshi Yamamuro b59808385c [SPARK-23023][SQL] Cast field data to strings in showString
## What changes were proposed in this pull request?
The current `Datset.showString` prints rows thru `RowEncoder` deserializers like;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------------------------------------------+
|a                                                           |
+------------------------------------------------------------+
|[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
+------------------------------------------------------------+
```
This result is incorrect because the correct one is;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------+
|a                       |
+------------------------+
|[[1, 2], [3], [4, 5, 6]]|
+------------------------+
```
So, this pr fixed code in `showString` to cast field data to strings before printing.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20214 from maropu/SPARK-23023.
2018-01-15 16:26:52 +08:00