Commit graph

4838 commits

Author SHA1 Message Date
maryannxue 75725057b3 [SPARK-24790][SQL] Allow complex aggregate expressions in Pivot
## What changes were proposed in this pull request?

Relax the check to allow complex aggregate expressions, like `ceil(sum(col1))` or `sum(col1) + 1`, which roughly means any aggregate expression that could appear in an Aggregate plan except pandas UDF (due to the fact that it is not supported in pivot yet).

## How was this patch tested?

Added 2 tests in pivot.sql

Author: maryannxue <maryannxue@apache.org>

Closes #21753 from maryannxue/pivot-relax-syntax.
2018-07-12 16:54:03 -07:00
Marco Gaido 11384893b6 [SPARK-24208][SQL][FOLLOWUP] Move test cases to proper locations
## What changes were proposed in this pull request?

The PR is a followup to move the test cases introduced by the original PR in their proper location.

## How was this patch tested?

moved UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21751 from mgaido91/SPARK-24208_followup.
2018-07-12 15:13:26 -07:00
Dongjoon Hyun 07704c971c [SPARK-23007][SQL][TEST] Add read schema suite for file-based data sources
## What changes were proposed in this pull request?

The reader schema is said to be evolved (or projected) when it changed after the data is written. The followings are already supported in file-based data sources. Note that partition columns are not maintained in files. In this PR, `column` means `non-partition column`.

   1. Add a column
   2. Hide a column
   3. Change a column position
   4. Change a column type (upcast)

This issue aims to guarantee users a backward-compatible read-schema test coverage on file-based data sources and to prevent future regressions by *adding read schema tests explicitly*.

Here, we consider safe changes without data loss. For example, data type change should be from small types to larger types like `int`-to-`long`, not vice versa.

As of today, in the master branch, file-based data sources have the following coverage.

File Format | Coverage  | Note
----------- | ---------- | ------------------------------------------------
TEXT          | N/A            | Schema consists of a single string column.
CSV            | 1, 2, 4        |
JSON          | 1, 2, 3, 4    |
ORC            | 1, 2, 3, 4    | Native vectorized ORC reader has the widest coverage among ORC formats.
PARQUET   | 1, 2, 3        |

## How was this patch tested?

Pass the Jenkins with newly added test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20208 from dongjoon-hyun/SPARK-SCHEMA-EVOLUTION.
2018-07-12 14:08:49 -07:00
Gengliang Wang e6c6f90a55 [SPARK-24691][SQL] Dispatch the type support check in FileFormat implementation
## What changes were proposed in this pull request?

With https://github.com/apache/spark/pull/21389,  data source schema is validated on driver side before launching read/write tasks.
However,

1. Putting all the validations together in `DataSourceUtils` is tricky and hard to maintain. On second thought after review, I find that the `OrcFileFormat` in hive package is not matched, so that its validation wrong.
2.  `DataSourceUtils.verifyWriteSchema` and `DataSourceUtils.verifyReadSchema` is not supposed to be called in every file format. We can move them to some upper entry.

So, I propose we can add a new method `validateDataType` in FileFormat. File format implementation can override the method to specify its supported/non-supported data types.
Although we should focus on data source V2 API, `FileFormat` should remain workable for some time. Adding this new method should be helpful.

## How was this patch tested?

Unit test

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21667 from gengliangwang/refactorSchemaValidate.
2018-07-13 00:26:49 +08:00
Kazuaki Ishizaki 301bff7063 [SPARK-23914][SQL] Add array_union function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_union`. The behavior of the function is based on Presto's one.

This function returns returns an array of the elements in the union of array1 and array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs

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

Closes #21061 from kiszk/SPARK-23914.
2018-07-12 17:42:29 +09:00
Maxim Gekk 3ab48f985c [SPARK-24761][SQL] Adding of isModifiable() to RuntimeConfig
## What changes were proposed in this pull request?

In the PR, I propose to extend `RuntimeConfig` by new method `isModifiable()` which returns `true` if a config parameter can be modified at runtime (for current session state). For static SQL and core parameters, the method returns `false`.

## How was this patch tested?

Added new test to `RuntimeConfigSuite` for checking Spark core and SQL parameters.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21730 from MaxGekk/is-modifiable.
2018-07-11 17:38:43 -07:00
Marco Gaido e008ad1752 [SPARK-24782][SQL] Simplify conf retrieval in SQL expressions
## What changes were proposed in this pull request?

The PR simplifies the retrieval of config in `size`, as we can access them from tasks too thanks to SPARK-24250.

## How was this patch tested?

existing UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21736 from mgaido91/SPARK-24605_followup.
2018-07-11 17:30:43 -07:00
Tathagata Das ff7f6ef75c [SPARK-24697][SS] Fix the reported start offsets in streaming query progress
## What changes were proposed in this pull request?

In ProgressReporter for streams, we use the `committedOffsets` as the startOffset and `availableOffsets` as the end offset when reporting the status of a trigger in `finishTrigger`. This is a bad pattern that has existed since the beginning of ProgressReporter and it is bad because its super hard to reason about when `availableOffsets` and `committedOffsets` are updated, and when they are recorded. Case in point, this bug silently existed in ContinuousExecution, since before MicroBatchExecution was refactored.

The correct fix it to record the offsets explicitly. This PR adds a simple method which is explicitly called from MicroBatch/ContinuousExecition before updating the `committedOffsets`.

## How was this patch tested?
Added new tests

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

Closes #21744 from tdas/SPARK-24697.
2018-07-11 12:44:42 -07:00
Marco Gaido ebf4bfb966 [SPARK-24208][SQL] Fix attribute deduplication for FlatMapGroupsInPandas
## What changes were proposed in this pull request?

A self-join on a dataset which contains a `FlatMapGroupsInPandas` fails because of duplicate attributes. This happens because we are not dealing with this specific case in our `dedupAttr` rules.

The PR fix the issue by adding the management of the specific case

## How was this patch tested?

added UT + manual tests

Author: Marco Gaido <marcogaido91@gmail.com>
Author: Marco Gaido <mgaido@hortonworks.com>

Closes #21737 from mgaido91/SPARK-24208.
2018-07-11 09:29:19 -07:00
Marco Gaido 592cc84583 [SPARK-24562][TESTS] Support different configs for same test in SQLQueryTestSuite
## What changes were proposed in this pull request?

The PR proposes to add support for running the same SQL test input files against different configs leading to the same result.

## How was this patch tested?

Involved UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21568 from mgaido91/SPARK-24562.
2018-07-11 23:43:06 +08:00
Marek Novotny 74a8d6308b [SPARK-24165][SQL] Fixing conditional expressions to handle nullability of nested types
## What changes were proposed in this pull request?
This PR is proposing a fix for the output data type of ```If``` and ```CaseWhen``` expression. Upon till now, the implementation of exprassions has ignored nullability of nested types from different execution branches and returned the type of the first branch.

This could lead to an unwanted ```NullPointerException``` from other expressions depending on a ```If```/```CaseWhen``` expression.

Example:
```
val rows = new util.ArrayList[Row]()
rows.add(Row(true, ("a", 1)))
rows.add(Row(false, (null, 2)))
val schema = StructType(Seq(
  StructField("cond", BooleanType, false),
  StructField("s", StructType(Seq(
    StructField("val1", StringType, true),
    StructField("val2", IntegerType, false)
  )), false)
))

val df = spark.createDataFrame(rows, schema)

