Commit graph

24480 commits

Author SHA1 Message Date
Dongjoon Hyun e5d95117e4 [SPARK-27979][BUILD][test-maven] Remove deprecated --force option in build/mvn and run-tests.py
## What changes were proposed in this pull request?

This is a second try of #24824.

Since Apache Spark 2.0.0, SPARK-14867 deprecated `--force` option and made it ignored. This PR cleans up the related code completely at 3.0.0.

**BEFORE (Jenkins)**
```
========================================================================
Building Spark
========================================================================
[info] Building Spark using Maven with these arguments:  -Phadoop-2.7 -Pkubernetes -Phive-thriftserver -Pkinesis-asl -Pyarn -Pspark-ganglia-lgpl -Phive -Pmesos clean package -DskipTests
WARNING: '--force' is deprecated and ignored.
...
========================================================================
Running Spark unit tests
========================================================================
[info] Running Spark tests using Maven with these arguments:  -Phadoop-2.7 -Phive-thriftserver -Phive -Dtest.exclude.tags=org.apache.spark.tags.ExtendedHiveTest,org.apache.spark.tags.ExtendedYarnTest test --fail-at-end
WARNING: '--force' is deprecated and ignored.
```

**AFTER (Jenkins)**
```
========================================================================
Building Spark
========================================================================
[info] Building Spark using Maven with these arguments:  -Phadoop-2.7 -Pkubernetes -Phive-thriftserver -Pkinesis-asl -Pyarn -Pspark-ganglia-lgpl -Phive -Pmesos clean package -DskipTests
...
========================================================================
Running Spark unit tests
========================================================================
[info] Running Spark tests using Maven with these arguments:  -Phadoop-2.7 -Pkubernetes -Phive-thriftserver -Pyarn -Pspark-ganglia-lgpl -Phive -Pkinesis-asl -Pmesos -Dtest.exclude.tags=org.apache.spark.tags.ExtendedHiveTest,org.apache.spark.tags.ExtendedYarnTest test --fail-at-end
```

## How was this patch tested?

Manually check the Jenkins logs.

Closes #24833 from dongjoon-hyun/SPARK-FORCE-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 18:40:46 -07:00
John Zhuge dbba3a33bc [SPARK-27947][SQL] Enhance redactOptions to accept any Map type
## What changes were proposed in this pull request?

Handle the case when ParsedStatement subclass has a Map field but not of type Map[String, String].

In ParsedStatement.productIterator, `case mapArg: Map[_, _]` can match any Map type due to type erasure, thus causing `asInstanceOf[Map[String, String]]` to throw ClassCastException.

The following test reproduces the issue:
```
case class TestStatement(p: Map[String, Int]) extends ParsedStatement {
 override def output: Seq[Attribute] = Nil
 override def children: Seq[LogicalPlan] = Nil
}

TestStatement(Map("abc" -> 1)).toString
```
Changing the code to `case mapArg: Map[String, String]` will not help due to type erasure. As a matter of fact, compiler gives this warning:
```
Warning:(41, 18) non-variable type argument String in type pattern
 scala.collection.immutable.Map[String,String] (the underlying of Map[String,String])
 is unchecked since it is eliminated by erasure
case mapArg: Map[String, String] =>
```

## How was this patch tested?

Add 2 unit tests.

Closes #24800 from jzhuge/SPARK-27947.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 11:58:37 -07:00
Zhu, Lipeng 3b37bfde2a [SPARK-27949][SQL] Support SUBSTRING(str FROM n1 [FOR n2]) syntax
## What changes were proposed in this pull request?

Currently, function `substr/substring`'s usage is like `substring(string_expression, n1 [,n2])`.

But, the ANSI SQL defined the pattern for substr/substring is like `SUBSTRING(str FROM n1 [FOR n2])`. This gap makes some inconvenient when we switch to the SparkSQL.

- ANSI SQL-92: http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt

Below are the mainly DB engines to support the ANSI standard for substring.
- PostgreSQL https://www.postgresql.org/docs/9.1/functions-string.html
- MySQL https://dev.mysql.com/doc/refman/8.0/en/string-functions.html#function_substring
- Redshift https://docs.aws.amazon.com/redshift/latest/dg/r_SUBSTRING.html
- Teradata https://docs.teradata.com/reader/756LNiPSFdY~4JcCCcR5Cw/XnePye0Cwexw6Pny_qnxVA

**Oracle, SQL Server, Hive, Presto don't have this additional syntax.**

## How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #24802 from lipzhu/SPARK-27949.

Authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 09:05:10 -07:00
Chaerim Yeo c1bb3316bd [SPARK-27425][SQL] Add count_if function
## What changes were proposed in this pull request?

Add `count_if` function which returns the number of records satisfying a given condition.

There is no aggregation function like this in Spark, so we need to write like
- `COUNT(CASE WHEN some_condition THEN 1 END)` or
- `SUM(CASE WHEN some_condition THEN 1 END)`, 
which looks painful.

