Commit graph

24712 commits

Author SHA1 Message Date
Yuming Wang ab6bb8fc1c [SPARK-28075][SQL] Enhance TRIM function
## What changes were proposed in this pull request?

The `TRIM` function accept these patterns:
```sql
TRIM(str)
TRIM(trimStr, str)
TRIM(BOTH trimStr FROM str)
TRIM(LEADING trimStr FROM str)
TRIM(TRAILING trimStr FROM str)
```
This pr add support other three patterns:
```sql
TRIM(BOTH FROM str)
TRIM(LEADING FROM str)
TRIM(TRAILING FROM str)
```

PostgreSQL, Vertica, MySQL, Teradata, Oracle and DB2 support these patterns. Hive, Presto and SQL Server does not support this feature.

**PostgreSQL**:
```sql
postgres=# select substr(version(), 0, 16), trim(BOTH from '    SparkSQL   '), trim(LEADING FROM '    SparkSQL   '), trim(TRAILING FROM '    SparkSQL   ');
     substr      |  btrim   |    ltrim    |    rtrim
-----------------+----------+-------------+--------------
 PostgreSQL 11.3 | SparkSQL | SparkSQL    |     SparkSQL
(1 row)
```
**Vertica**:
```
dbadmin=> select version(), trim(BOTH from '    SparkSQL   '), trim(LEADING FROM '    SparkSQL   '), trim(TRAILING FROM '    SparkSQL   ');
              version               |  btrim   |    ltrim    |    rtrim
------------------------------------+----------+-------------+--------------
 Vertica Analytic Database v9.1.1-0 | SparkSQL | SparkSQL    |     SparkSQL
(1 row)
```
**MySQL**:
```
mysql> select version(), trim(BOTH from '    SparkSQL   '), trim(LEADING FROM '    SparkSQL   '), trim(TRAILING FROM '    SparkSQL   ');
+-----------+-----------------------------------+--------------------------------------+---------------------------------------+
| version() | trim(BOTH from '    SparkSQL   ') | trim(LEADING FROM '    SparkSQL   ') | trim(TRAILING FROM '    SparkSQL   ') |
+-----------+-----------------------------------+--------------------------------------+---------------------------------------+
| 5.7.26    | SparkSQL                          | SparkSQL                             |     SparkSQL                          |
+-----------+-----------------------------------+--------------------------------------+---------------------------------------+
1 row in set (0.01 sec)
```
**Teradata**:
![image](https://user-images.githubusercontent.com/5399861/59587081-070bcd00-9117-11e9-8534-df547860b585.png)
**Oracle**:
![image](https://user-images.githubusercontent.com/5399861/59587003-cf048a00-9116-11e9-839e-90da9e5183e0.png)
**DB2**:
![image](https://user-images.githubusercontent.com/5399861/59587801-af6e6100-9118-11e9-80be-ee1f6bbbeceb.png)

## How was this patch tested?

unit tests

Closes #24891 from wangyum/SPARK-28075.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-18 12:26:10 +08:00
Xiangrui Meng 1b2448bc10 [SPARK-28056][PYTHON] add doc for SCALAR_ITER Pandas UDF
## What changes were proposed in this pull request?

Add docs for `SCALAR_ITER` Pandas UDF.

cc: WeichenXu123 HyukjinKwon

## How was this patch tested?

Tested example code manually.

Closes #24897 from mengxr/SPARK-28056.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-17 20:51:36 -07:00
wuyi bb17aec916 [SPARK-27666][CORE] Do not release lock while TaskContext already completed
## What changes were proposed in this pull request?

PythonRunner uses an asynchronous way, which produces elements in WriteThread but consumes elements in another thread, to execute task. When child operator, like take()/first(), does not consume all elements produced by WriteThread, task would finish before WriteThread and releases all locks on blocks. However, WriteThread would continue to produce elements by pulling elements from parent operator until it exhausts all elements. And at the time WriteThread exhausts all elements, it will try to release the corresponding block but hit a AssertionError since task has already released that lock previously.

#24542 previously fix this by catching AssertionError, so that we won't fail our executor.

However, when not using PySpark, issue still exists when user implements a custom RDD or task, which spawn a separate child thread to consume iterator from a cached parent RDD. Below is a demo which could easily reproduce the issue.

```
    val rdd0 = sc.parallelize(Range(0, 10), 1).cache()
    rdd0.collect()
    rdd0.mapPartitions { iter =>
      val t = new Thread(new Runnable {
        override def run(): Unit = {
          while(iter.hasNext) {
            println(iter.next())
            Thread.sleep(1000)
          }
        }
      })
      t.setDaemon(false)
      t.start()
      Iterator(0)
    }.collect()
    Thread.sleep(100000)
```

So, if we could prevent the separate thread from releasing lock on block when TaskContext has already completed, we won't hit this issue again.

## How was this patch tested?

Added in new unit test in RDDSuite.

Closes #24699 from Ngone51/SPARK-27666.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-18 10:15:44 +08:00
Bryan Cutler 90f80395af [SPARK-28041][PYTHON] Increase minimum supported Pandas to 0.23.2
## What changes were proposed in this pull request?

This increases the minimum supported version of Pandas to 0.23.2. Using a lower version will raise an error `Pandas >= 0.23.2 must be installed; however, your version was 0.XX`. Also, a workaround for using pyarrow with Pandas 0.19.2 was removed.

## How was this patch tested?

Existing Tests

Closes #24867 from BryanCutler/pyspark-increase-min-pandas-SPARK-28041.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-18 09:10:58 +09:00
Sean Owen 4576dfde19 [SPARK-28066][CORE] Optimize UTF8String.trim() for common case of no whitespace
## What changes were proposed in this pull request?

UTF8String.trim() allocates a new object even if the string has no whitespace, when it can just return itself. A simple check for this case makes the method about 3x faster in the common case.

## How was this patch tested?

Existing tests.

A rough benchmark of 90% strings without whitespace (at ends), and 10% that do have whitespace, suggests the average runtime goes from 20 ns to 6 ns.

Closes #24884 from srowen/SPARK-28066.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-17 08:49:11 -07:00
Mellacheruvu Sandeep b7b4452553 [SPARK-24898][DOC] Adding spark.checkpoint.compress to the docs
## What changes were proposed in this pull request?

Adding spark.checkpoint.compress configuration parameter to the documentation

![](https://user-images.githubusercontent.com/3538013/59580409-a7013080-90ee-11e9-9b2c-3d29015f597e.png)

## How was this patch tested?

Checked locally for jeykyll html docs. Also validated the html for any issues.

Closes #24883 from sandeepvja/SPARK-24898.

Authored-by: Mellacheruvu Sandeep <mellacheruvu.sandeep@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-16 22:54:08 -07:00
Dongjoon Hyun d6a479b1f8 [SPARK-28063][SQL] Replace deprecated .newInstance() in DSv2 Catalogs
## What changes were proposed in this pull request?

This PR aims to replace deprecated `.newInstance()` in DSv2 `Catalogs` and distinguish the plugin class errors more. According to the JDK11 build log, there is no other new instance.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-jdk-11-ubuntu-testing/978/consoleFull

SPARK-25984 removes all instances of the deprecated `.newInstance()` usages at Nov 10, 2018, but this was added at SPARK-24252 on March 8, 2019.

## How was this patch tested?

Pass the Jenkins with the updated test case.

Closes #24882 from dongjoon-hyun/SPARK-28063.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-16 19:58:02 -07:00
Takuya UESHIN 5ae1a6bf0d [SPARK-28052][SQL] Make ArrayExists follow the three-valued boolean logic.
## What changes were proposed in this pull request?

Currently `ArrayExists` always returns boolean values (if the arguments are not `null`), but it should follow the three-valued boolean logic:

- `true` if the predicate holds at least one `true`
- otherwise, `null` if the predicate holds `null`
- otherwise, `false`

This behavior change is made to match Postgres' equivalent function `ANY/SOME (array)`'s behavior: https://www.postgresql.org/docs/9.6/functions-comparisons.html#AEN21174

## How was this patch tested?

Modified tests and existing tests.

Closes #24873 from ueshin/issues/SPARK-28052/fix_exists.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-15 10:48:06 -07:00
WeichenXu 6d441dcdc6 [SPARK-26412][PYSPARK][SQL] Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series
## What changes were proposed in this pull request?

Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series.
Note the UDF input args will be always one iterator:
* if the udf take only column as input, the iterator's element will be pd.Series (corresponding to the column values batch)
* if the udf take multiple columns as inputs, the iterator's element will be a tuple composed of multiple `pd.Series`s, each one corresponding to the multiple columns as inputs (keep the same order). For example:
```
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def the_udf(iterator):
    for col1_batch, col2_batch in iterator:
        yield col1_batch + col2_batch

df.select(the_udf("col1", "col2"))
```
The udf above will add col1 and col2.

I haven't add unit tests, but manually tests show it works fine. So it is ready for first pass review.
We can test several typical cases:

```
from pyspark.sql import SparkSession
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.functions import udf
from pyspark.taskcontext import TaskContext

df = spark.createDataFrame([(1, 20), (3, 40)], ["a", "b"])

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi1(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi1: do init stuff, partitionId=" + str(pid))
    for batch in it:
        yield batch + 100
    print("DBG: fi1: do close stuff, partitionId=" + str(pid))

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi2(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi2: do init stuff, partitionId=" + str(pid))
    for batch in it:
        yield batch + 10000
    print("DBG: fi2: do close stuff, partitionId=" + str(pid))

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi3(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi3: do init stuff, partitionId=" + str(pid))
    for x, y in it:
        yield x + y * 10 + 100000
    print("DBG: fi3: do close stuff, partitionId=" + str(pid))

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

udf("int")
def fu1(x):
    return x + 10

# test select "pandas iter udf/pandas udf/sql udf" expressions at the same time.
# Note this case the `fi1("a"), fi2("b"), fi3("a", "b")` will generate only one plan,
# and `fu1("a")`, `fp1("a")` will generate another two separate plans.
df.select(fi1("a"), fi2("b"), fi3("a", "b"), fu1("a"), fp1("a")).show()

# test chain two pandas iter udf together
# Note this case `fi2(fi1("a"))` will generate only one plan
# Also note the init stuff/close stuff call order will be like:
# (debug output following)
#     DBG: fi2: do init stuff, partitionId=0
#     DBG: fi1: do init stuff, partitionId=0
#     DBG: fi1: do close stuff, partitionId=0
#     DBG: fi2: do close stuff, partitionId=0
df.select(fi2(fi1("a"))).show()

# test more complex chain
# Note this case `fi1("a"), fi2("a")` will generate one plan,
# and `fi3(fi1_output, fi2_output)` will generate another plan
df.select(fi3(fi1("a"), fi2("a"))).show()
```

## How was this patch tested?

To be added.

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

Closes #24643 from WeichenXu123/pandas_udf_iter.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-15 08:29:20 -07:00
Terry Kim a950570f91 [MINOR][CORE] Remove unused variables, unused imports, etc.
## What changes were proposed in this pull request?

- Remove unused variables.
- Remove unused imports.
- Change var to val in few places.

## How was this patch tested?

Unit tests.

Closes #24857 from imback82/unused_variable.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-15 09:42:22 -05:00
HyukjinKwon 26998b86c1 [SPARK-27870][SQL][PYTHON] Add a runtime buffer size configuration for Pandas UDFs
## What changes were proposed in this pull request?

This PR is an alternative approach for #24734.

This PR fixes two things:

1. Respects `spark.buffer.size` in Python workers.
2. Adds a runtime buffer size configuration for Pandas UDFs, `spark.sql.pandas.udf.buffer.size` (which falls back to `spark.buffer.size`.

## How was this patch tested?

Manually tested:

```python
import time
from pyspark.sql.functions import *

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))
```

```
consume time: 62.68265891075134
```

```python
import time
from pyspark.sql.functions import *

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
spark.conf.set('spark.sql.pandas.udf.buffer.size', '4')
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))
```

```
consume time: 34.00594782829285
```

Closes #24826 from HyukjinKwon/SPARK-27870.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-15 20:56:22 +09:00
Gengliang Wang 23ebd389b5 [SPARK-27418][SQL] Migrate Parquet to File Data Source V2
## What changes were proposed in this pull request?

 Migrate Parquet to File Data Source V2

## How was this patch tested?

Unit test

Closes #24327 from gengliangwang/parquetV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-15 20:52:50 +09:00
maryannxue c79f471d04 [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL
## What changes were proposed in this pull request?

Implemented a new SparkPlan that executes the query adaptively. It splits the query plan into independent stages and executes them in order according to their dependencies. The query stage materializes its output at the end. When one stage completes, the data statistics of the materialized output will be used to optimize the remainder of the query.

The adaptive mode is off by default, when turned on, user can see "AdaptiveSparkPlan" as the top node of a query or sub-query. The inner plan of "AdaptiveSparkPlan" is subject to change during query execution but becomes final once the execution is complete. Whether the inner plan is final is included in the EXPLAIN string. Below is an example of the EXPLAIN plan before and after execution:

Query:
```
SELECT * FROM testData JOIN testData2 ON key = a WHERE value = '1'
```

Before execution:
```
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=false)
+- SortMergeJoin [key#13], [a#23], Inner
   :- Sort [key#13 ASC NULLS FIRST], false, 0
   :  +- Exchange hashpartitioning(key#13, 5)
   :     +- Filter (isnotnull(value#14) AND (value#14 = 1))
   :        +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false) AS value#14]
   :           +- Scan[obj#12]
   +- Sort [a#23 ASC NULLS FIRST], false, 0
      +- Exchange hashpartitioning(a#23, 5)
         +- SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24]
            +- Scan[obj#22]
```

After execution:
```
== Physical Plan ==
AdaptiveSparkPlan(isFinalPlan=true)
+- *(1) BroadcastHashJoin [key#13], [a#23], Inner, BuildLeft
   :- BroadcastQueryStage 2
   :  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
   :     +- ShuffleQueryStage 0
   :        +- Exchange hashpartitioning(key#13, 5)
   :           +- *(1) Filter (isnotnull(value#14) AND (value#14 = 1))
   :              +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false) AS value#14]
   :                 +- Scan[obj#12]
   +- ShuffleQueryStage 1
      +- Exchange hashpartitioning(a#23, 5)
         +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24]
            +- Scan[obj#22]
```

Credit also goes to carsonwang and cloud-fan

## How was this patch tested?

Added new UT.

Closes #24706 from maryannxue/aqe.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: herman <herman@databricks.com>
2019-06-15 11:27:15 +02:00
Sean Owen 15462e1a8f [SPARK-28004][UI] Update jquery to 3.4.1
## What changes were proposed in this pull request?

We're using an old-ish jQuery, 1.12.4, and should probably update for Spark 3 to keep up in general, but also to keep up with CVEs. In fact, we know of at least one resolved in only 3.4.0+ (https://nvd.nist.gov/vuln/detail/CVE-2019-11358). They may not affect Spark, but, if the update isn't painful, maybe worthwhile in order to make future 3.x updates easier.

jQuery 1 -> 2 doesn't sound like a breaking change, as 2.0 is supposed to maintain compatibility with 1.9+ (https://blog.jquery.com/2013/04/18/jquery-2-0-released/)

2 -> 3 has breaking changes: https://jquery.com/upgrade-guide/3.0/. It's hard to evaluate each one, but the most likely area for problems is in ajax(). However, our usage of jQuery (and plugins) is pretty simple.

Update jquery to 3.4.1; update jquery blockUI and mustache to latest

## How was this patch tested?

Manual testing of docs build (except R docs), worker/master UI, spark application UI.
Note: this really doesn't guarantee it works, as our tests can't test javascript, and this is merely anecdotal testing, although I clicked about every link I could find. There's a risk this breaks a minor part of the UI; it does seem to work fine in the main.

Closes #24843 from srowen/SPARK-28004.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-14 22:19:20 -07:00
Peter Toth 9e6666bde1 [SPARK-28002][SQL] Support WITH clause column aliases
## What changes were proposed in this pull request?

This PR adds support of column aliasing in a CTE so this query becomes valid:
```
WITH t(x) AS (SELECT 1)
SELECT * FROM t WHERE x = 1
```
## How was this patch tested?

Added new UTs.

Closes #24842 from peter-toth/SPARK-28002.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-14 20:47:11 -07:00
Dongjoon Hyun fd8240d10c [SPARK-28051][INFRA] Exposing JIRA issue component types at GitHub PRs
## What changes were proposed in this pull request?

This PR aims to expose JIRA issue component types at GitHub PRs.

## How was this patch tested?

Manual.
```
$ export GITHUB_OAUTH_KEY=...
$ export JIRA_PASSWORD=...
$ export GITHUB_API_BASE='https://api.github.com/repos/your-id/spark'
$ dev/github_jira_sync.py
```

Please note that the existing script will raise the following exceptions if your repo has less than 100 PRs. This will be handled at #24874 .
```
Traceback (most recent call last):
  File "dev/github_jira_sync.py", line 139, in <module>
    jira_prs = get_jira_prs()
  File "dev/github_jira_sync.py", line 83, in get_jira_prs
    link_header = filter(lambda k: k.startswith("Link"), page.info().headers)[0]
IndexError: list index out of range
```
That is beyond the scope of this PR.

Closes #24871 from dongjoon-hyun/SPARK-28051.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-14 20:36:45 -07:00
Jungtaek Lim (HeartSaVioR) bd0a04baab [SPARK-26949][SS] Prevent 'purge' to remove needed batch files in CompactibleFileStreamLog
## What changes were proposed in this pull request?

This patch proposes making `purge` in `CompactibleFileStreamLog` to throw `UnsupportedOperationException` to prevent purging necessary batch files, as well as adding javadoc to document its behavior. Actually it would only break when latest compaction batch is requested to be purged, but caller wouldn't be aware of this so safer to just prevent it.

## How was this patch tested?

Added UT.

Closes #23850 from HeartSaVioR/SPARK-26949.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-14 20:34:18 -07:00
maryannxue d1951aa23b [SPARK-28057][SQL] Add method clone in catalyst TreeNode
## What changes were proposed in this pull request?

Implemented the `clone` method for `TreeNode` based on `mapChildren`.

## How was this patch tested?

Added new UT.

Closes #24876 from maryannxue/treenode-clone.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: herman <herman@databricks.com>
2019-06-15 00:40:55 +02:00
Sean Owen b508eab985 [SPARK-21882][CORE] OutputMetrics doesn't count written bytes correctly in the saveAsHadoopDataset function
## What changes were proposed in this pull request?

(Continuation of https://github.com/apache/spark/pull/19118 ; see for details)

## How was this patch tested?

Existing tests.

Closes #24863 from srowen/SPARK-21882.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-14 12:44:43 -05:00
Dongjoon Hyun 7533cccc5d [SPARK-28053][INFRA] Handle a corner case where there is no Link header
## What changes were proposed in this pull request?

Currently, `github_jira_sync.py` assumes that there is `Link` always. However, it will fail when the number of the open PR is less than 100 (the default paging number). It will not happen in Apache Spark, but we had better fix that because it happens during review process for `github_jira_sync.py` script.
```
Traceback (most recent call last):
  File "dev/github_jira_sync.py", line 139, in <module>
    jira_prs = get_jira_prs()
  File "dev/github_jira_sync.py", line 83, in get_jira_prs
    link_header = filter(lambda k: k.startswith("Link"), page.info().headers)[0]
IndexError: list index out of range
```

## How was this patch tested?

Manually check with another repo which has small number of open PRs (< 100).
```
$ export JIRA_PASSWORD=...
$ export GITHUB_API_BASE='https://api.github.com/repos/your-id/spark'
$ dev/github_jira_sync.py
```

Closes #24874 from dongjoon-hyun/SPARK-28053.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-14 16:33:34 +09:00
Liang-Chi Hsieh c0297dedd8 [MINOR][PYSPARK][SQL][DOC] Fix rowsBetween doc in Window
## What changes were proposed in this pull request?

I suspect that the doc of `rowsBetween` methods in Scala and PySpark looks wrong.
Because:

```scala
scala> val df = Seq((1, "a"), (2, "a"), (3, "a"), (4, "a"), (5, "a"), (6, "a")).toDF("id", "category")
df: org.apache.spark.sql.DataFrame = [id: int, category: string]

scala> val byCategoryOrderedById = Window.partitionBy('category).orderBy('id).rowsBetween(-1, 2)
byCategoryOrderedById: org.apache.spark.sql.expressions.WindowSpec = org.apache.spark.sql.expressions.WindowSpec7f04de97

scala> df.withColumn("sum", sum('id) over byCategoryOrderedById).show()
+---+--------+---+
| id|category|sum|
+---+--------+---+
|  1|       a|  6|              # sum from index 0 to (0 + 2): 1 + 2 + 3 = 6
|  2|       a| 10|              # sum from index (1 - 1) to (1 + 2): 1 + 2 + 3 + 4 = 10
|  3|       a| 14|
|  4|       a| 18|
|  5|       a| 15|
|  6|       a| 11|
+---+--------+---+
```

So the frame (-1, 2) for row with index 5, as described in the doc, should range from index 4 to index 7.

## How was this patch tested?

N/A, just doc change.

Closes #24864 from viirya/window-spec-doc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-14 09:56:37 +09:00
Zhu, Lipeng 5700c39c89 [SPARK-27578][SQL] Support INTERVAL ... HOUR TO SECOND syntax
## What changes were proposed in this pull request?

Currently, SparkSQL can support interval format like this.
```sql
SELECT INTERVAL '0 23:59:59.155' DAY TO SECOND
 ```

Like Presto/Teradata, this PR aims to support grammar like below.
```sql
SELECT INTERVAL '23:59:59.155' HOUR TO SECOND
```

Although we can add a new function for this pattern, we had better extend the existing code to handle a missing day case. So, the following is also supported.
```sql
SELECT INTERVAL '23:59:59.155' DAY TO SECOND
SELECT INTERVAL '1 23:59:59.155' HOUR TO SECOND
```
Currently Vertica/Teradata/Postgresql/SQL Server have fully support of below interval functions.
- interval ... year to month
- interval ... day to hour
- interval ... day to minute
- interval ... day to second
- interval ... hour to minute
- interval ... hour to second
- interval ... minute to second

https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Literals/interval-qualifier.htm
df1a699e5b/src/test/regress/sql/interval.sql (L180-L203)
https://docs.teradata.com/reader/S0Fw2AVH8ff3MDA0wDOHlQ/KdCtT3pYFo~_enc8~kGKVw
https://docs.microsoft.com/en-us/sql/odbc/reference/appendixes/interval-literals?view=sql-server-2017

## How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #24472 from lipzhu/SPARK-27578.

Lead-authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Lipeng Zhu <lipzhu@icloud.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-13 10:12:55 -07:00
zhengruifeng 7281784883 [SPARK-16692][ML][PYTHON] add MultilabelClassificationEvaluator
## What changes were proposed in this pull request?
add MultilabelClassificationEvaluator

## How was this patch tested?
added testsuites

Closes #24777 from zhengruifeng/multi_label_eval.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-13 07:58:22 -05:00
John Zhuge abe370f971 [SPARK-27322][SQL] DataSourceV2 table relation
## What changes were proposed in this pull request?

Support multi-catalog in the following SELECT code paths:

- SELECT * FROM catalog.db.tbl
- TABLE catalog.db.tbl
- JOIN or UNION tables from different catalogs
- SparkSession.table("catalog.db.tbl")
- CTE relation
- View text

## How was this patch tested?

New unit tests.
All existing unit tests in catalyst and sql core.

Closes #24741 from jzhuge/SPARK-27322-pr.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-13 13:48:40 +08:00
Liang-Chi Hsieh ddf4a50312 [SPARK-28031][PYSPARK][TEST] Improve doctest on over function of Column
## What changes were proposed in this pull request?

Just found the doctest on `over` function of `Column` is commented out. The window spec is also not for the window function used there.

We should either remove the doctest, or improve it.

Because other functions of `Column` have doctest generally, so this PR tries to improve it.

## How was this patch tested?

Added doctest.

Closes #24854 from viirya/column-test-minor.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-13 11:04:41 +09:00
Xiangrui Meng 4f4829b4ae [SPARK-28030][SQL] convert filePath to URI in binary file data source
## What changes were proposed in this pull request?

Convert `PartitionedFile.filePath` to URI first in binary file data source. Otherwise Spark will throw a FileNotFound exception because we create `Path` with URL encoded string, instead of wrapping it with URI.

## How was this patch tested?

Unit test.

Closes #24855 from mengxr/SPARK-28030.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-12 13:24:02 -07:00
Dongjoon Hyun 37ab43339d [SPARK-28013][BUILD][SS] Upgrade to Kafka 2.2.1
## What changes were proposed in this pull request?

For Apache Spark 3.0.0 release, this PR aims to update Kafka dependency to 2.2.1 to bring the following improvement and bug fixes like [KAFKA-8134](https://issues.apache.org/jira/browse/KAFKA-8134) (`'linger.ms' must be a long`).

https://issues.apache.org/jira/projects/KAFKA/versions/12345010

## How was this patch tested?

Pass the Jenkins.

Closes #24847 from dongjoon-hyun/SPARK-28013.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-12 07:34:42 -07:00
Liang-Chi Hsieh 2c9597f88f [SPARK-27701][SQL] Extend NestedColumnAliasing to general nested field cases including GetArrayStructField
## What changes were proposed in this pull request?

`NestedColumnAliasing` rule covers `GetStructField` only, currently. It means that some nested field extraction expressions aren't pruned. For example, if only accessing a nested field in an array of struct (`GetArrayStructFields`), this column isn't pruned.

This patch extends the rule to cover general nested field cases, including `GetArrayStructFields`.
## How was this patch tested?

Added tests.

Closes #24599 from viirya/nested-pruning-extract-value.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-11 20:12:53 -07:00
Yesheng Ma 3ddc77d9ac [SPARK-21136][SQL] Disallow FROM-only statements and show better warnings for Hive-style single-from statements
Current Spark SQL parser can have pretty confusing error messages when parsing an incorrect SELECT SQL statement. The proposed fix has the following effect.

BEFORE:
```
spark-sql> SELECT * FROM test WHERE x NOT NULL;
Error in query:
mismatched input 'FROM' expecting {<EOF>, 'CLUSTER', 'DISTRIBUTE', 'EXCEPT', 'GROUP', 'HAVING', 'INTERSECT', 'LATERAL', 'LIMIT', 'ORDER', 'MINUS', 'SORT', 'UNION', 'WHERE', 'WINDOW'}(line 1, pos 9)

== SQL ==
SELECT * FROM test WHERE x NOT NULL
---------^^^
```
where in fact the error message should be hinted to be near `NOT NULL`.

AFTER:
```
spark-sql> SELECT * FROM test WHERE x NOT NULL;
Error in query:
mismatched input 'NOT' expecting {<EOF>, 'AND', 'CLUSTER', 'DISTRIBUTE', 'EXCEPT', 'GROUP', 'HAVING', 'INTERSECT', 'LIMIT', 'OR', 'ORDER', 'MINUS', 'SORT', 'UNION', 'WINDOW'}(line 1, pos 27)

== SQL ==
SELECT * FROM test WHERE x NOT NULL
---------------------------^^^
```

In fact, this problem is brought by some problematic Spark SQL grammar. There are two kinds of SELECT statements that are supported by Hive (and thereby supported in SparkSQL):
* `FROM table SELECT blahblah SELECT blahblah`
* `SELECT blah FROM table`

*Reference* [HiveQL single-from stmt grammar](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g)

It is fine when these two SELECT syntaxes are supported separately. However, since we are currently supporting these two kinds of syntaxes in a single ANTLR rule, this can be problematic and therefore leading to confusing parser errors. This is because when a  SELECT clause was parsed, it can't tell whether the following FROM clause actually belongs to it or is just the beginning of a new `FROM table SELECT *` statement.

## What changes were proposed in this pull request?
1. Modify ANTLR grammar to fix the above-mentioned problem. This fix is important because the previous problematic grammar does affect a lot of real-world queries. Due to the previous problematic and messy grammar, we refactored the grammar related to `querySpecification`.
2. Modify `AstBuilder` to have separate visitors for `SELECT ... FROM ...` and `FROM ... SELECT ...` statements.
3. Drop the `FROM table` statement, which is supported by accident and is actually parsed in the wrong code path. Both Hive and Presto do not support this syntax.

## How was this patch tested?
Existing UTs and new UTs.

Closes #24809 from yeshengm/parser-refactor.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-06-11 18:30:56 -07:00
HyukjinKwon 1217996f15 [SPARK-27995][PYTHON] Note the difference between str of Python 2 and 3 at Arrow optimized
## What changes were proposed in this pull request?

When Arrow optimization is enabled in Python 2.7,

```python
import pandas
pdf = pandas.DataFrame(["test1", "test2"])
df = spark.createDataFrame(pdf)
df.show()
```

I got the following output:

```
+----------------+
|               0|
+----------------+
|[74 65 73 74 31]|
|[74 65 73 74 32]|
+----------------+
```

This looks because Python's `str` and `byte` are same. it does look right:

```python
>>> str == bytes
True
>>> isinstance("a", bytes)
True
```

To cut it short:

1. Python 2 treats `str` as `bytes`.
2. PySpark added some special codes and hacks to recognizes `str` as string types.
3. PyArrow / Pandas followed Python 2 difference

To fix, we have two options:

1. Fix it to match the behaviour to PySpark's
2. Note the differences

 but Python 2 is deprecated anyway. I think it's better to just note it and for go option 2.

## How was this patch tested?

Manually tested.

Doc was checked too:

![Screen Shot 2019-06-11 at 6 40 07 PM](https://user-images.githubusercontent.com/6477701/59261402-59ad3b00-8c78-11e9-94a6-3236a2c338d4.png)

Closes #24838 from HyukjinKwon/SPARK-27995.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-11 18:43:59 +09:00
Yuming Wang 6284ac7088 [SPARK-27934][SQL][TEST] Port case.sql
## What changes were proposed in this pull request?

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

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

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

- [SPARK-27930](https://issues.apache.org/jira/browse/SPARK-27930): Add built-in Math Function: RANDOM

## How was this patch tested?

N/A

Closes #24782 from wangyum/SPARK-27934.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-11 15:57:59 +08:00
LantaoJin 63e0711524 [SPARK-27899][SQL] Make HiveMetastoreClient.getTableObjectsByName available in ExternalCatalog/SessionCatalog API
## What changes were proposed in this pull request?

The new Spark ThriftServer SparkGetTablesOperation implemented in https://github.com/apache/spark/pull/22794 does a catalog.getTableMetadata request for every table. This can get very slow for large schemas (~50ms per table with an external Hive metastore).
Hive ThriftServer GetTablesOperation uses HiveMetastoreClient.getTableObjectsByName to get table information in bulk, but we don't expose that through our APIs that go through Hive -> HiveClientImpl (HiveClient) -> HiveExternalCatalog (ExternalCatalog) -> SessionCatalog.

If we added and exposed getTableObjectsByName through our catalog APIs, we could resolve that performance problem in SparkGetTablesOperation.

## How was this patch tested?

Add UT

Closes #24774 from LantaoJin/SPARK-27899.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-11 15:32:59 +08:00
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