df
  .select(when('cond, struct(lit("x").as("val1"), lit(10).as("val2"))).otherwise('s) as "res")
  .select('res.getField("val1"))
  .show()
```
Exception:
```
Exception in thread "main" java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.codegen.UnsafeWriter.write(UnsafeWriter.java:109)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.execution.LocalTableScanExec$$anonfun$unsafeRows$1.apply(LocalTableScanExec.scala:44)
	at org.apache.spark.sql.execution.LocalTableScanExec$$anonfun$unsafeRows$1.apply(LocalTableScanExec.scala:44)
...
```
Output schema:
```
root
 |-- res.val1: string (nullable = false)
```

## How was this patch tested?
New test cases added into
- DataFrameSuite.scala
- conditionalExpressions.scala

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21687 from mn-mikke/SPARK-24165.
2018-07-11 12:21:03 +08:00
Tathagata Das 6078b891da [SPARK-24730][SS] Add policy to choose max as global watermark when streaming query has multiple watermarks
## What changes were proposed in this pull request?

Currently, when a streaming query has multiple watermark, the policy is to choose the min of them as the global watermark. This is safe to do as the global watermark moves with the slowest stream, and is therefore is safe as it does not unexpectedly drop some data as late, etc. While this is indeed the safe thing to do, in some cases, you may want the watermark to advance with the fastest stream, that is, take the max of multiple watermarks. This PR is to add that configuration. It makes the following changes.

- Adds a configuration to specify max as the policy.
- Saves the configuration in OffsetSeqMetadata because changing it in the middle can lead to unpredictable results.
   - For old checkpoints without the configuration, it assumes the default policy as min (irrespective of the policy set at the session where the query is being restarted). This is to ensure that existing queries are affected in any way.

TODO
- [ ] Add a test for recovery from existing checkpoints.

## How was this patch tested?
New unit test

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

Closes #21701 from tdas/SPARK-24730.
2018-07-10 18:03:40 -07:00
Mukul Murthy 32cb50835e [SPARK-24662][SQL][SS] Support limit in structured streaming
## What changes were proposed in this pull request?

Support the LIMIT operator in structured streaming.

For streams in append or complete output mode, a stream with a LIMIT operator will return no more than the specified number of rows. LIMIT is still unsupported for the update output mode.

This change reverts e4fee395ec as part of it because it is a better and more complete implementation.

## How was this patch tested?

New and existing unit tests.

Author: Mukul Murthy <mukul.murthy@gmail.com>

Closes #21662 from mukulmurthy/SPARK-24662.
2018-07-10 11:08:04 -07:00
Wenchen Fan e0559f2380 [SPARK-21743][SQL][FOLLOWUP] free aggregate map when task ends
## What changes were proposed in this pull request?

This is the first follow-up of https://github.com/apache/spark/pull/21573 , which was only merged to 2.3.

This PR fixes the memory leak in another way: free the `UnsafeExternalMap` when the task ends. All the data buffers in Spark SQL are using `UnsafeExternalMap` and `UnsafeExternalSorter` under the hood, e.g. sort, aggregate, window, SMJ, etc. `UnsafeExternalSorter` registers a task completion listener to free the resource, we should apply the same thing to `UnsafeExternalMap`.

TODO in the next PR:
do not consume all the inputs when having limit in whole stage codegen.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21738 from cloud-fan/limit.
2018-07-10 23:07:10 +08:00
Yuming Wang a289009567 [SPARK-24706][SQL] ByteType and ShortType support pushdown to parquet
## What changes were proposed in this pull request?

`ByteType` and `ShortType` support pushdown to parquet data source.
[Benchmark result](https://issues.apache.org/jira/browse/SPARK-24706?focusedCommentId=16528878&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16528878).

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21682 from wangyum/SPARK-24706.
2018-07-10 15:58:14 +08:00
Xiao Li eb6e988039 [SPARK-24759][SQL] No reordering keys for broadcast hash join
## What changes were proposed in this pull request?

As the implementation of the broadcast hash join is independent of the input hash partitioning, reordering keys is not necessary. Thus, we solve this issue by simply removing the broadcast hash join from the reordering rule in EnsureRequirements.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21728 from gatorsmile/cleanER.
2018-07-09 14:53:14 -07:00
Xiao Li aec966b05e Revert "[SPARK-24268][SQL] Use datatype.simpleString in error messages"
This reverts commit 1bd3d61f41.
2018-07-09 14:24:23 -07:00
Marco Gaido 1bd3d61f41 [SPARK-24268][SQL] Use datatype.simpleString in error messages
## What changes were proposed in this pull request?

SPARK-22893 tried to unify error messages about dataTypes. Unfortunately, still many places were missing the `simpleString` method in other to have the same representation everywhere.

The PR unified the messages using alway the simpleString representation of the dataTypes in the messages.

## How was this patch tested?

existing/modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21321 from mgaido91/SPARK-24268.
2018-07-09 22:59:05 +08:00
Bruce Robbins 034913b62b [SPARK-23936][SQL] Implement map_concat
## What changes were proposed in this pull request?

Implement map_concat high order function.

This implementation does not pick a winner when the specified maps have overlapping keys. Therefore, this implementation preserves existing duplicate keys in the maps and potentially introduces new duplicates (After discussion with ueshin, we settled on option 1 from [here](https://issues.apache.org/jira/browse/SPARK-23936?focusedCommentId=16464245&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16464245)).

## How was this patch tested?

New tests
Manual tests
Run all sbt SQL tests
Run all pyspark sql tests

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21073 from bersprockets/SPARK-23936.
2018-07-09 21:21:38 +09:00
Maxim Gekk 79c6689429 [SPARK-24757][SQL] Improving the error message for broadcast timeouts
## What changes were proposed in this pull request?

In the PR, I propose to provide a tip to user how to resolve the issue of timeout expiration for broadcast joins. In particular, they can increase the timeout via **spark.sql.broadcastTimeout** or disable the broadcast at all by setting **spark.sql.autoBroadcastJoinThreshold** to `-1`.

## How was this patch tested?

It tested manually from `spark-shell`:
```
scala> spark.conf.set("spark.sql.broadcastTimeout", 1)
scala> val df = spark.range(100).join(spark.range(15).as[Long].map { x =>
               Thread.sleep(5000)
               x
            }).where("id = value")
scala> df.count()
```
```
org.apache.spark.SparkException: Could not execute broadcast in 1 secs. You can increase the timeout for broadcasts via spark.sql.broadcastTimeout or disable broadcast join by setting spark.sql.autoBroadcastJoinThreshold to -1
  at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecuteBroadcast(BroadcastExchangeExec.scala:150)
```

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21727 from MaxGekk/broadcast-timeout-error.
2018-07-07 22:16:48 +02:00
Liang-Chi Hsieh 4de0425df8 [SPARK-24569][SQL] Aggregator with output type Option should produce consistent schema
## What changes were proposed in this pull request?

SQL `Aggregator` with output type `Option[Boolean]` creates column of type `StructType`. It's not in consistency with a Dataset of similar java class.

This changes the way `definedByConstructorParams` checks given type. For `Option[_]`, it goes to check its type argument.

## How was this patch tested?

Added test.

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

Closes #21611 from viirya/SPARK-24569.
2018-07-07 10:54:14 +08:00
Yuming Wang bf67f70c48 [SPARK-24692][TESTS] Improvement FilterPushdownBenchmark
## What changes were proposed in this pull request?
Refer to the [`WideSchemaBenchmark`](https://github.com/apache/spark/blob/v2.3.1/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala) update `FilterPushdownBenchmark`:
1. Write the result to `benchmarks/FilterPushdownBenchmark-results.txt` for easy maintenance.
2. Add more benchmark case: `StringStartsWith`, `Decimal`, `InSet -> InFilters` and `tinyint`.

## How was this patch tested?

manual tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21677 from wangyum/SPARK-24692.
2018-07-06 11:13:57 +08:00
Gengliang Wang 33952cfa81 [SPARK-24675][SQL] Rename table: validate existence of new location
## What changes were proposed in this pull request?
If table is renamed to a existing new location, data won't show up.
```
scala>  Seq("hello").toDF("a").write.format("parquet").saveAsTable("t")

scala> sql("select * from t").show()
+-----+
|    a|
+-----+
|hello|
+-----+

scala> sql("alter table t rename to test")
res2: org.apache.spark.sql.DataFrame = []

scala> sql("select * from test").show()
+---+
|  a|
+---+
+---+
```
The file layout is like
```
$ tree test
test
├── gabage
└── t
    ├── _SUCCESS
    └── part-00000-856b0f10-08f1-42d6-9eb3-7719261f3d5e-c000.snappy.parquet
```

In Hive, if the new location exists, the renaming will fail even the location is empty.

We should have the same validation in Catalog, in case of unexpected bugs.

## How was this patch tested?

New unit test.

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21655 from gengliangwang/validate_rename_table.
2018-07-05 09:25:19 -07:00
Antonio Murgia 4be9f0c028 [SPARK-24673][SQL] scala sql function from_utc_timestamp second argument could be Column instead of String
## What changes were proposed in this pull request?

Add an overloaded version to `from_utc_timestamp` and `to_utc_timestamp` having second argument as a `Column` instead of `String`.

## How was this patch tested?

Unit testing, especially adding two tests to org.apache.spark.sql.DateFunctionsSuite.scala

Author: Antonio Murgia <antonio.murgia@agilelab.it>
Author: Antonio Murgia <antonio.murgia2@studio.unibo.it>

Closes #21693 from tmnd1991/feature/SPARK-24673.
2018-07-05 16:10:34 +08:00
Xiao Li 489a5294d1 [SPARK-17213][SPARK-17213][FOLLOW-UP] Improve the test of
## What changes were proposed in this pull request?
This is a minor improvement for the test of SPARK-17213

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21716 from gatorsmile/testMaster23.
2018-07-05 09:56:48 +08:00
Yuming Wang 021145f364 [SPARK-24716][SQL] Refactor ParquetFilters
## What changes were proposed in this pull request?

Replace DataFrame schema to Parquet file schema when create `ParquetFilters`.
Thus we can easily implement `Decimal` and `Timestamp` push down. some thing like this:
```scala
// DecimalType: 32BitDecimalType
case ParquetSchemaType(DECIMAL, INT32, decimal)
  if pushDownDecimal =>
  (n: String, v: Any) => FilterApi.eq(
    intColumn(n),
    Option(v).map(_.asInstanceOf[JBigDecimal].unscaledValue().intValue()
      .asInstanceOf[Integer]).orNull)
// DecimalType: 64BitDecimalType
case ParquetSchemaType(DECIMAL, INT64, decimal)
  if pushDownDecimal =>
  (n: String, v: Any) => FilterApi.eq(
    longColumn(n),
    Option(v).map(_.asInstanceOf[JBigDecimal].unscaledValue().longValue()
      .asInstanceOf[java.lang.Long]).orNull)
// DecimalType: LegacyParquetFormat 32BitDecimalType & 64BitDecimalType
case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, decimal)
  if pushDownDecimal && decimal.getPrecision <= Decimal.MAX_LONG_DIGITS =>
  (n: String, v: Any) => FilterApi.eq(
    binaryColumn(n),
    Option(v).map(d => decimalToBinaryUsingUnscaledLong(decimal.getPrecision,
      d.asInstanceOf[JBigDecimal])).orNull)
// DecimalType: ByteArrayDecimalType
case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, decimal)
  if pushDownDecimal && decimal.getPrecision > Decimal.MAX_LONG_DIGITS =>
  (n: String, v: Any) => FilterApi.eq(
    binaryColumn(n),
    Option(v).map(d => decimalToBinaryUsingUnscaledBytes(decimal.getPrecision,
      d.asInstanceOf[JBigDecimal])).orNull)
```

```scala
// INT96 doesn't support pushdown
case ParquetSchemaType(TIMESTAMP_MICROS, INT64, null) =>
  (n: String, v: Any) => FilterApi.eq(
    longColumn(n),
    Option(v).map(t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp])
      .asInstanceOf[java.lang.Long]).orNull)