This kind of function is already supported in Presto, BigQuery and even Excel.
- Presto: [`count_if`](https://prestodb.github.io/docs/current/functions/aggregate.html#count_if)
- BigQuery: [`countif`](https://cloud.google.com/bigquery/docs/reference/standard-sql/aggregate_functions?hl=en#countif)
- Excel: [`COUNTIF`](https://support.office.com/en-us/article/countif-function-e0de10c6-f885-4e71-abb4-1f464816df34?omkt=en-US&ui=en-US&rs=en-US&ad=US) (It is a little different from above twos)

## How was this patch tested?

This patch is tested by unit test.

Closes #24335 from cryeo/SPARK-27425.

Authored-by: Chaerim Yeo <yeochaerim@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-10 19:51:55 +09:00
sandeep katta 773cfde680 [SPARK-27917][SQL] canonical form of CaseWhen object is incorrect
## What changes were proposed in this pull request?

For caseWhen Object canonicalized is not handled

for e.g let's consider below CaseWhen Object
    val attrRef = AttributeReference("ACCESS_CHECK", StringType)()
    val caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A"))))

caseWhenObj1.canonicalized **ouput** is as below

CASE WHEN ACCESS_CHECK#0 THEN A END (**Before Fix)**

**After Fix** : CASE WHEN none#0 THEN A END

So when there will be aliasref like below statements, semantic equals will fail. Sematic equals returns true if the canonicalized form of both the expressions are same.

val attrRef = AttributeReference("ACCESS_CHECK", StringType)()
val aliasAttrRef = attrRef.withName("access_check")
val caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A"))))
val caseWhenObj2 = CaseWhen(Seq((aliasAttrRef, Literal("A"))))

**assert(caseWhenObj2.semanticEquals(caseWhenObj1.semanticEquals) fails**

**caseWhenObj1.canonicalized**

Before Fix:CASE WHEN ACCESS_CHECK#0 THEN A END
After Fix: CASE WHEN none#0 THEN A END
**caseWhenObj2.canonicalized**

Before Fix:CASE WHEN access_check#0 THEN A END
After Fix: CASE WHEN none#0 THEN A END

## How was this patch tested?
Added UT

Closes #24766 from sandeep-katta/caseWhenIssue.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-10 00:33:47 -07:00
Josh Rosen 95a9212db7 [SPARK-27846][CORE] Eagerly compute Configuration.properties in sc.hadoopConfiguration
## What changes were proposed in this pull request?

Hadoop `Configuration` has an internal `properties` map which is lazily initialized. Initialization of this field, done in the private `Configuration.getProps()` method, is rather expensive because it ends up parsing XML configuration files. When cloning a `Configuration`, this `properties` field is cloned if it has been initialized.

In some cases it's possible that `sc.hadoopConfiguration` never ends up computing this `properties` field, leading to performance problems when this configuration is cloned in `SessionState.newHadoopConf()` because each cloned `Configuration` needs to re-parse configuration XML files from disk.

To avoid this problem, we can call `Configuration.size()` to trigger a call to `getProps()`, ensuring that this expensive computation is cached and re-used when cloning configurations.

I discovered this problem while performance profiling the Spark ThriftServer while running a SQL fuzzing workload.

## How was this patch tested?

Examined YourKit profiles before and after my change.

Closes #24714 from JoshRosen/fuzzing-perf-improvements.

Authored-by: Josh Rosen <rosenville@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-09 17:58:24 -07:00
Stavros Kontopoulos 7912ab85a6 [SPARK-27872][K8S] Fix executor service account inconsistency
## What changes were proposed in this pull request?

Fixes the service account inconsistency that breaks pull secrets. It gives the option to the user to setup a specific service account for the executors if he has to
(via `spark.kubernetes.authenticate.executor.serviceAccountName`). Defaults to the driver's one.
We are not supporting special authentication credentials for the executors with this PR.

## How was this patch tested?

Tested manually by launching a Spark job exercising the introduced settings.
Added a new integration tests for this fix.

Closes #24748 from skonto/fix_executor_sa.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-09 16:28:37 -05:00
Dongjoon Hyun 742f805177 Revert "[SPARK-27979][BUILD][test-maven] Remove deprecated --force option in build/mvn and run-tests.py"
This reverts commit 354ec254c5.
2019-06-09 08:33:21 -07:00
HyukjinKwon f984f6acfe Revert "[SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)"
## What changes were proposed in this pull request?

This PR reverts 9c4eb99c52 for the reasons below:

1. An alternative was not considered properly, https://github.com/apache/spark/pull/24734#issuecomment-500101639 https://github.com/apache/spark/pull/24734#issuecomment-500102340 https://github.com/apache/spark/pull/24734#issuecomment-499202982 - I opened a PR https://github.com/apache/spark/pull/24826

2. 9c4eb99c52 fixed timely flushing which behaviour is somewhat hacky and the timing isn't also guaranteed (in case each batch takes longer to process).

3. For pipelining for smaller batches, looks it's better to allow to configure buffer size rather than having another factor to flush

## How was this patch tested?

N/A

Closes #24827 from HyukjinKwon/revert-flush.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-09 08:28:31 -07:00
Martin Junghanns 709387d660 [SPARK-27300][GRAPH] Add Spark Graph modules and dependencies
## What changes were proposed in this pull request?

This PR introduces the necessary Maven modules for the new [Spark Graph](https://issues.apache.org/jira/browse/SPARK-25994) feature for Spark 3.0.

* `spark-graph` is a parent module that users depend on to get all graph functionalities (Cypher and Graph Algorithms)
* `spark-graph-api` defines the [Property Graph API](https://docs.google.com/document/d/1Wxzghj0PvpOVu7XD1iA8uonRYhexwn18utdcTxtkxlI) that is being shared between Cypher and Algorithms
* `spark-cypher` contains a Cypher query engine implementation

Both, `spark-graph-api` and `spark-cypher` depend on Spark SQL.

Note, that the Maven module for Graph Algorithms is not part of this PR and will be introduced in https://issues.apache.org/jira/browse/SPARK-27302

A PoC for a running Cypher implementation can be found in this WIP PR https://github.com/apache/spark/pull/24297

## How was this patch tested?

Pass the Jenkins with all profiles and manually build and check the followings.
```
$ ls assembly/target/scala-2.12/jars/spark-cypher*
assembly/target/scala-2.12/jars/spark-cypher_2.12-3.0.0-SNAPSHOT.jar

$ ls assembly/target/scala-2.12/jars/spark-graph* | grep -v graphx
assembly/target/scala-2.12/jars/spark-graph-api_2.12-3.0.0-SNAPSHOT.jar
assembly/target/scala-2.12/jars/spark-graph_2.12-3.0.0-SNAPSHOT.jar
```

Closes #24490 from s1ck/SPARK-27300.

Lead-authored-by: Martin Junghanns <martin.junghanns@neotechnology.com>
Co-authored-by: Max Kießling <max@kopfueber.org>
Co-authored-by: Martin Junghanns <martin.junghanns@neo4j.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-09 00:26:26 -07:00
HyukjinKwon 6dcf09becc [SPARK-27971][SQL][R] MapPartitionsInRWithArrowExec.evaluate shouldn't eagerly read the first batch
## What changes were proposed in this pull request?

This PR is the same fix as https://github.com/apache/spark/pull/24816 but in vectorized `dapply` in SparkR.

## How was this patch tested?

Manually tested.

Closes #24818 from HyukjinKwon/SPARK-27971.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-09 11:40:20 +09:00
Dongjoon Hyun e561e92765 [SPARK-27981][CORE] Remove Illegal reflective access warning for java.nio.Bits.unaligned() in JDK9+
## What changes were proposed in this pull request?

This PR aims to remove the following warnings for `java.nio.Bits.unaligned` at JDK9/10/11/12. Please note that there are more warnings which is beyond of this PR's scope. JDK9+ shows the first warning only if you don't give `--illegal-access=warn`.

**BEFORE (Among 5 warnings, there is `java.nio.Bits.unaligned` warning at the startup)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to method java.nio.Bits.unaligned()
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getInstance()
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getDefaultRealm()
19/06/08 11:01:19 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560016882712).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
```

**AFTER (Among 4 warnings, there is no `java.nio.Bits.unaligned` warning with `hadoop-2.7` profile)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getInstance()
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getDefaultRealm()
19/06/08 11:08:27 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560017311171).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
```

**AFTER (Among 2 warnings, there is no `java.nio.Bits.unaligned` warning with `hadoop-3.2` profile)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
19/06/08 10:52:06 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560016330287).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
...
```

## How was this patch tested?

Manual. Run Spark command like `spark-shell` with `--driver-java-options=--illegal-access=warn` option in JDK9/10/11/12 environment.

Closes #24825 from dongjoon-hyun/SPARK-27981.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-08 16:39:32 -07:00
Gengliang Wang db0f6b4674 [SPARK-27961][SQL] DataSourceV2Relation should not have refresh method
## What changes were proposed in this pull request?

The newly added `Refresh` method in PR #24401 prevented the work of moving DataSourceV2Relation into catalyst. It calls `case table: FileTable => table.fileIndex.refresh()` while `FileTable` belongs to sql/core.

More importantly, Ryan Blue pointed out DataSourceV2Relation is immutable by design, it should not have refresh method.

## How was this patch tested?

Unit test

Closes #24815 from gengliangwang/removeRefreshTable.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-08 10:59:10 -07:00
Dongjoon Hyun 354ec254c5 [SPARK-27979][BUILD][test-maven] Remove deprecated --force option in build/mvn and run-tests.py
## What changes were proposed in this pull request?

Since Apache Spark 2.0.0, SPARK-14867 deprecated `--force` option and made it ignored. This PR cleans up the related code completely at 3.0.0.

**BEFORE (Jenkins)**
```
========================================================================
Building Spark
========================================================================
[info] Building Spark using Maven with these arguments:  -Phadoop-2.7 -Pkubernetes -Phive-thriftserver -Pkinesis-asl -Pyarn -Pspark-ganglia-lgpl -Phive -Pmesos clean package -DskipTests
WARNING: '--force' is deprecated and ignored.
...
========================================================================
Running Spark unit tests
========================================================================
[info] Running Spark tests using Maven with these arguments:  -Phadoop-2.7 -Phive-thriftserver -Phive -Dtest.exclude.tags=org.apache.spark.tags.ExtendedHiveTest,org.apache.spark.tags.ExtendedYarnTest test --fail-at-end
WARNING: '--force' is deprecated and ignored.
```

**AFTER (Jenkins)**
```
========================================================================
Building Spark
========================================================================
[info] Building Spark using Maven with these arguments:  -Phadoop-2.7 -Pkubernetes -Phive-thriftserver -Pkinesis-asl -Pyarn -Pspark-ganglia-lgpl -Phive -Pmesos clean package -DskipTests
...
========================================================================
Running Spark unit tests
========================================================================
[info] Running Spark tests using Maven with these arguments:  -Phadoop-2.7 -Pkubernetes -Phive-thriftserver -Pyarn -Pspark-ganglia-lgpl -Phive -Pkinesis-asl -Pmesos -Dtest.exclude.tags=org.apache.spark.tags.ExtendedHiveTest,org.apache.spark.tags.ExtendedYarnTest test --fail-at-end
```

## How was this patch tested?

Manually check the Jenkins logs.

Closes #24824 from dongjoon-hyun/SPARK-27979.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-08 08:17:12 -07:00
Yuming Wang 2926890ffb [SPARK-27970][SQL] Support Hive 3.0 metastore
## What changes were proposed in this pull request?

It seems that some users are using Hive 3.0.0. This pr makes it support Hive 3.0 metastore.

## How was this patch tested?

unit tests

Closes #24688 from wangyum/SPARK-26145.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 15:24:07 -07:00
WeichenXu 9c4eb99c52 [SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)
## What changes were proposed in this pull request?

Flush batch timely for pandas UDF.

This could improve performance when multiple pandas UDF plans are pipelined.

When batch being flushed in time, downstream pandas UDFs will get pipelined as soon as possible, and pipeline will help hide the donwstream UDFs computation time. For example:

When the first UDF start computing on batch-3, the second pipelined UDF can start computing on batch-2, and the third pipelined UDF can start computing on batch-1.

If we do not flush each batch in time, the donwstream UDF's pipeline will lag behind too much, which may increase the total processing time.

I add flush at two places:
* JVM process feed data into python worker. In jvm side, when write one batch, flush it
* VM process read data from python worker output, In python worker side, when write one batch, flush it

If no flush, the default buffer size for them are both 65536. Especially in the ML case, in order to make realtime prediction, we will make batch size very small. The buffer size is too large for the case, which cause downstream pandas UDF pipeline lag behind too much.

### Note
* This is only applied to pandas scalar UDF.
* Do not flush for each batch. The minimum interval between two flush is 0.1 second. This avoid too frequent flushing when batch size is small. It works like:
```
        last_flush_time = time.time()
        for batch in iterator:
                writer.write_batch(batch)
                flush_time = time.time()
                if self.flush_timely and (flush_time - last_flush_time > 0.1):
                      stream.flush()
                      last_flush_time = flush_time
```

## How was this patch tested?

### Benchmark to make sure the flush do not cause performance regression
#### Test code:
```
numRows = ...
batchSize = ...

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', str(batchSize))
df = spark.range(1, numRows + 1, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    return x + 10

beg_time = time.time()
result = df.select(sum(fp1('a'))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```
#### Test Result:

 params        | Consume time (Before) | Consume time (After)
------------ | ----------------------- | ----------------------
numRows=100000000, batchSize=10000 | 23.43s | 24.64s
numRows=100000000, batchSize=1000 | 36.73s | 34.50s
numRows=10000000, batchSize=100 | 35.67s | 32.64s
numRows=1000000, batchSize=10 | 33.60s | 32.11s
numRows=100000, batchSize=1 | 33.36s | 31.82s

### Benchmark pipelined pandas UDF
#### Test code:
```
spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))

```
#### Test Result:

**Before**: consume time: 63.57s
**After**: consume time: 32.43s
**So the PR improve performance by make downstream UDF get pipelined early.**

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

Closes #24734 from WeichenXu123/improve_pandas_udf_pipeline.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 14:02:43 -07:00
Liang-Chi Hsieh 527d936049 [SPARK-27798][SQL] from_avro shouldn't produces same value when converted to local relation
## What changes were proposed in this pull request?

When using `from_avro` to deserialize avro data to catalyst StructType format, if `ConvertToLocalRelation` is applied at the time, `from_avro` produces only the last value (overriding previous values).

The cause is `AvroDeserializer` reuses output row for StructType. Normally, it should be fine in Spark SQL. But `ConvertToLocalRelation` just uses `InterpretedProjection` to project local rows. `InterpretedProjection` creates new row for each output thro, it includes the same nested row object from `AvroDeserializer`. By the end, converted local relation has only last value.

I think there're two possible options:

1. Make `AvroDeserializer` output new row for StructType.
2. Use `InterpretedMutableProjection` in `ConvertToLocalRelation` and call `copy()` on output rows.

Option 2 is chose because previously `ConvertToLocalRelation` also creates new rows, this `InterpretedMutableProjection` + `copy()` shoudn't bring too much performance penalty. `ConvertToLocalRelation` should be arguably less critical, compared with `AvroDeserializer`.

## How was this patch tested?

Added test.

Closes #24805 from viirya/SPARK-27798.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-07 13:47:36 -07:00
Yuexin Zhang 5cdc506848 [SPARK-27973][MINOR] [EXAMPLES]correct DirectKafkaWordCount usage text with groupId
## What changes were proposed in this pull request?

Usage: DirectKafkaWordCount <brokers> <topics>
--
<brokers> is a list of one or more Kafka brokers
<groupId> is a consumer group name to consume from topics
<topics> is a list of one or more kafka topics to consume from

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

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

Closes #24819 from cnZach/minor_DirectKafkaWordCount_UsageWithGroupId.

Authored-by: Yuexin Zhang <zach.yx.zhang@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-07 08:02:02 -05:00
Ryan Blue b30655bdef [SPARK-27965][SQL] Add extractors for v2 catalog transforms.
## What changes were proposed in this pull request?

Add extractors for v2 catalog transforms.

These extractors are used to match transforms that are equivalent to Spark's internal case classes. This makes it easier to work with v2 transforms.

## How was this patch tested?

Added test suite for the new extractors.

Closes #24812 from rdblue/SPARK-27965-add-transform-extractors.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 00:20:36 -07:00
liwensun eee3467b1e [SPARK-27938][SQL] Remove feature flag LEGACY_PASS_PARTITION_BY_AS_OPTIONS
## What changes were proposed in this pull request?
In PR https://github.com/apache/spark/pull/24365, we pass in the partitionBy columns as options in `DataFrameWriter`.  To make this change less intrusive for a patch release, we added a feature flag `LEGACY_PASS_PARTITION_BY_AS_OPTIONS` with the default to be false.

For 3.0, we should just do the correct behavior for DSV1, i.e., always passing partitionBy as options, and remove this legacy feature flag.

## How was this patch tested?
Existing tests.

Closes #24784 from liwensun/SPARK-27453-default.

Authored-by: liwensun <liwen.sun@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-07 11:33:58 +09:00
Xiangrui Meng 4d770db0eb [SPARK-27968] ArrowEvalPythonExec.evaluate shouldn't eagerly read the first row
## What changes were proposed in this pull request?

Issued fixed in https://github.com/apache/spark/pull/24734 but that PR might takes longer to merge.

## How was this patch tested?

It should pass existing unit tests.

Closes #24816 from mengxr/SPARK-27968.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-06 15:45:44 -07:00
Thomas Graves d30284b5a5 [SPARK-27760][CORE] Spark resources - change user resource config from .count to .amount
## What changes were proposed in this pull request?

Change the resource config spark.{executor/driver}.resource.{resourceName}.count to .amount to allow future usage of containing both a count and a unit.  Right now we only support counts - # of gpus for instance, but in the future we may want to support units for things like memory - 25G. I think making the user only have to specify a single config .amount is better then making them specify 2 separate configs of a .count and then a .unit.  Change it now since its a user facing config.

Amount also matches how the spark on yarn configs are setup.

## How was this patch tested?

Unit tests and manually verified on yarn and local cluster mode

Closes #24810 from tgravescs/SPARK-27760-amount.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-06-06 14:16:05 -05:00
Yuming Wang eadb53824d [SPARK-27918][SQL] Port boolean.sql
## What changes were proposed in this pull request?

This PR is to port boolean.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/boolean.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/expected/boolean.out

When porting the test cases, found two PostgreSQL specific features that do not exist in Spark SQL:
- [SPARK-27931](https://issues.apache.org/jira/browse/SPARK-27931): Accept 'on' and 'off' as input for boolean data type / Trim the string when cast to boolean type / Accept unique prefixes thereof
- [SPARK-27924](https://issues.apache.org/jira/browse/SPARK-27924): Support E061-14: Search Conditions

Also, found an inconsistent behavior:
- [SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Unsupported input throws an exception in PostgreSQL but Spark accepts it and sets the value to `NULL`, for example:
```sql
SELECT bool 'test' AS error; -- SELECT boolean('test') AS error;
```

## How was this patch tested?

N/A

Closes #24767 from wangyum/SPARK-27918.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-06 10:57:10 -07:00
Yuming Wang 4de96493ae [SPARK-27883][SQL] Port AGGREGATES.sql [Part 2]
## What changes were proposed in this pull request?

This PR is to port AGGREGATES.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/aggregates.sql#L145-L350

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/expected/aggregates.out#L499-L984

When porting the test cases, found four PostgreSQL specific features that do not exist in Spark SQL:

- [SPARK-27877](https://issues.apache.org/jira/browse/SPARK-27877): Implement SQL-standard LATERAL subqueries
- [SPARK-27878](https://issues.apache.org/jira/browse/SPARK-27878): Support ARRAY(sub-SELECT) expressions
- [SPARK-27879](https://issues.apache.org/jira/browse/SPARK-27879): Implement bitwise integer aggregates(BIT_AND and BIT_OR)
- [SPARK-27880](https://issues.apache.org/jira/browse/SPARK-27880): Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY)

## How was this patch tested?

N/A

Closes #24743 from wangyum/SPARK-27883.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-06 09:28:59 -07:00
Ryan Blue d1371a2dad [SPARK-27964][SQL] Move v2 catalog update methods to CatalogV2Util
## What changes were proposed in this pull request?

Move methods that implement v2 catalog operations to CatalogV2Util so they can be used in #24768.

## How was this patch tested?

Behavior is validated by existing tests.

Closes #24813 from rdblue/SPARK-27964-add-catalog-v2-util.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-05 19:44:53 -07:00
Jordan Sanders 20e8843350 [MINOR][SQL] Skip warning if JOB_SUMMARY_LEVEL is set to NONE
## What changes were proposed in this pull request?

I believe the log message: `Committer $committerClass is not a ParquetOutputCommitter and cannot create job summaries. Set Parquet option ${ParquetOutputFormat.JOB_SUMMARY_LEVEL} to NONE.` is at odds with the `if` statement that logs the warning. Despite the instructions in the warning, users still encounter the warning if `JOB_SUMMARY_LEVEL` is already set to `NONE`.

This pull request introduces a change to skip logging the warning if `JOB_SUMMARY_LEVEL` is set to `NONE`.

## How was this patch tested?

I built to make sure everything still compiled and I ran the existing test suite. I didn't feel it was worth the overhead to add a test to make sure a log message does not get logged, but if reviewers feel differently, I can add one.

Closes #24808 from jmsanders/master.

Authored-by: Jordan Sanders <jmsanders@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-05 14:57:36 -07:00
Ryan Blue 5d6758c0e7 [SPARK-27857][SQL] Move ALTER TABLE parsing into Catalyst
## What changes were proposed in this pull request?

This moves parsing logic for `ALTER TABLE` into Catalyst and adds parsed logical plans for alter table changes that use multi-part identifiers. This PR is similar to SPARK-27108, PR #24029, that created parsed logical plans for create and CTAS.

* Create parsed logical plans
* Move parsing logic into Catalyst's AstBuilder
* Convert to DataSource plans in DataSourceResolution
* Parse `ALTER TABLE ... SET LOCATION ...` separately from the partition variant
* Parse `ALTER TABLE ... ALTER COLUMN ... [TYPE dataType] [COMMENT comment]` [as discussed on the dev list](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Syntax-for-table-DDL-td25197.html#a25270)
* Parse `ALTER TABLE ... RENAME COLUMN ... TO ...`
* Parse `ALTER TABLE ... DROP COLUMNS ...`

## How was this patch tested?

* Added new tests in Catalyst's `DDLParserSuite`
* Moved converted plan tests from SQL `DDLParserSuite` to `PlanResolutionSuite`
* Existing tests for regressions

Closes #24723 from rdblue/SPARK-27857-add-alter-table-statements-in-catalyst.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 13:21:30 -07:00
Jacek Laskowski 6c28ef144d [SPARK-27933][SS] Extracting common purge behaviour to the parent StreamExecution
Extracting the common purge "behaviour" to the parent StreamExecution.

## How was this patch tested?

No added behaviour so relying on existing tests.

Closes #24781 from jaceklaskowski/StreamExecution-purge.

Authored-by: Jacek Laskowski <jacek@japila.pl>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-05 12:39:31 -05:00
Wenchen Fan 8b6232b119 [SPARK-27521][SQL] Move data source v2 to catalyst module
## What changes were proposed in this pull request?

Currently we are in a strange status that, some data source v2 interfaces(catalog related) are in sql/catalyst, some data source v2 interfaces(Table, ScanBuilder, DataReader, etc.) are in sql/core.

I don't see a reason to keep data source v2 API in 2 modules. If we should pick one module, I think sql/catalyst is the one to go.

Catalyst module already has some user-facing stuff like DataType, Row, etc. And we have to update `Analyzer` and `SessionCatalog` to support the new catalog plugin, which needs to be in the catalyst module.

This PR can solve the problem we have in https://github.com/apache/spark/pull/24246

## How was this patch tested?

existing tests

Closes #24416 from cloud-fan/move.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 09:55:55 -07:00
Yuming Wang 3f102a8229 [SPARK-27749][SQL] hadoop-3.2 support hive-thriftserver
## What changes were proposed in this pull request?

This PR mainly makes the following changes to make `hadoop-3.2` support `sql/hive-thriftserver`:
1. Upgrade [`TCLIService.thrift`](https://github.com/apache/hive/blob/rel/release-2.3.5/service-rpc/if/TCLIService.thrift) and related code to Hive 2.3.5 because of [HIVE-12442](https://issues.apache.org/jira/browse/HIVE-12442)(Note that we only migrate code without adding features, such as [HIVE-4924](https://issues.apache.org/jira/browse/HIVE-4924) and [HIVE-15473](https://issues.apache.org/jira/browse/HIVE-15473)).
2. Use slf4j as logging facade because of [HIVE-12237](https://issues.apache.org/jira/browse/HIVE-12237).
3. Port [HIVE-13169](https://issues.apache.org/jira/browse/HIVE-13169) to compatible with Hive 2.3.

## How was this patch tested?

Exiting test

Closes #24628 from wangyum/SPARK-27749.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 08:40:05 -07:00
LantaoJin 18834e85d0 [SPARK-27899][SQL] Refactor getTableOption() to extract a common method
## What changes were proposed in this pull request?

This is a part of #24774, to reduce the code changes made by that.

## How was this patch tested?

Exist UTs.

Closes #24803 from LantaoJin/SPARK-27899_refactor.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 08:36:25 -07:00
Marcelo Vanzin b312033bd3 [SPARK-20286][CORE] Improve logic for timing out executors in dynamic allocation.
This change refactors the portions of the ExecutorAllocationManager class that
track executor state into a new class, to achieve a few goals:

- make the code easier to understand
- better separate concerns (task backlog vs. executor state)
- less synchronization between event and allocation threads
- less coupling between the allocation code and executor state tracking

The executor tracking code was moved to a new class (ExecutorMonitor) that
encapsulates all the logic of tracking what happens to executors and when
they can be timed out. The logic to actually remove the executors remains
in the EAM, since it still requires information that is not tracked by the
new executor monitor code.

In the executor monitor itself, of interest, specifically, is a change in
how cached blocks are tracked; instead of polling the block manager, the
monitor now uses events to track which executors have cached blocks, and
is able to detect also unpersist events and adjust the time when the executor
should be removed accordingly. (That's the bug mentioned in the PR title.)

Because of the refactoring, a few tests in the old EAM test suite were removed,
since they're now covered by the newly added test suite. The EAM suite was
also changed a little bit to not instantiate a SparkContext every time. This
allowed some cleanup, and the tests also run faster.

Tested with new and updated unit tests, and with multiple TPC-DS workloads
running with dynamic allocation on; also some manual tests for the caching
behavior.

Closes #24704 from vanzin/SPARK-20286.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-06-05 08:09:44 -05:00
Xingbo Jiang fcb3fb04c5 [SPARK-27948][CORE][TEST] Use ResourceName to represent resource names
## What changes were proposed in this pull request?

Use objects in `ResourceName` to represent resource names.

## How was this patch tested?

Existing tests.

Closes #24799 from jiangxb1987/ResourceName.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-04 19:59:07 -07:00
Xingbo Jiang ac808e2a02 [SPARK-27366][CORE] Support GPU Resources in Spark job scheduling
## What changes were proposed in this pull request?

This PR adds support to schedule tasks with extra resource requirements (eg. GPUs) on executors with available resources. It also introduce a new method `TaskContext.resources()` so tasks can access available resource addresses allocated to them.

## How was this patch tested?

* Added new end-to-end test cases in `SparkContextSuite`;
* Added new test case in `CoarseGrainedSchedulerBackendSuite`;
* Added new test case in `CoarseGrainedExecutorBackendSuite`;
* Added new test case in `TaskSchedulerImplSuite`;
* Added new test case in `TaskSetManagerSuite`;
* Updated existing tests.

Closes #24374 from jiangxb1987/gpu.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-04 16:57:47 -07:00
Jules Damji b71abd654d [MINOR][DOC] Avro data source documentation change
## What changes were proposed in this pull request?

This is a minor documentation change whereby the https://spark.apache.org/docs/latest/sql-data-sources-avro.html mentions "The date type and naming of record fields should match the input Avro data or Catalyst data,"

The term Catalyst data is confusing. It should instead say, Spark's internal data type such as String
Type or IntegerType.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
There are no code changes; only doc changes.
Please review https://spark.apache.org/contributing.html before opening a pull request.

Closes #24787 from dmatrix/br-orc-ds.doc.changes.

Authored-by: Jules Damji <dmatrix@comcast.net>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-04 16:17:53 -07:00
Ryan Blue de73a54269 [SPARK-27909][SQL] Do not run analysis inside CTE substitution
## What changes were proposed in this pull request?

This updates CTE substitution to avoid needing to run all resolution rules on each substituted expression. Running resolution rules was previously used to avoid infinite recursion. In the updated rule, CTE plans are substituted as sub-queries from right to left. Using this scope-based order, it is not necessary to replace multiple CTEs at the same time using `resolveOperatorsDown`. Instead, `resolveOperatorsUp` is used to replace each CTE individually.

By resolving using `resolveOperatorsUp`, this no longer needs to run all analyzer rules on each substituted expression. Previously, this was done to apply `ResolveRelations`, which would throw an `AnalysisException` for all unresolved relations so that unresolved relations that may cause recursive substitutions were not left in the plan. Because this is no longer needed, `ResolveRelations` no longer needs to throw `AnalysisException` and resolution can be done in multiple rules.

## How was this patch tested?

Existing tests in `SQLQueryTestSuite`, `cte.sql`.

Closes #24763 from rdblue/SPARK-27909-fix-cte-substitution.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-04 14:46:13 -07:00
David Vogelbacher f9ca8ab196 [SPARK-27805][PYTHON] Propagate SparkExceptions during toPandas with arrow enabled
## What changes were proposed in this pull request?
Similar to https://github.com/apache/spark/pull/24070, we now propagate SparkExceptions that are encountered during the collect in the java process to the python process.

Fixes https://jira.apache.org/jira/browse/SPARK-27805

## How was this patch tested?
Added a new unit test

Closes #24677 from dvogelbacher/dv/betterErrorMsgWhenUsingArrow.

Authored-by: David Vogelbacher <dvogelbacher@palantir.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-04 10:10:27 -07:00
Luca Canali adf72e26d9 [SPARK-27773][FOLLOWUP][DOC] Add numCaughtExceptions metric to monitoring doc
## What changes were proposed in this pull request?

SPARK-27773 has introduced a new metric (counter) numCaughtExceptions to the Spark Dropwizard monitoring system. This PR adds an entry in the monitoring documentation to document this.

Closes #24790 from LucaCanali/addDocFollowingSPARK27773.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-04 08:40:32 -07:00
HyukjinKwon d1f3c994c7 [SPARK-27942][DOCS][PYTHON] Note that Python 2.7 is deprecated in Spark documentation
## What changes were proposed in this pull request?

This PR adds deprecation notes in Spark documentation.

## How was this patch tested?

git grep -r "python 2.6"
git grep -r "python 2.6"
git grep -r "python 2.7"
git grep -r "python 2.7"

Closes #24789 from HyukjinKwon/SPARK-27942.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-04 07:59:25 -07:00
williamwong d5715a9b23 [SPARK-27772][SQL][TEST] Refactor SQLTestUtils to use tryWithSafeFinally
## What changes were proposed in this pull request?

The current `SQLTestUtils` created many `withXXX` utility functions to clean up tables/views/caches created for testing purpose. Java's `try-with-resources` statement does something similar, but it does not mask exception throwing in the try block with any exception caught in the 'close()' statement. Exception caught in the 'close()' statement would add as a suppressed exception instead.

This PR standardizes those 'withXXX' function to use`Utils.tryWithSafeFinally` function, which does something similar to Java's try-with-resources statement. The purpose of this proposal is to help developers to identify what actually breaks their tests.

## How was this patch tested?
Existing testcases.

Closes #24747 from William1104/feature/SPARK-27772-2.

Lead-authored-by: williamwong <william1104@gmail.com>
Co-authored-by: William Wong <william1104@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-04 09:26:24 -05:00
ozan a38d605d0d [SPARK-18570][ML][R] RFormula support * and ^ operators
## What changes were proposed in this pull request?

Added support for `*` and `^` operators, along with expressions within parentheses. New operators just expand to already supported terms, such as;

 - y ~ a * b = y ~ a + b + a : b
 - y ~ (a+b+c)^3 = y ~ a + b + c + a : b + a : c + a :b : c

## How was this patch tested?

Added new unit tests to RFormulaParserSuite

mengxr yanboliang

Closes #24764 from ozancicek/rformula.

Authored-by: ozan <ozancancicekci@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-04 08:59:30 -05:00
Michael Chirico 3ddc26ddd8 [MINOR][DOCS] Add a clarifying note to str_to_map documentation
I was quite surprised by the following behavior:

`SELECT str_to_map('1:2|3:4', '|')`

vs

`SELECT str_to_map(replace('1:2|3:4', '|', ','))`

The documentation does not make clear at all what's going on here, but a [dive into the source code shows](fa0d4bf699/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala (L461-L466)) that `split` is being used and in turn the interpretation of `split`'s arguments as RegEx is clearly documented.

## What changes were proposed in this pull request?

Documentation clarification

## How was this patch tested?

N/A

Closes #23888 from MichaelChirico/patch-2.

Authored-by: Michael Chirico <michaelchirico4@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-04 16:58:25 +09:00
Gengliang Wang d1937c1479 [SPARK-27926][SQL] Allow altering table add columns with CSVFileFormat/JsonFileFormat provider
## What changes were proposed in this pull request?

In the previous work of csv/json migration, CSVFileFormat/JsonFileFormat is removed in the table provider whitelist of `AlterTableAddColumnsCommand.verifyAlterTableAddColumn`:
https://github.com/apache/spark/pull/24005
https://github.com/apache/spark/pull/24058

This is regression. If a table is created with Provider `org.apache.spark.sql.execution.datasources.csv.CSVFileFormat` or `org.apache.spark.sql.execution.datasources.json.JsonFileFormat`, Spark should allow the "alter table add column" operation.

## How was this patch tested?

Unit test

Closes #24776 from gengliangwang/v1Table.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-03 23:51:05 -07:00
Xiangrui Meng 216eb36560 [SPARK-27887][PYTHON] Add deprecation warning for Python 2
## What changes were proposed in this pull request?

Add deprecation warning for Python 2.

## How was this patch tested?

Manual tests:

Interactive shell:

~~~
$ bin/pyspark
Python 2.7.15 |Anaconda, Inc.| (default, Nov 13 2018, 17:07:45)
[GCC 4.2.1 Compatible Clang 4.0.1 (tags/RELEASE_401/final)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
19/06/03 14:54:13 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
/Users/meng/src/spark/python/pyspark/context.py:219: DeprecationWarning: Support for Python 2 is deprecated as of Spark 3.0. See the plan for dropping Python 2 support at https://spark.apache.org/news/plan-for-dropping-python-2-support.html.
  DeprecationWarning)
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /__ / .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Python version 2.7.15 (default, Nov 13 2018 17:07:45)
SparkSession available as 'spark'.
>>>
~~~

spark-submit job (with default log level set to WARN):

~~~
$ bin/spark-submit test.py
19/06/03 14:54:38 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
/Users/meng/src/spark/python/lib/pyspark.zip/pyspark/context.py:219: DeprecationWarning: Support for Python 2 is deprecated as of Spark 3.0. See the plan for dropping Python 2 support at https://spark.apache.org/news/plan-for-dropping-python-2-support.html.
  DeprecationWarning)
~~~

Verified that warning messages do not show up in Python 3.

`  DeprecationWarning)` is displayed at the end because `warn` by default print the code line. This behavior can be changed by monkey patching `showwarning` https://stackoverflow.com/questions/2187269/print-only-the-message-on-warnings. It might not worth the effort.

Closes #24786 from mengxr/SPARK-27887.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-04 15:36:52 +09:00
zhengruifeng 98708de38c [MINOR][ML] add missing since annotation of meanAveragePrecision
## What changes were proposed in this pull request?
add missing since annotation of meanAveragePrecision

## How was this patch tested?
existing tests

Closes #24778 from zhengruifeng/ranking_missing_since.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-03 18:07:23 -05:00
Gabor Somogyi 911fadf33a [SPARK-27748][SS] Kafka consumer/producer password/token redaction.
## What changes were proposed in this pull request?

Kafka parameters are logged at several places and the following parameters has to be redacted:
* Delegation token
* `ssl.truststore.password`
* `ssl.keystore.password`
* `ssl.key.password`

This PR contains:
* Spark central redaction framework used to redact passwords (`spark.redaction.regex`)
* Custom redaction added to handle `sasl.jaas.config` (delegation token)
* Redaction code added into consumer/producer code
* Test refactor

## How was this patch tested?

Existing + additional unit tests.

Closes #24627 from gaborgsomogyi/SPARK-27748.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-06-03 15:43:08 -07:00
Dongjoon Hyun 8486680b34 [SPARK-24544][SQL][FOLLOWUP] Remove a wrong warning on Hive fallback lookup
## What changes were proposed in this pull request?

This PR is a follow-up of https://github.com/apache/spark/pull/21790 which causes a regression to show misleading warnings always at first invocation for all Hive function. Hive fallback lookup should not be warned. It's a normal process in function lookups.

**CURRENT (Showing `NoSuchFunctionException` and working)**
```scala
scala> sql("select histogram_numeric(a,2) from values(1) T(a)").show
19/06/02 22:02:10 WARN HiveSessionCatalog: Encountered a failure during looking up
function: org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException:
Undefined function: 'histogram_numeric'. This function is neither a registered temporary
function nor a permanent function registered in the database 'default'.;
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.failFunctionLookup(SessionCatalog.scala:1234)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupFunction(SessionCatalog.scala:1302)
...
+------------------------+
|histogram_numeric( a, 2)|
+------------------------+
|            [[1.0, 1.0]]|
+------------------------+
```

## How was this patch tested?

Manually execute the above query.

Closes #24773 from dongjoon-hyun/SPARK-24544.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-03 00:04:00 -07:00
HyukjinKwon 8b18ef5c7b [MINOR] Avoid hardcoded py4j-0.10.8.1-src.zip in Scala
## What changes were proposed in this pull request?

This PR targets to deduplicate hardcoded `py4j-0.10.8.1-src.zip` in order to make py4j upgrade easier.

## How was this patch tested?

N/A

Closes #24770 from HyukjinKwon/minor-py4j-dedup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 21:23:17 -07:00
Dongjoon Hyun 809821a283 [SPARK-27920][SQL][TEST] Add interceptParseException test utility function
## What changes were proposed in this pull request?

This PR aims to add `interceptParseException` test utility function to `AnalysisTest` to reduce the duplications of `intercept` functions.

## How was this patch tested?

Pass the Jenkins with the updated test suites.

Closes #24769 from dongjoon-hyun/SPARK-27920.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 21:11:35 -07:00
Yuming Wang d53b61c311 [SPARK-27831][SQL][TEST] Move Hive test jars to maven dependency
## What changes were proposed in this pull request?

This pr moves Hive test jars(`hive-contrib-0.13.1.jar`, `hive-hcatalog-core-0.13.1.jar`, `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.5.jar`) to maven dependency.

## How was this patch tested?

Existing test

Please note that this pr need test with `maven` and `sbt`.

Closes #24751 from wangyum/SPARK-27831.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 20:23:08 -07:00