case ParquetSchemaType(TIMESTAMP_MILLIS, INT64, null) =>
  (n: String, v: Any) => FilterApi.eq(
    longColumn(n),
    Option(v).map(_.asInstanceOf[Timestamp].getTime.asInstanceOf[java.lang.Long]).orNull)
```

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21696 from wangyum/SPARK-24716.
2018-07-04 20:15:40 +08:00
Takeshi Yamamuro b2deef64f6 [SPARK-24727][SQL] Add a static config to control cache size for generated classes
## What changes were proposed in this pull request?
Since SPARK-24250 has been resolved, executors correctly references user-defined configurations. So, this pr added a static config to control cache size for generated classes in `CodeGenerator`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21705 from maropu/SPARK-24727.
2018-07-04 20:04:18 +08:00
Maxim Gekk 776f299fc8 [SPARK-24709][SQL] schema_of_json() - schema inference from an example
## What changes were proposed in this pull request?

In the PR, I propose to add new function - *schema_of_json()* which infers schema of JSON string literal. The result of the function is a string containing a schema in DDL format.

One of the use cases is using of *schema_of_json()* in the combination with *from_json()*. Currently, _from_json()_ requires a schema as a mandatory argument. The *schema_of_json()* function will allow to point out an JSON string as an example which has the same schema as the first argument of _from_json()_. For instance:

```sql
select from_json(json_column, schema_of_json('{"c1": [0], "c2": [{"c3":0}]}'))
from json_table;
```

## How was this patch tested?

Added new test to `JsonFunctionsSuite`, `JsonExpressionsSuite` and SQL tests to `json-functions.sql`

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21686 from MaxGekk/infer_schema_json.
2018-07-04 09:38:18 +08:00
DB Tsai 5585c5765f
[SPARK-24420][BUILD] Upgrade ASM to 6.1 to support JDK9+
## What changes were proposed in this pull request?

Upgrade ASM to 6.1 to support JDK9+

## How was this patch tested?

Existing tests.

Author: DB Tsai <d_tsai@apple.com>

Closes #21459 from dbtsai/asm.
2018-07-03 10:13:48 -07:00
Marco Gaido a7c8f0c8cb [SPARK-24385][SQL] Resolve self-join condition ambiguity for EqualNullSafe
## What changes were proposed in this pull request?

In Dataset.join we have a small hack for resolving ambiguity in the column name for self-joins. The current code supports only `EqualTo`.

The PR extends the fix to `EqualNullSafe`.

Credit for this PR should be given to daniel-shields.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21605 from mgaido91/SPARK-24385_2.
2018-07-03 12:20:03 +08:00
maryannxue 797971ed42 [SPARK-24696][SQL] ColumnPruning rule fails to remove extra Project
## What changes were proposed in this pull request?

The ColumnPruning rule tries adding an extra Project if an input node produces fields more than needed, but as a post-processing step, it needs to remove the lower Project in the form of "Project - Filter - Project" otherwise it would conflict with PushPredicatesThroughProject and would thus cause a infinite optimization loop. The current post-processing method is defined as:
```
  private def removeProjectBeforeFilter(plan: LogicalPlan): LogicalPlan = plan transform {
    case p1  Project(_, f  Filter(_, p2  Project(_, child)))
      if p2.outputSet.subsetOf(child.outputSet) =>
      p1.copy(child = f.copy(child = child))
  }
```
This method works well when there is only one Filter but would not if there's two or more Filters. In this case, there is a deterministic filter and a non-deterministic filter so they stay as separate filter nodes and cannot be combined together.

An simplified illustration of the optimization process that forms the infinite loop is shown below (F1 stands for the 1st filter, F2 for the 2nd filter, P for project, S for scan of relation, PredicatePushDown as abbrev. of PushPredicatesThroughProject):
```
                             F1 - F2 - P - S
PredicatePushDown      =>    F1 - P - F2 - S
ColumnPruning          =>    F1 - P - F2 - P - S
                       =>    F1 - P - F2 - S        (Project removed)
PredicatePushDown      =>    P - F1 - F2 - S
ColumnPruning          =>    P - F1 - P - F2 - S
                       =>    P - F1 - P - F2 - P - S
                       =>    P - F1 - F2 - P - S    (only one Project removed)
RemoveRedundantProject =>    F1 - F2 - P - S        (goes back to the loop start)
```
So the problem is the ColumnPruning rule adds a Project under a Filter (and fails to remove it in the end), and that new Project triggers PushPredicateThroughProject. Once the filters have been push through the Project, a new Project will be added by the ColumnPruning rule and this goes on and on.
The fix should be when adding Projects, the rule applies top-down, but later when removing extra Projects, the process should go bottom-up to ensure all extra Projects can be matched.

## How was this patch tested?

Added a optimization rule test in ColumnPruningSuite; and a end-to-end test in SQLQuerySuite.

Author: maryannxue <maryannxue@apache.org>

Closes #21674 from maryannxue/spark-24696.
2018-06-29 23:46:12 -07:00
Yuming Wang 03545ce6de [SPARK-24638][SQL] StringStartsWith support push down
## What changes were proposed in this pull request?

`StringStartsWith` support push down. About 50% savings in compute time.

## How was this patch tested?
unit tests, manual tests and performance test:
```scala
cat <<EOF > SPARK-24638.scala
def benchmark(func: () => Unit): Long = {
  val start = System.currentTimeMillis()
  for(i <- 0 until 100) { func() }
  val end = System.currentTimeMillis()
  end - start
}
val path = "/tmp/spark/parquet/string/"
spark.range(10000000).selectExpr("concat(id, 'str', id) as id").coalesce(1).write.mode("overwrite").option("parquet.block.size", 1048576).parquet(path)
val df = spark.read.parquet(path)

spark.sql("set spark.sql.parquet.filterPushdown.string.startsWith=true")
val pushdownEnable = benchmark(() => df.where("id like '999998%'").count())

spark.sql("set spark.sql.parquet.filterPushdown.string.startsWith=false")
val pushdownDisable = benchmark(() => df.where("id like '999998%'").count())

val improvements = pushdownDisable - pushdownEnable
println(s"improvements: $improvements")
EOF

bin/spark-shell -i SPARK-24638.scala
```
result:
```scala
Loading SPARK-24638.scala...
benchmark: (func: () => Unit)Long
path: String = /tmp/spark/parquet/string/
df: org.apache.spark.sql.DataFrame = [id: string]
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]
pushdownEnable: Long = 11608
res2: org.apache.spark.sql.DataFrame = [key: string, value: string]
pushdownDisable: Long = 31981
improvements: Long = 20373
```

Author: Yuming Wang <yumwang@ebay.com>

Closes #21623 from wangyum/SPARK-24638.
2018-06-30 13:58:50 +08:00
Jose Torres f6e6899a8b [SPARK-24386][SS] coalesce(1) aggregates in continuous processing
## What changes were proposed in this pull request?

Provide a continuous processing implementation of coalesce(1), as well as allowing aggregates on top of it.

The changes in ContinuousQueuedDataReader and such are to use split.index (the ID of the partition within the RDD currently being compute()d) rather than context.partitionId() (the partition ID of the scheduled task within the Spark job - that is, the post coalesce writer). In the absence of a narrow dependency, these values were previously always the same, so there was no need to distinguish.

## How was this patch tested?

new unit test

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21560 from jose-torres/coalesce.
2018-06-28 16:25:40 -07:00
Jacek Laskowski e1d3f80103 [SPARK-24408][SQL][DOC] Move abs function to math_funcs group
## What changes were proposed in this pull request?

A few math functions (`abs` , `bitwiseNOT`, `isnan`, `nanvl`) are not in **math_funcs** group. They should really be.

## How was this patch tested?

Awaiting Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #21448 from jaceklaskowski/SPARK-24408-math-funcs-doc.
2018-06-28 13:22:52 -07:00
Xingbo Jiang 5b05966488 [SPARK-24564][TEST] Add test suite for RecordBinaryComparator
## What changes were proposed in this pull request?

Add a new test suite to test RecordBinaryComparator.

## How was this patch tested?

New test suite.

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

Closes #21570 from jiangxb1987/rbc-test.
2018-06-28 14:19:50 +08:00
Fokko Driesprong 6a97e8eb31 [SPARK-24603][SQL] Fix findTightestCommonType reference in comments
findTightestCommonTypeOfTwo has been renamed to findTightestCommonType

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

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

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

Author: Fokko Driesprong <fokkodriesprong@godatadriven.com>

Closes #21597 from Fokko/fd-typo.
2018-06-28 09:59:00 +08:00
Takeshi Yamamuro 1c9acc2438 [SPARK-24206][SQL][FOLLOW-UP] Update DataSourceReadBenchmark benchmark results
## What changes were proposed in this pull request?
This pr corrected the default configuration (`spark.master=local[1]`) for benchmarks. Also, this updated performance results on the AWS `r3.xlarge`.

## How was this patch tested?
N/A

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21625 from maropu/FixDataSourceReadBenchmark.
2018-06-28 09:21:10 +08:00
Takeshi Yamamuro bd32b509a1 [SPARK-24645][SQL] Skip parsing when csvColumnPruning enabled and partitions scanned only
## What changes were proposed in this pull request?
In the master, when `csvColumnPruning`(implemented in [this commit](64fad0b519 (diff-d19881aceddcaa5c60620fdcda99b4c4))) enabled and partitions scanned only, it throws an exception below;

```
scala> val dir = "/tmp/spark-csv/csv"
scala> spark.range(10).selectExpr("id % 2 AS p", "id").write.mode("overwrite").partitionBy("p").csv(dir)
scala> spark.read.csv(dir).selectExpr("sum(p)").collect()
18/06/25 13:12:51 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 5)
java.lang.NullPointerException
        at org.apache.spark.sql.execution.datasources.csv.UnivocityParser.org$apache$spark$sql$execution$datasources$csv$UnivocityParser$$convert(UnivocityParser.scala:197)
        at org.apache.spark.sql.execution.datasources.csv.UnivocityParser.parse(UnivocityParser.scala:190)
        at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anonfun$5.apply(UnivocityParser.scala:309)
        at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anonfun$5.apply(UnivocityParser.scala:309)
        at org.apache.spark.sql.execution.datasources.FailureSafeParser.parse(FailureSafeParser.scala:61)
        ...
```
This pr modified code to skip CSV parsing in the case.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21631 from maropu/SPARK-24645.
2018-06-28 09:19:25 +08:00
Kallman, Steven c5aa54d54b [SPARK-24553][WEB-UI] http 302 fixes for href redirect
## What changes were proposed in this pull request?

Updated URL/href links to include a '/' before '?id' to make links consistent and avoid http 302 redirect errors within UI port 4040 tabs.

## How was this patch tested?

Built a runnable distribution and executed jobs. Validated that http 302 redirects are no longer encountered when clicking on links within UI port 4040 tabs.

Author: Steven Kallman <SJKallmangmail.com>

Author: Kallman, Steven <Steven.Kallman@CapitalOne.com>

Closes #21600 from SJKallman/{Spark-24553}{WEB-UI}-redirect-href-fixes.
2018-06-27 15:36:59 -07:00
Takeshi Yamamuro 893ea224cc [SPARK-24204][SQL] Verify a schema in Json/Orc/ParquetFileFormat
## What changes were proposed in this pull request?
This pr added code to verify a schema in Json/Orc/ParquetFileFormat along with CSVFileFormat.

## How was this patch tested?
Added verification tests in `FileBasedDataSourceSuite` and  `HiveOrcSourceSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21389 from maropu/SPARK-24204.
2018-06-27 15:25:51 -07:00
debugger87 c04cb2d1b7 [SPARK-21687][SQL] Spark SQL should set createTime for Hive partition
## What changes were proposed in this pull request?

Set createTime for every hive partition created in Spark SQL, which could be used to manage data lifecycle in Hive warehouse. We found  that almost every partition modified by spark sql has not been set createTime.

```
mysql> select * from partitions where create_time=0 limit 1\G;
*************************** 1. row ***************************
         PART_ID: 1028584
     CREATE_TIME: 0
LAST_ACCESS_TIME: 1502203611
       PART_NAME: date=20170130
           SD_ID: 1543605
          TBL_ID: 211605
  LINK_TARGET_ID: NULL
1 row in set (0.27 sec)
```

## How was this patch tested?
 N/A

Author: debugger87 <yangchaozhong.2009@gmail.com>
Author: Chaozhong Yang <yangchaozhong.2009@gmail.com>

Closes #18900 from debugger87/fix/set-create-time-for-hive-partition.
2018-06-27 11:34:28 -07:00
Yuanjian Li 6a0b77a55d [SPARK-24215][PYSPARK][FOLLOW UP] Implement eager evaluation for DataFrame APIs in PySpark
## What changes were proposed in this pull request?

Address comments in #21370 and add more test.

## How was this patch tested?

Enhance test in pyspark/sql/test.py and DataFrameSuite

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21553 from xuanyuanking/SPARK-24215-follow.
2018-06-27 10:43:06 -07:00
Vayda, Oleksandr: IT (PRG) 2669b4de3b [SPARK-23927][SQL] Add "sequence" expression
## What changes were proposed in this pull request?
The PR adds the SQL function ```sequence```.
https://issues.apache.org/jira/browse/SPARK-23927

The behavior of the function is based on Presto's one.
Ref: https://prestodb.io/docs/current/functions/array.html

- ```sequence(start, stop) → array<bigint>```
Generate a sequence of integers from ```start``` to ```stop```, incrementing by ```1``` if ```start``` is less than or equal to ```stop```, otherwise ```-1```.
- ```sequence(start, stop, step) → array<bigint>```
Generate a sequence of integers from ```start``` to ```stop```, incrementing by ```step```.
- ```sequence(start_date, stop_date) → array<date>```
Generate a sequence of dates from ```start_date``` to ```stop_date```, incrementing by ```interval 1 day``` if ```start_date``` is less than or equal to ```stop_date```, otherwise ```- interval 1 day```.
- ```sequence(start_date, stop_date, step_interval) → array<date>```
Generate a sequence of dates from ```start_date``` to ```stop_date```, incrementing by ```step_interval```. The type of ```step_interval``` is ```CalendarInterval```.
- ```sequence(start_timestemp, stop_timestemp) → array<timestamp>```
Generate a sequence of timestamps from ```start_timestamps``` to ```stop_timestamps```, incrementing by ```interval 1 day``` if ```start_date``` is less than or equal to ```stop_date```, otherwise ```- interval 1 day```.
- ```sequence(start_timestamp, stop_timestamp, step_interval) → array<timestamp>```
Generate a sequence of timestamps from ```start_timestamps``` to ```stop_timestamps```, incrementing by ```step_interval```. The type of ```step_interval``` is ```CalendarInterval```.

## How was this patch tested?

Added unit tests.

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21155 from wajda/feature/array-api-sequence.
2018-06-27 11:52:31 +09:00
Maxim Gekk d08f53dc61 [SPARK-24605][SQL] size(null) returns null instead of -1
## What changes were proposed in this pull request?

In PR, I propose new behavior of `size(null)` under the config flag `spark.sql.legacy.sizeOfNull`. If the former one is disabled, the `size()` function returns `null` for `null` input. By default the `spark.sql.legacy.sizeOfNull` is enabled to keep backward compatibility with previous versions. In that case, `size(null)` returns `-1`.

## How was this patch tested?

Modified existing tests for the `size()` function to check new behavior (`null`) and old one (`-1`).

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21598 from MaxGekk/legacy-size-of-null.
2018-06-27 10:36:51 +08:00
Dilip Biswal 02f8781fa2 [SPARK-24423][SQL] Add a new option for JDBC sources
## What changes were proposed in this pull request?
Here is the description in the JIRA -

Currently, our JDBC connector provides the option `dbtable` for users to specify the to-be-loaded JDBC source table.

 ```SQL
 val jdbcDf = spark.read
   .format("jdbc")
   .option("dbtable", "dbName.tableName")
   .options(jdbcCredentials: Map)
   .load()
 ```

Normally, users do not fetch the whole JDBC table due to the poor performance/throughput of JDBC. Thus, they normally just fetch a small set of tables. For advanced users, they can pass a subquery as the option.

 ```SQL
 val query = """ (select * from tableName limit 10) as tmp """
 val jdbcDf = spark.read
   .format("jdbc")
   .option("dbtable", query)
   .options(jdbcCredentials: Map)
   .load()
 ```
However, this is straightforward to end users. We should simply allow users to specify the query by a new option `query`. We will handle the complexity for them.

 ```SQL
 val query = """select * from tableName limit 10"""
 val jdbcDf = spark.read
   .format("jdbc")
   .option("query", query)
   .options(jdbcCredentials: Map)
   .load()
```

## How was this patch tested?
Added tests in JDBCSuite and JDBCWriterSuite.
Also tested against MySQL, Postgress, Oracle, DB2 (using docker infrastructure) to make sure there are no syntax issues.

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

Closes #21590 from dilipbiswal/SPARK-24423.
2018-06-26 15:17:00 -07:00
Marek Novotny e07aee2165 [SPARK-24636][SQL] Type coercion of arrays for array_join function
## What changes were proposed in this pull request?
Presto's implementation accepts arbitrary arrays of primitive types as an input:

```
presto> SELECT array_join(ARRAY [1, 2, 3], ', ');
_col0
---------
1, 2, 3
(1 row)
```

This PR proposes to implement a type coercion rule for ```array_join``` function that converts arrays of primitive as well as non-primitive types to arrays of string.

## How was this patch tested?

New test cases add into:
- sql-tests/inputs/typeCoercion/native/arrayJoin.sql
- DataFrameFunctionsSuite.scala

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21620 from mn-mikke/SPARK-24636.
2018-06-26 09:51:55 +08:00
Marcelo Vanzin 6d16b9885d [SPARK-24552][CORE][SQL] Use task ID instead of attempt number for writes.
This passes the unique task attempt id instead of attempt number to v2 data sources because attempt number is reused when stages are retried. When attempt numbers are reused, sources that track data by partition id and attempt number may incorrectly clean up data because the same attempt number can be both committed and aborted.

For v1 / Hadoop writes, generate a unique ID based on available attempt numbers to avoid a similar problem.

Closes #21558

Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Ryan Blue <blue@apache.org>

Closes #21606 from vanzin/SPARK-24552.2.
2018-06-25 16:54:57 -07:00
Stacy Kerkela 5264164a67 [SPARK-24648][SQL] SqlMetrics should be threadsafe
Use LongAdder to make SQLMetrics thread safe.

## What changes were proposed in this pull request?
Replace += with LongAdder.add() for concurrent counting

## How was this patch tested?
Unit tests with local threads

Author: Stacy Kerkela <stacy.kerkela@databricks.com>

Closes #21634 from dbkerkela/sqlmetrics-concurrency-stacy.
2018-06-25 23:41:39 +02:00
Marco Gaido 594ac4f7b8 [SPARK-24633][SQL] Fix codegen when split is required for arrays_zip
## What changes were proposed in this pull request?

In function array_zip, when split is required by the high number of arguments, a codegen error can happen.

The PR fixes codegen for cases when splitting the code is required.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21621 from mgaido91/SPARK-24633.
2018-06-25 23:44:20 +08:00
Maryann Xue bac50aa371 [SPARK-24596][SQL] Non-cascading Cache Invalidation
## What changes were proposed in this pull request?

1. Add parameter 'cascade' in CacheManager.uncacheQuery(). Under 'cascade=false' mode, only invalidate the current cache, and for other dependent caches, rebuild execution plan and reuse cached buffer.
2. Pass true/false from callers in different uncache scenarios:
- Drop tables and regular (persistent) views: regular mode
- Drop temporary views: non-cascading mode
- Modify table contents (INSERT/UPDATE/MERGE/DELETE): regular mode
- Call `DataSet.unpersist()`: non-cascading mode
- Call `Catalog.uncacheTable()`: follow the same convention as drop tables/view, which is, use non-cascading mode for temporary views and regular mode for the rest

Note that a regular (persistent) view is a database object just like a table, so after dropping a regular view (whether cached or not cached), any query referring to that view should no long be valid. Hence if a cached persistent view is dropped, we need to invalidate the all dependent caches so that exceptions will be thrown for any later reference. On the other hand, a temporary view is in fact equivalent to an unnamed DataSet, and dropping a temporary view should have no impact on queries referencing that view. Thus we should do non-cascading uncaching for temporary views, which also guarantees a consistent uncaching behavior between temporary views and unnamed DataSets.

## How was this patch tested?

New tests in CachedTableSuite and DatasetCacheSuite.

Author: Maryann Xue <maryannxue@apache.org>

Closes #21594 from maryannxue/noncascading-cache.
2018-06-25 07:17:30 -07:00
Takuya UESHIN 6e0596e263 [SPARK-23931][SQL][FOLLOW-UP] Make arrays_zip in function.scala @scala.annotation.varargs.
## What changes were proposed in this pull request?

This is a follow-up pr of #21045 which added `arrays_zip`.
The `arrays_zip` in functions.scala should've been `scala.annotation.varargs`.

This pr makes it `scala.annotation.varargs`.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21630 from ueshin/issues/SPARK-23931/fup1.
2018-06-24 23:56:47 -07:00
Takeshi Yamamuro f596ebe4d3 [SPARK-24327][SQL] Verify and normalize a partition column name based on the JDBC resolved schema
## What changes were proposed in this pull request?
This pr modified JDBC datasource code to verify and normalize a partition column based on the JDBC resolved schema before building `JDBCRelation`.

Closes #20370

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21379 from maropu/SPARK-24327.
2018-06-24 23:14:42 -07:00
Bryan Cutler a5849ad9a3 [SPARK-24324][PYTHON] Pandas Grouped Map UDF should assign result columns by name
## What changes were proposed in this pull request?

Currently, a `pandas_udf` of type `PandasUDFType.GROUPED_MAP` will assign the resulting columns based on index of the return pandas.DataFrame.  If a new DataFrame is returned and constructed using a dict, then the order of the columns could be arbitrary and be different than the defined schema for the UDF.  If the schema types still match, then no error will be raised and the user will see column names and column data mixed up.

This change will first try to assign columns using the return type field names.  If a KeyError occurs, then the column index is checked if it is string based. If so, then the error is raised as it is most likely a naming mistake, else it will fallback to assign columns by position and raise a TypeError if the field types do not match.

## How was this patch tested?

Added a test that returns a new DataFrame with column order different than the schema.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #21427 from BryanCutler/arrow-grouped-map-mixesup-cols-SPARK-24324.
2018-06-24 09:28:46 +08:00
Takeshi Yamamuro 98f363b774 [SPARK-24206][SQL] Improve FilterPushdownBenchmark benchmark code
## What changes were proposed in this pull request?
This pr added benchmark code `FilterPushdownBenchmark` for string pushdown and updated performance results on the AWS `r3.xlarge`.

## How was this patch tested?
N/A

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21288 from maropu/UpdateParquetBenchmark.
2018-06-23 17:51:18 -07:00
Maxim Gekk c7e2742f9b [SPARK-24190][SQL] Allow saving of JSON files in UTF-16 and UTF-32
## What changes were proposed in this pull request?

Currently, restrictions in JSONOptions for `encoding` and `lineSep` are the same for read and for write. For example, a requirement for `lineSep` in the code:

```
df.write.option("encoding", "UTF-32BE").json(file)
```
doesn't allow to skip `lineSep` and use its default value `\n` because it throws the exception:
```
equirement failed: The lineSep option must be specified for the UTF-32BE encoding
java.lang.IllegalArgumentException: requirement failed: The lineSep option must be specified for the UTF-32BE encoding
```

In the PR, I propose to separate JSONOptions in read and write, and make JSONOptions in write less restrictive.

## How was this patch tested?

Added new test for blacklisted encodings in read. And the `lineSep` option was removed in write for some tests.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21247 from MaxGekk/json-options-in-write.
2018-06-23 17:40:20 -07:00
Marek Novotny 92c2f00bd2 [SPARK-23934][SQL] Adding map_from_entries function
## What changes were proposed in this pull request?
The PR adds the `map_from_entries` function that returns a map created from the given array of entries.

## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionSuite`

## CodeGen Examples
### Primitive-type Keys and Values
```
val idf = Seq(
  Seq((1, 10), (2, 20), (3, 10)),
  Seq((1, 10), null, (2, 20))
).toDF("a")
idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */         MapData project_value_0 = null;
/* 044 */
/* 045 */         for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) {
/* 046 */           project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2);
/* 047 */         }
/* 048 */         if (!project_isNull_0) {
/* 049 */           final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */           final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 052 */           final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 053 */           final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0;
/* 054 */           if (project_byteArraySize_0 > 2147483632) {
/* 055 */             final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 056 */             final Object[] project_values_0 = new Object[project_numEntries_0];
/* 057 */
/* 058 */             for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) {
/* 059 */               InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2);
/* 060 */
/* 061 */               project_keys_0[project_idx_1] = project_entry_1.getInt(0);
/* 062 */               project_values_0[project_idx_1] = project_entry_1.getInt(1);
/* 063 */             }
/* 064 */
/* 065 */             project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */           } else {
/* 068 */             final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0];
/* 069 */             UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData();
/* 070 */             Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0);
/* 071 */             Platform.putLong(project_byteArray_0, 24, project_numEntries_0);
/* 072 */             Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0);
/* 073 */             project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0);
/* 074 */             ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray();
/* 075 */             ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray();
/* 076 */
/* 077 */             for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 078 */               InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 079 */
/* 080 */               project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0));
/* 081 */               project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1));
/* 082 */             }
/* 083 */
/* 084 */             project_value_0 = project_unsafeMapData_0;
/* 085 */           }
/* 086 */
/* 087 */         }
```
### Non-primitive-type Keys and Values
```
val sdf = Seq(
  Seq(("a", null), ("b", "bb"), ("c", "aa")),
  Seq(("a", "aa"), null, (null, "bb"))
).toDF("a")
sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */         MapData project_value_0 = null;
/* 044 */
/* 045 */         for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) {
/* 046 */           project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1);
/* 047 */         }
/* 048 */         if (!project_isNull_0) {
/* 049 */           final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */           final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 052 */           final Object[] project_values_0 = new Object[project_numEntries_0];
/* 053 */
/* 054 */           for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 055 */             InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 056 */
/* 057 */             if (project_entry_0.isNullAt(0)) {
/* 058 */               throw new RuntimeException("The first field from a struct (key) can't be null.");
/* 059 */             }
/* 060 */
/* 061 */             project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0);
/* 062 */             project_values_0[project_idx_0] = project_entry_0.getUTF8String(1);
/* 063 */           }
/* 064 */
/* 065 */           project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */         }
```

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 16:18:22 +09:00
Wenchen Fan dc8a6befa5 [SPARK-24588][SS] streaming join should require HashClusteredPartitioning from children
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/19080 we simplified the distribution/partitioning framework, and make all the join-like operators require `HashClusteredDistribution` from children. Unfortunately streaming join operator was missed.

This can cause wrong result. Think about
```
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]

val df1 = input1.toDF.select('value as 'a, 'value * 2 as 'b)
val df2 = input2.toDF.select('value as 'a, 'value * 2 as 'b).repartition('b)
val joined = df1.join(df2, Seq("a", "b")).select('a)
```

The physical plan is
```
*(3) Project [a#5]
+- StreamingSymmetricHashJoin [a#5, b#6], [a#10, b#11], Inner, condition = [ leftOnly = null, rightOnly = null, both = null, full = null ], state info [ checkpoint = <unknown>, runId = 54e31fce-f055-4686-b75d-fcd2b076f8d8, opId = 0, ver = 0, numPartitions = 5], 0, state cleanup [ left = null, right = null ]
   :- Exchange hashpartitioning(a#5, b#6, 5)
   :  +- *(1) Project [value#1 AS a#5, (value#1 * 2) AS b#6]
   :     +- StreamingRelation MemoryStream[value#1], [value#1]
   +- Exchange hashpartitioning(b#11, 5)
      +- *(2) Project [value#3 AS a#10, (value#3 * 2) AS b#11]
         +- StreamingRelation MemoryStream[value#3], [value#3]
```

The left table is hash partitioned by `a, b`, while the right table is hash partitioned by `b`. This means, we may have a matching record that is in different partitions, which should be in the output but not.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21587 from cloud-fan/join.
2018-06-21 15:38:46 -07:00
Maryann Xue b9a6f7499a [SPARK-24613][SQL] Cache with UDF could not be matched with subsequent dependent caches
## What changes were proposed in this pull request?

Wrap the logical plan with a `AnalysisBarrier` for execution plan compilation in CacheManager, in order to avoid the plan being analyzed again.

## How was this patch tested?

Add one test in `DatasetCacheSuite`

Author: Maryann Xue <maryannxue@apache.org>

Closes #21602 from maryannxue/cache-mismatch.
2018-06-21 11:45:30 -07:00
Marcelo Vanzin c8e909cd49 [SPARK-24589][CORE] Correctly identify tasks in output commit coordinator.
When an output stage is retried, it's possible that tasks from the previous
attempt are still running. In that case, there would be a new task for the
same partition in the new attempt, and the coordinator would allow both
tasks to commit their output since it did not keep track of stage attempts.

The change adds more information to the stage state tracked by the coordinator,
so that only one task is allowed to commit the output in the above case.
The stage state in the coordinator is also maintained across stage retries,
so that a stray speculative task from a previous stage attempt is not allowed
to commit.

This also removes some code added in SPARK-18113 that allowed for duplicate
commit requests; with the RPC code used in Spark 2, that situation cannot
happen, so there is no need to handle it.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21577 from vanzin/SPARK-24552.
2018-06-21 13:25:15 -05:00
Chongguang LIU 7236e759c9 [SPARK-24574][SQL] array_contains, array_position, array_remove and element_at functions deal with Column type
## What changes were proposed in this pull request?

For the function ```def array_contains(column: Column, value: Any): Column ``` , if we pass the `value` parameter as a Column type, it will yield a runtime exception.

This PR proposes a pattern matching to detect if `value` is of type Column. If yes, it will use the .expr of the column, otherwise it will work as it used to.

Same thing for ```array_position, array_remove and element_at``` functions

## How was this patch tested?

Unit test modified to cover this code change.

Ping ueshin

Author: Chongguang LIU <chong@Chongguangs-MacBook-Pro.local>

Closes #21581 from chongguang/SPARK-24574.
2018-06-21 14:58:57 +08:00
Maxim Gekk 54fcaafb09 [SPARK-24571][SQL] Support Char literals
## What changes were proposed in this pull request?

In the PR, I propose to automatically convert a `Literal` with `Char` type to a `Literal` of `String` type. Currently, the following code:
```scala
val df = Seq("Amsterdam", "San Francisco", "London").toDF("city")
df.where($"city".contains('o')).show(false)
```
fails with the exception:
```
Unsupported literal type class java.lang.Character o
java.lang.RuntimeException: Unsupported literal type class java.lang.Character o
at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:78)
```
The PR fixes this issue by converting `char` to `string` of length `1`. I believe it makes sense to does not differentiate `char` and `string(1)` in _a unified, multi-language data platform_ like Spark which supports languages like Python/R.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21578 from MaxGekk/support-char-literals.
2018-06-20 23:38:37 -07:00
Huaxin Gao 9de11d3f90 [SPARK-23912][SQL] add array_distinct
## What changes were proposed in this pull request?

Add array_distinct to remove duplicate value from the array.

## How was this patch tested?

Add unit tests

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

Closes #21050 from huaxingao/spark-23912.
2018-06-21 12:24:53 +09:00
aokolnychyi c5a0d1132a [SPARK-24575][SQL] Prohibit window expressions inside WHERE and HAVING clauses
## What changes were proposed in this pull request?

As discussed [before](https://github.com/apache/spark/pull/19193#issuecomment-393726964), this PR prohibits window expressions inside WHERE and HAVING clauses.

## How was this patch tested?

This PR comes with a dedicated unit test.

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

Closes #21580 from aokolnychyi/spark-24575.
2018-06-20 18:57:13 +02:00
Jungtaek Lim c8ef9232cf [MINOR][SQL] Remove invalid comment from SparkStrategies
## What changes were proposed in this pull request?

This patch is removing invalid comment from SparkStrategies, given that TODO-like comment is no longer preferred one as the comment: https://github.com/apache/spark/pull/21388#issuecomment-396856235

Removing invalid comment will prevent contributors to spend their times which is not going to be merged.

## How was this patch tested?

N/A

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21595 from HeartSaVioR/MINOR-remove-invalid-comment-on-spark-strategies.
2018-06-20 18:38:42 +02:00
Maryann Xue bc0498d582 [SPARK-24583][SQL] Wrong schema type in InsertIntoDataSourceCommand
## What changes were proposed in this pull request?

Change insert input schema type: "insertRelationType" -> "insertRelationType.asNullable", in order to avoid nullable being overridden.

## How was this patch tested?

Added one test in InsertSuite.

Author: Maryann Xue <maryannxue@apache.org>

Closes #21585 from maryannxue/spark-24583.
2018-06-19 15:27:20 -07:00
Tathagata Das 2cb976355c [SPARK-24565][SS] Add API for in Structured Streaming for exposing output rows of each microbatch as a DataFrame
## What changes were proposed in this pull request?

Currently, the micro-batches in the MicroBatchExecution is not exposed to the user through any public API. This was because we did not want to expose the micro-batches, so that all the APIs we expose, we can eventually support them in the Continuous engine. But now that we have better sense of buiding a ContinuousExecution, I am considering adding APIs which will run only the MicroBatchExecution. I have quite a few use cases where exposing the microbatch output as a dataframe is useful.
- Pass the output rows of each batch to a library that is designed only the batch jobs (example, uses many ML libraries need to collect() while learning).
- Reuse batch data sources for output whose streaming version does not exists (e.g. redshift data source).
- Writer the output rows to multiple places by writing twice for each batch. This is not the most elegant thing to do for multiple-output streaming queries but is likely to be better than running two streaming queries processing the same data twice.

The proposal is to add a method `foreachBatch(f: Dataset[T] => Unit)` to Scala/Java/Python `DataStreamWriter`.

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

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

Closes #21571 from tdas/foreachBatch.
2018-06-19 13:56:51 -07:00
yucai 9dbe53eb6b [SPARK-24556][SQL] Always rewrite output partitioning in ReusedExchangeExec and InMemoryTableScanExec
## What changes were proposed in this pull request?

Currently, ReusedExchange and InMemoryTableScanExec only rewrite output partitioning if child's partitioning is HashPartitioning and do nothing for other partitioning, e.g., RangePartitioning. We should always rewrite it, otherwise, unnecessary shuffle could be introduced like https://issues.apache.org/jira/browse/SPARK-24556.

## How was this patch tested?

Add new tests.

Author: yucai <yyu1@ebay.com>

Closes #21564 from yucai/SPARK-24556.
2018-06-19 10:52:51 -07:00
Li Jin a78a904641 [SPARK-24521][SQL][TEST] Fix ineffective test in CachedTableSuite
## What changes were proposed in this pull request?

test("withColumn doesn't invalidate cached dataframe") in CachedTableSuite doesn't not work because:

The UDF is executed and test count incremented when "df.cache()" is called and the subsequent "df.collect()" has no effect on the test result.

This PR fixed this test and add another test for caching UDF.

## How was this patch tested?

Add new tests.

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

Closes #21531 from icexelloss/fix-cache-test.
2018-06-19 10:42:08 -07:00
Wenchen Fan 1737d45e08 [SPARK-24478][SQL][FOLLOWUP] Move projection and filter push down to physical conversion
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/21503, to completely move operator pushdown to the planner rule.

The code are mostly from https://github.com/apache/spark/pull/21319

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21574 from cloud-fan/followup.
2018-06-18 20:15:01 -07:00
Liang-Chi Hsieh 8f225e055c [SPARK-24548][SQL] Fix incorrect schema of Dataset with tuple encoders
## What changes were proposed in this pull request?

When creating tuple expression encoders, we should give the serializer expressions of tuple items correct names, so we can have correct output schema when we use such tuple encoders.

## How was this patch tested?

Added test.

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

Closes #21576 from viirya/SPARK-24548.
2018-06-18 11:01:17 -07:00
Takeshi Yamamuro e219e692ef [SPARK-23772][SQL] Provide an option to ignore column of all null values or empty array during JSON schema inference
## What changes were proposed in this pull request?
This pr added a new JSON option `dropFieldIfAllNull ` to ignore column of all null values or empty array/struct during JSON schema inference.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Xiangrui Meng <meng@databricks.com>

Closes #20929 from maropu/SPARK-23772.
2018-06-19 00:24:54 +08:00
Mukul Murthy e4fee395ec [SPARK-24525][SS] Provide an option to limit number of rows in a MemorySink
## What changes were proposed in this pull request?

Provide an option to limit number of rows in a MemorySink. Currently, MemorySink and MemorySinkV2 have unbounded size, meaning that if they're used on big data, they can OOM the stream. This change adds a maxMemorySinkRows option to limit how many rows MemorySink and MemorySinkV2 can hold. By default, they are still unbounded.

## How was this patch tested?

Added new unit tests.

Author: Mukul Murthy <mukul.murthy@databricks.com>

Closes #21559 from mukulmurthy/SPARK-24525.
2018-06-15 13:56:48 -07:00
Kazuaki Ishizaki 90da7dc241 [SPARK-24452][SQL][CORE] Avoid possible overflow in int add or multiple
## What changes were proposed in this pull request?

This PR fixes possible overflow in int add or multiply. In particular, their overflows in multiply are detected by [Spotbugs](https://spotbugs.github.io/)

The following assignments may cause overflow in right hand side. As a result, the result may be negative.
```
long = int * int
long = int + int
```

To avoid this problem, this PR performs cast from int to long in right hand side.

## How was this patch tested?

Existing UTs.

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

Closes #21481 from kiszk/SPARK-24452.
2018-06-15 13:47:48 -07:00
Tathagata Das b5ccf0d395 [SPARK-24396][SS][PYSPARK] Add Structured Streaming ForeachWriter for python
## What changes were proposed in this pull request?

This PR adds `foreach` for streaming queries in Python. Users will be able to specify their processing logic in two different ways.
- As a function that takes a row as input.
- As an object that has methods `open`, `process`, and `close` methods.

See the python docs in this PR for more details.

## How was this patch tested?
Added java and python unit tests

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

Closes #21477 from tdas/SPARK-24396.
2018-06-15 12:56:39 -07:00
Ryan Blue 22daeba59b [SPARK-24478][SQL] Move projection and filter push down to physical conversion
## What changes were proposed in this pull request?

This removes the v2 optimizer rule for push-down and instead pushes filters and required columns when converting to a physical plan, as suggested by marmbrus. This makes the v2 relation cleaner because the output and filters do not change in the logical plan.

A side-effect of this change is that the stats from the logical (optimized) plan no longer reflect pushed filters and projection. This is a temporary state, until the planner gathers stats from the physical plan instead. An alternative to this approach is 9d3a11e68b.

The first commit was proposed in #21262. This PR replaces #21262.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #21503 from rdblue/SPARK-24478-move-push-down-to-physical-conversion.
2018-06-14 20:59:42 -07:00
Maxim Gekk b8f27ae3b3 [SPARK-24543][SQL] Support any type as DDL string for from_json's schema
## What changes were proposed in this pull request?

In the PR, I propose to support any DataType represented as DDL string for the from_json function. After the changes, it will be possible to specify `MapType` in SQL like:
```sql
select from_json('{"a":1, "b":2}', 'map<string, int>')
```
and in Scala (similar in other languages)
```scala
val in = Seq("""{"a": {"b": 1}}""").toDS()
val schema = "map<string, map<string, int>>"
val out = in.select(from_json($"value", schema, Map.empty[String, String]))
```

## How was this patch tested?

Added a couple sql tests and modified existing tests for Python and Scala. The former tests were modified because it is not imported for them in which format schema for `from_json` is provided.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21550 from MaxGekk/from_json-ddl-schema.
2018-06-14 13:27:27 -07:00
Marco Gaido fdadc4be08 [SPARK-24495][SQL] EnsureRequirement returns wrong plan when reordering equal keys
## What changes were proposed in this pull request?

`EnsureRequirement` in its `reorder` method currently assumes that the same key appears only once in the join condition. This of course might not be the case, and when it is not satisfied, it returns a wrong plan which produces a wrong result of the query.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21529 from mgaido91/SPARK-24495.
2018-06-14 09:20:41 -07:00
Jose Torres 1b46f41c55 [SPARK-24235][SS] Implement continuous shuffle writer for single reader partition.
## What changes were proposed in this pull request?

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit

Implement continuous shuffle write RDD for a single reader partition. (I don't believe any implementation changes are actually required for multiple reader partitions, but this PR is already very large, so I want to exclude those for now to keep the size down.)

## How was this patch tested?

new unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21428 from jose-torres/writerTask.
2018-06-13 13:13:01 -07:00
Herman van Hovell 299d297e25 [SPARK-24500][SQL] Make sure streams are materialized during Tree transforms.
## What changes were proposed in this pull request?
If you construct catalyst trees using `scala.collection.immutable.Stream` you can run into situations where valid transformations do not seem to have any effect. There are two causes for this behavior:
- `Stream` is evaluated lazily. Note that default implementation will generally only evaluate a function for the first element (this makes testing a bit tricky).
- `TreeNode` and `QueryPlan` use side effects to detect if a tree has changed. Mapping over a stream is lazy and does not need to trigger this side effect. If this happens the node will invalidly assume that it did not change and return itself instead if the newly created node (this is for GC reasons).

This PR fixes this issue by forcing materialization on streams in `TreeNode` and `QueryPlan`.

## How was this patch tested?
Unit tests were added to `TreeNodeSuite` and `LogicalPlanSuite`. An integration test was added to the `PlannerSuite`

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

Closes #21539 from hvanhovell/SPARK-24500.
2018-06-13 07:09:48 -07:00
Arun Mahadevan 7703b46d28 [SPARK-24479][SS] Added config for registering streamingQueryListeners
## What changes were proposed in this pull request?

Currently a "StreamingQueryListener" can only be registered programatically. We could have a new config "spark.sql.streamingQueryListeners" similar to  "spark.sql.queryExecutionListeners" and "spark.extraListeners" for users to register custom streaming listeners.

## How was this patch tested?

New unit test and running example programs.

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

Author: Arun Mahadevan <arunm@apache.org>

Closes #21504 from arunmahadevan/SPARK-24480.
2018-06-13 20:43:16 +08:00
Jungtaek Lim 4c388bccf1 [SPARK-24485][SS] Measure and log elapsed time for filesystem operations in HDFSBackedStateStoreProvider
## What changes were proposed in this pull request?

This patch measures and logs elapsed time for each operation which communicate with file system (mostly remote HDFS in production) in HDFSBackedStateStoreProvider to help investigating any latency issue.

## How was this patch tested?

Manually tested.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21506 from HeartSaVioR/SPARK-24485.
2018-06-13 12:36:20 +08:00
Jungtaek Lim 3352d6fe9a [SPARK-24466][SS] Fix TextSocketMicroBatchReader to be compatible with netcat again
## What changes were proposed in this pull request?

TextSocketMicroBatchReader was no longer be compatible with netcat due to launching temporary reader for reading schema, and closing reader, and re-opening reader. While reliable socket server should be able to handle this without any issue, nc command normally can't handle multiple connections and simply exits when closing temporary reader.

This patch fixes TextSocketMicroBatchReader to be compatible with netcat again, via deferring opening socket to the first call of planInputPartitions() instead of constructor.

## How was this patch tested?

Added unit test which fails on current and succeeds with the patch. And also manually tested.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21497 from HeartSaVioR/SPARK-24466.
2018-06-13 12:34:46 +08:00
Li Jin 9786ce66c5 [SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.

```
       >>> from pyspark.sql.functions import pandas_udf, PandasUDFType
       >>> from pyspark.sql import Window
       >>> df = spark.createDataFrame(
       ...     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
       ...     ("id", "v"))
       >>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
       ... def mean_udf(v):
       ...     return v.mean()
       >>> w = Window.partitionBy('id')
       >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
       +---+----+------+
       | id|   v|mean_v|
       +---+----+------+
       |  1| 1.0|   1.5|
       |  1| 2.0|   1.5|
       |  2| 3.0|   6.0|
       |  2| 5.0|   6.0|
       |  2|10.0|   6.0|
       +---+----+------+
```

The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)

Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.

## How was this patch tested?

WindowPandasUDFTests

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

Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-13 09:10:52 +08:00
Kazuaki Ishizaki ada28f2595 [SPARK-23933][SQL] Add map_from_arrays function
## What changes were proposed in this pull request?

The PR adds the SQL function `map_from_arrays`. The behavior of the function is based on Presto's `map`. Since SparkSQL already had a `map` function, we prepared the different name for this behavior.

This function returns returns a map from a pair of arrays for keys and values.

## How was this patch tested?

Added UTs

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

Closes #21258 from kiszk/SPARK-23933.
2018-06-12 12:31:22 -07:00
Fangshi Li cc88d7fad1 [SPARK-24216][SQL] Spark TypedAggregateExpression uses getSimpleName that is not safe in scala
## What changes were proposed in this pull request?

When user create a aggregator object in scala and pass the aggregator to Spark Dataset's agg() method, Spark's will initialize TypedAggregateExpression with the nodeName field as aggregator.getClass.getSimpleName. However, getSimpleName is not safe in scala environment, depending on how user creates the aggregator object. For example, if the aggregator class full qualified name is "com.my.company.MyUtils$myAgg$2$", the getSimpleName will throw java.lang.InternalError "Malformed class name". This has been reported in scalatest https://github.com/scalatest/scalatest/pull/1044 and discussed in many scala upstream jiras such as SI-8110, SI-5425.

To fix this issue, we follow the solution in https://github.com/scalatest/scalatest/pull/1044 to add safer version of getSimpleName as a util method, and TypedAggregateExpression will invoke this util method rather than getClass.getSimpleName.

## How was this patch tested?
added unit test

Author: Fangshi Li <fli@linkedin.com>

Closes #21276 from fangshil/SPARK-24216.
2018-06-12 12:10:08 -07:00
DylanGuedes f0ef1b311d [SPARK-23931][SQL] Adds arrays_zip function to sparksql
Signed-off-by: DylanGuedes <djmgguedesgmail.com>

## What changes were proposed in this pull request?

Addition of arrays_zip function to spark sql functions.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Unit tests that checks if the results are correct.

Author: DylanGuedes <djmgguedes@gmail.com>

Closes #21045 from DylanGuedes/SPARK-23931.
2018-06-12 11:57:25 -07:00
Tom Saleeba 1d7db65e96 docs: fix typo
no => no[t]

## What changes were proposed in this pull request?

Fixing a typo.

## How was this patch tested?

Visual check of the docs.

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

Author: Tom Saleeba <tom.saleeba@gmail.com>

Closes #21496 from tomsaleeba/patch-1.
2018-06-12 09:22:52 -05:00
Wenchen Fan 01452ea9c7 [SPARK-24502][SQL] flaky test: UnsafeRowSerializerSuite
## What changes were proposed in this pull request?

`UnsafeRowSerializerSuite` calls `UnsafeProjection.create` which accesses `SQLConf.get`, while the current active SparkSession may already be stopped, and we may hit exception like this

```
sbt.ForkMain$ForkError: java.lang.IllegalStateException: LiveListenerBus is stopped.
	at org.apache.spark.scheduler.LiveListenerBus.addToQueue(LiveListenerBus.scala:97)
	at org.apache.spark.scheduler.LiveListenerBus.addToStatusQueue(LiveListenerBus.scala:80)
	at org.apache.spark.sql.internal.SharedState.<init>(SharedState.scala:93)
	at org.apache.spark.sql.SparkSession$$anonfun$sharedState$1.apply(SparkSession.scala:120)
	at org.apache.spark.sql.SparkSession$$anonfun$sharedState$1.apply(SparkSession.scala:120)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.sql.SparkSession.sharedState$lzycompute(SparkSession.scala:120)
	at org.apache.spark.sql.SparkSession.sharedState(SparkSession.scala:119)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:286)
	at org.apache.spark.sql.test.TestSparkSession.sessionState$lzycompute(TestSQLContext.scala:42)
	at org.apache.spark.sql.test.TestSparkSession.sessionState(TestSQLContext.scala:41)
	at org.apache.spark.sql.SparkSession$$anonfun$1$$anonfun$apply$1.apply(SparkSession.scala:95)
	at org.apache.spark.sql.SparkSession$$anonfun$1$$anonfun$apply$1.apply(SparkSession.scala:95)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.sql.SparkSession$$anonfun$1.apply(SparkSession.scala:95)
	at org.apache.spark.sql.SparkSession$$anonfun$1.apply(SparkSession.scala:94)
	at org.apache.spark.sql.internal.SQLConf$.get(SQLConf.scala:126)
	at org.apache.spark.sql.catalyst.expressions.CodeGeneratorWithInterpretedFallback.createObject(CodeGeneratorWithInterpretedFallback.scala:54)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:157)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:150)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite.org$apache$spark$sql$execution$UnsafeRowSerializerSuite$$unsafeRowConverter(UnsafeRowSerializerSuite.scala:54)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite.org$apache$spark$sql$execution$UnsafeRowSerializerSuite$$toUnsafeRow(UnsafeRowSerializerSuite.scala:49)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite$$anonfun$2.apply(UnsafeRowSerializerSuite.scala:63)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite$$anonfun$2.apply(UnsafeRowSerializerSuite.scala:60)
...
```

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21518 from cloud-fan/test.
2018-06-11 22:08:44 -07:00
liutang123 048197749e [SPARK-22144][SQL] ExchangeCoordinator combine the partitions of an 0 sized pre-shuffle to 0
## What changes were proposed in this pull request?
when the length of pre-shuffle's partitions is 0, the length of post-shuffle's partitions should be 0 instead of spark.sql.shuffle.partitions.

## How was this patch tested?
ExchangeCoordinator converted a  pre-shuffle that partitions is 0 to a post-shuffle that partitions is 0 instead of one that partitions is spark.sql.shuffle.partitions.

Author: liutang123 <liutang123@yeah.net>

Closes #19364 from liutang123/SPARK-22144.
2018-06-11 17:48:07 -07:00
Marco Gaido f07c5064a3 [SPARK-24468][SQL] Handle negative scale when adjusting precision for decimal operations
## What changes were proposed in this pull request?

In SPARK-22036 we introduced the possibility to allow precision loss in arithmetic operations (according to the SQL standard). The implementation was drawn from Hive's one, where Decimals with a negative scale are not allowed in the operations.

The PR handles the case when the scale is negative, removing the assertion that it is not.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21499 from mgaido91/SPARK-24468.
2018-06-08 18:51:56 -07:00
Thiruvasakan Paramasivan 36a3409134
[SPARK-24412][SQL] Adding docs about automagical type casting in isin and isInCollection APIs
## What changes were proposed in this pull request?
Update documentation for `isInCollection` API to clealy explain the "auto-casting" of elements if their types are different.

## How was this patch tested?
No-Op

Author: Thiruvasakan Paramasivan <thiru@apple.com>

Closes #21519 from trvskn/sql-doc-update.
2018-06-08 17:17:43 -07:00
Asher Saban e76b0124fb [SPARK-23803][SQL] Support bucket pruning
## What changes were proposed in this pull request?
support bucket pruning when filtering on a single bucketed column on the following predicates -
EqualTo, EqualNullSafe, In, And/Or predicates

## How was this patch tested?
refactored unit tests to test the above.

based on gatorsmile work in e3c75c6398

Author: Asher Saban <asaban@palantir.com>
Author: asaban <asaban@palantir.com>

Closes #20915 from sabanas/filter-prune-buckets.
2018-06-06 07:14:08 -07:00
Tathagata Das 2c2a86b5d5 [SPARK-24453][SS] Fix error recovering from the failure in a no-data batch
## What changes were proposed in this pull request?

The error occurs when we are recovering from a failure in a no-data batch (say X) that has been planned (i.e. written to offset log) but not executed (i.e. not written to commit log). Upon recovery the following sequence of events happen.

1. `MicroBatchExecution.populateStartOffsets` sets `currentBatchId` to X. Since there was no data in the batch, the `availableOffsets` is same as `committedOffsets`, so `isNewDataAvailable` is `false`.
2. When `MicroBatchExecution.constructNextBatch` is called, ideally it should immediately return true because the next batch has already been constructed. However, the check of whether the batch has been constructed was `if (isNewDataAvailable) return true`. Since the planned batch is a no-data batch, it escaped this check and proceeded to plan the same batch X *once again*.

The solution is to have an explicit flag that signifies whether a batch has already been constructed or not. `populateStartOffsets` is going to set the flag appropriately.

## How was this patch tested?

new unit test

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

Closes #21491 from tdas/SPARK-24453.
2018-06-05 01:08:55 -07:00
Yuanjian Li dbb4d83829 [SPARK-24215][PYSPARK] Implement _repr_html_ for dataframes in PySpark
## What changes were proposed in this pull request?

Implement `_repr_html_` for PySpark while in notebook and add config named "spark.sql.repl.eagerEval.enabled" to control this.

The dev list thread for context: http://apache-spark-developers-list.1001551.n3.nabble.com/eager-execution-and-debuggability-td23928.html

## How was this patch tested?

New ut in DataFrameSuite and manual test in jupyter. Some screenshot below.

**After:**
![image](https://user-images.githubusercontent.com/4833765/40268422-8db5bef0-5b9f-11e8-80f1-04bc654a4f2c.png)

**Before:**
![image](https://user-images.githubusercontent.com/4833765/40268431-9f92c1b8-5b9f-11e8-9db9-0611f0940b26.png)

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21370 from xuanyuanking/SPARK-24215.
2018-06-05 08:23:08 +07:00
aokolnychyi 7297ae04d8 [SPARK-21896][SQL] Fix StackOverflow caused by window functions inside aggregate functions
## What changes were proposed in this pull request?

This PR explicitly prohibits window functions inside aggregates. Currently, this will cause StackOverflow during analysis. See PR #19193 for previous discussion.

## How was this patch tested?

This PR comes with a dedicated unit test.

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

Closes #21473 from aokolnychyi/fix-stackoverflow-window-funcs.
2018-06-04 13:28:16 -07:00
Yuming Wang 0be5aa2746 [SPARK-23903][SQL] Add support for date extract
## What changes were proposed in this pull request?

Add support for date `extract` function:
```sql
spark-sql> SELECT EXTRACT(YEAR FROM TIMESTAMP '2000-12-16 12:21:13');
2000
```
Supported field same as [Hive](https://github.com/apache/hive/blob/rel/release-2.3.3/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g#L308-L316): `YEAR`, `QUARTER`, `MONTH`, `WEEK`, `DAY`, `DAYOFWEEK`, `HOUR`, `MINUTE`, `SECOND`.

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21479 from wangyum/SPARK-23903.
2018-06-04 10:16:13 -07:00
Maxim Gekk 1d9338bb10 [SPARK-23786][SQL] Checking column names of csv headers
## What changes were proposed in this pull request?

Currently column names of headers in CSV files are not checked against provided schema of CSV data. It could cause errors like showed in the [SPARK-23786](https://issues.apache.org/jira/browse/SPARK-23786) and https://github.com/apache/spark/pull/20894#issuecomment-375957777. I introduced new CSV option - `enforceSchema`. If it is enabled (by default `true`), Spark forcibly applies provided or inferred schema to CSV files. In that case, CSV headers are ignored and not checked against the schema. If `enforceSchema` is set to `false`, additional checks can be performed. For example, if column in CSV header and in the schema have different ordering, the following exception is thrown:

```
java.lang.IllegalArgumentException: CSV file header does not contain the expected fields
 Header: depth, temperature
 Schema: temperature, depth
CSV file: marina.csv
```

## How was this patch tested?

The changes were tested by existing tests of CSVSuite and by 2 new tests.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #20894 from MaxGekk/check-column-names.
2018-06-03 22:02:21 -07:00
Wenchen Fan 416cd1fd96 [SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set
## What changes were proposed in this pull request?

bring back https://github.com/apache/spark/pull/21443

This is a different approach: just change the check to count distinct columns with `toSet`

## How was this patch tested?

a new test to verify the planner behavior.

Author: Wenchen Fan <wenchen@databricks.com>
Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21487 from cloud-fan/back.
2018-06-03 21:57:42 -07:00
Xiao Li d2c3de7efc Revert "[SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set"
This reverts commit 1e46f92f95.
2018-06-01 11:51:10 -07:00
Huang Tengfei 6039b13230 [SPARK-24351][SS] offsetLog/commitLog purge thresholdBatchId should be computed with current committed epoch but not currentBatchId in CP mode
## What changes were proposed in this pull request?
Compute the thresholdBatchId to purge metadata based on current committed epoch instead of currentBatchId in CP mode to avoid cleaning all the committed metadata in some case as described in the jira [SPARK-24351](https://issues.apache.org/jira/browse/SPARK-24351).

## How was this patch tested?
Add new unit test.

Author: Huang Tengfei <tengfei.h@gmail.com>

Closes #21400 from ivoson/branch-cp-meta.
2018-06-01 10:47:53 -07:00