Commit graph

24578 commits

Author SHA1 Message Date
WeichenXu 31e7c37354 [SPARK-28185][PYTHON][SQL] Closes the generator when Python UDFs stop early
## What changes were proposed in this pull request?

 Closes the generator when Python UDFs stop early.

### Manually verification on pandas iterator UDF and mapPartitions

```python
from pyspark.sql import SparkSession
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.functions import col, udf
from pyspark.taskcontext import TaskContext
import time
import os

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
spark.conf.set('spark.sql.pandas.udf.buffer.size', '4')

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi1(it):
    try:
        for batch in it:
            yield batch + 100
            time.sleep(1.0)
    except BaseException as be:
        print("Debug: exception raised: " + str(type(be)))
        raise be
    finally:
        open("/tmp/000001.tmp", "a").close()

df1 = spark.range(10).select(col('id').alias('a')).repartition(1)

# will see log Debug: exception raised: <class 'GeneratorExit'>
# and file "/tmp/000001.tmp" generated.
df1.select(col('a'), fi1('a')).limit(2).collect()

def mapper(it):
    try:
        for batch in it:
                yield batch
    except BaseException as be:
        print("Debug: exception raised: " + str(type(be)))
        raise be
    finally:
        open("/tmp/000002.tmp", "a").close()

df2 = spark.range(10000000).repartition(1)

# will see log Debug: exception raised: <class 'GeneratorExit'>
# and file "/tmp/000002.tmp" generated.
df2.rdd.mapPartitions(mapper).take(2)

```

## How was this patch tested?

Unit test added.

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

Closes #24986 from WeichenXu123/pandas_iter_udf_limit.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-28 17:10:25 +09:00
Yuming Wang 410a898cf9 [SPARK-28179][SQL] Avoid hard-coded config: spark.sql.globalTempDatabase
## What changes were proposed in this pull request?

Avoid hard-coded config: `spark.sql.globalTempDatabase`.

## How was this patch tested?

N/A

Closes #24979 from wangyum/SPARK-28179.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-28 10:42:35 +09:00
Marcelo Vanzin 11e21cc17a [SPARK-28187][BUILD] Add support for hadoop-cloud to the PR builder.
Closes #24987 from vanzin/SPARK-28187.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-06-27 15:59:05 -07:00
Marcelo Vanzin 6af47b93ec [SPARK-28150][CORE] Log in user before getting delegation tokens.
This ensures that tokens are always created with an empty UGI, which
allows multiple contexts to be (sequentially) started from the same JVM.

Tested with code attached to the bug, and also usual kerberos tests.

Closes #24955 from vanzin/SPARK-28150.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-06-27 13:30:28 -07:00
Wenchen Fan cded421aeb [SPARK-27871][SQL] LambdaVariable should use per-query unique IDs instead of globally unique IDs
## What changes were proposed in this pull request?

For simplicity, all `LambdaVariable`s are globally unique, to avoid any potential conflicts. However, this causes a perf problem: we can never hit codegen cache for encoder expressions that deal with collections (which means they contain `LambdaVariable`).

To overcome this problem, `LambdaVariable` should have per-query unique IDs. This PR does 2 things:
1. refactor `LambdaVariable` to carry an ID, so that it's easier to change the ID.
2. add an optimizer rule to reassign `LambdaVariable` IDs, which are per-query unique.

## How was this patch tested?

new tests

Closes #24735 from cloud-fan/dataset.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-27 11:34:47 -07:00
Dongjoon Hyun a7e16199f3 [SPARK-28174][BUILD][SS] Upgrade to Kafka 2.3.0
## What changes were proposed in this pull request?

This issue updates Kafka dependency to 2.3.0 to bring the following 9 client-side patches at least. Among them, the blocker issue [KAFKA-7703](https://issues.apache.org/jira/browse/KAFKA-7703) was reported by Apache Spark community. This dependency update will help us remove the workaround later.
- https://issues.apache.org/jira/issues/?jql=project%20%3D%20KAFKA%20AND%20fixVersion%20%3D%202.3.0%20AND%20fixVersion%20NOT%20IN%20(2.2.0%2C%202.2.1)%20AND%20component%20%3D%20clients

The following is the full release note.
- https://www.apache.org/dist/kafka/2.3.0/RELEASE_NOTES.html

## How was this patch tested?

Pass the Jenkins.

Closes #24976 from dongjoon-hyun/SPARK-28174.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-27 07:49:24 -07:00
Marco Gaido 3139d642fa [SPARK-23179][SQL] Support option to throw exception if overflow occurs during Decimal arithmetic
## What changes were proposed in this pull request?

SQL ANSI 2011 states that in case of overflow during arithmetic operations, an exception should be thrown. This is what most of the SQL DBs do (eg. SQLServer, DB2). Hive currently returns NULL (as Spark does) but HIVE-18291 is open to be SQL compliant.

The PR introduce an option to decide which behavior Spark should follow, ie. returning NULL on overflow or throwing an exception.

## How was this patch tested?

added UTs

Closes #20350 from mgaido91/SPARK-23179.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-27 19:02:07 +08:00
wuyi 7cbe01e8ef [SPARK-27369][CORE] Setup resources when Standalone Worker starts up
## What changes were proposed in this pull request?

To support GPU-aware scheduling in Standalone (cluster mode), Worker should have ability to setup resources(e.g. GPU/FPGA) when it starts up.

Similar as driver/executor do, Worker has two ways(resourceFile & resourceDiscoveryScript) to setup resources when it starts up.  User could use `SPARK_WORKER_OPTS` to apply resource configs on Worker in the form of "-Dx=y". For example,
```
SPARK_WORKER_OPTS="-Dspark.worker.resource.gpu.amount=2 \
                   -Dspark.worker.resource.fpga.amount=1 \
                   -Dspark.worker.resource.fpga.discoveryScript=/Users/wuyi/tmp/getFPGAResources.sh \
                   -Dspark.worker.resourcesFile=/Users/wuyi/tmp/worker-resource-file"
 ```
## How was this patch tested?

Tested manually in Standalone locally:

- Worker could start up normally when no resources are configured

- Worker should fail to start up when exception threw during setup resources(e.g. unknown directory, parse fail)

- Worker could setup resources from resource file

- Worker could setup resources from discovery scripts

- Worker should setup resources from resource file & discovery scripts when both are configure.

Closes #24841 from Ngone51/dev-worker-resources-setup.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-06-26 19:19:00 -07:00
Bryan Cutler c277afb12b [SPARK-27992][PYTHON] Allow Python to join with connection thread to propagate errors
## What changes were proposed in this pull request?

Currently with `toLocalIterator()` and `toPandas()` with Arrow enabled, if the Spark job being run in the background serving thread errors, it will be caught and sent to Python through the PySpark serializer.
This is not the ideal solution because it is only catch a SparkException, it won't handle an error that occurs in the serializer, and each method has to have it's own special handling to propagate the error.

This PR instead returns the Python Server object along with the serving port and authentication info, so that it allows the Python caller to join with the serving thread. During the call to join, the serving thread Future is completed either successfully or with an exception. In the latter case, the exception will be propagated to Python through the Py4j call.

## How was this patch tested?

Existing tests

Closes #24834 from BryanCutler/pyspark-propagate-server-error-SPARK-27992.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-26 13:05:41 -07:00
Dongjoon Hyun 7eeca02940 [SPARK-28157][CORE] Make SHS clear KVStore LogInfos for the blacklisted entries
## What changes were proposed in this pull request?

At Spark 2.4.0/2.3.2/2.2.3, [SPARK-24948](https://issues.apache.org/jira/browse/SPARK-24948) delegated access permission checks to the file system, and maintains a blacklist for all event log files failed once at reading. The blacklisted log files are released back after `CLEAN_INTERVAL_S` seconds.

However, the released files whose sizes don't changes are ignored forever due to `info.fileSize < entry.getLen()` condition (previously [here](3c96937c7b (diff-a7befb99e7bd7e3ab5c46c2568aa5b3eR454)) and now at [shouldReloadLog](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala#L571)) which returns `false` always when the size is the same with the existing value in `KVStore`. This is recovered only via SHS restart.

This PR aims to remove the existing entry from `KVStore` when it goes to the blacklist.

## How was this patch tested?

Pass the Jenkins with the updated test case.

Closes #24966 from dongjoon-hyun/SPARK-28157.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-06-26 18:56:06 +00:00
shivusondur bd232b98b4 [SPARK-28164] Fix usage description of start-slave.sh
## What changes were proposed in this pull request?

updated the usage message in sbin/start-slave.sh.
<masterURL> argument moved to first

## How was this patch tested?
tested locally with
Starting master
starting slave with (./start-slave.sh spark://<IP>:<PORT> -c 1
and opening spark shell with ./spark-shell --master spark://<IP>:<PORT>

Closes #24974 from shivusondur/jira28164.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-26 12:42:33 -05:00
Gabor Somogyi 8313015e8d [SPARK-28005][YARN] Remove unnecessary log from SparkRackResolver
## What changes were proposed in this pull request?

SparkRackResolver generates an INFO message every time is called with 0 arguments.
In this PR I've deleted it because it's too verbose.

## How was this patch tested?

Existing unit tests + spark-shell.

Closes #24935 from gaborgsomogyi/SPARK-28005.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-06-26 09:50:54 -05:00
Josh Rosen d83f84a122 [SPARK-27676][SQL][SS] InMemoryFileIndex should respect spark.sql.files.ignoreMissingFiles
## What changes were proposed in this pull request?

Spark's `InMemoryFileIndex` contains two places where `FileNotFound` exceptions are caught and logged as warnings (during [directory listing](bcd3b61c4b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala (L274)) and [block location lookup](bcd3b61c4b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala (L333))). This logic was added in #15153 and #21408.

I think that this is a dangerous default behavior because it can mask bugs caused by race conditions (e.g. overwriting a table while it's being read) or S3 consistency issues (there's more discussion on this in the [JIRA ticket](https://issues.apache.org/jira/browse/SPARK-27676)). Failing fast when we detect missing files is not sufficient to make concurrent table reads/writes or S3 listing safe (there are other classes of eventual consistency issues to worry about), but I think it's still beneficial to throw exceptions and fail-fast on the subset of inconsistencies / races that we _can_ detect because that increases the likelihood that an end user will notice the problem and investigate further.

There may be some cases where users _do_ want to ignore missing files, but I think that should be an opt-in behavior via the existing `spark.sql.files.ignoreMissingFiles` flag (the current behavior is itself race-prone because a file might be be deleted between catalog listing and query execution time, triggering FileNotFoundExceptions on executors (which are handled in a way that _does_ respect `ignoreMissingFIles`)).

This PR updates `InMemoryFileIndex` to guard the log-and-ignore-FileNotFoundException behind the existing `spark.sql.files.ignoreMissingFiles` flag.

**Note**: this is a change of default behavior, so I think it needs to be mentioned in release notes.

## How was this patch tested?

New unit tests to simulate file-deletion race conditions, tested with both values of the `ignoreMissingFIles` flag.

Closes #24668 from JoshRosen/SPARK-27676.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-26 09:11:28 +09:00
sychen 38263f6d15 [SPARK-27630][CORE] Properly handle task end events from completed stages
## What changes were proposed in this pull request?
Track tasks separately for each stage attempt (instead of tracking by stage), and do NOT reset the numRunningTasks to 0 on StageCompleted.

In the case of stage retry, the `taskEnd` event from the zombie stage sometimes makes the number of `totalRunningTasks` negative, which will causes the job to get stuck.
Similar problem also exists with `stageIdToTaskIndices` & `stageIdToSpeculativeTaskIndices`.
If it is a failed `taskEnd` event of the zombie stage, this will cause `stageIdToTaskIndices` or `stageIdToSpeculativeTaskIndices` to remove the task index of the active stage, and the number of `totalPendingTasks` will increase unexpectedly.
## How was this patch tested?
unit test properly handle task end events from completed stages

Closes #24497 from cxzl25/fix_stuck_job_follow_up.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-06-25 14:30:13 -05:00
“attilapiros” b71c130fc6 [SPARK-27622][CORE] Avoiding the network when block manager fetches disk persisted RDD blocks from the same host
## What changes were proposed in this pull request?

Before this PR during fetching a disk persisted RDD block the network was always used to get the requested block content even when both the source and fetcher executor was running on the same host.

The idea to access another executor local disk files by directly reading the disk comes from the external shuffle service where the local dirs are stored for each executor (block manager).

To make this possible the following changes are done:
- `RegisterBlockManager` message is extended with the `localDirs` which is stored by the block manager master for each block manager as a new property of the `BlockManagerInfo`
- `GetLocationsAndStatus` is extended with the requester host
- `BlockLocationsAndStatus` (the reply for `GetLocationsAndStatus` message) is extended with the an option of local directories, which is filled with a local directories of a same host executor (if there is any, otherwise None is used). This is where the block content can be read from.

Shuffle blocks are out of scope of this PR: there will be a separate PR opened for that (for another Jira issue).

## How was this patch tested?

With a new unit test in `BlockManagerSuite`. See the the test prefixed by "SPARK-27622: avoid the network when block requested from same host".

Closes #24554 from attilapiros/SPARK-27622.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-06-25 07:35:44 -07:00
ketank-new 1a3858a769 [SPARK-26985][CORE] Fix "access only some column of the all of columns " for big endian architecture BUG
continuation to https://github.com/apache/spark/pull/24788

## What changes were proposed in this pull request?

Changes are related to BIG ENDIAN system
This changes are done to

identify s390x platform.
use byteorder to BIG_ENDIAN for big endian systems
changes for 2 are done in access functions putFloats() and putDouble()

## How was this patch tested?

Changes have been tested to build successfully on s390x as well x86 platform to make sure build is successful.

Closes #24861 from ketank-new/ketan_latest_v2.3.2.

Authored-by: ketank-new <ketan22584@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-25 08:24:10 -05:00
zhengruifeng c83b3ddb56 [SPARK-28154][ML][FOLLOWUP] GMM fix double caching
## What changes were proposed in this pull request?
if the input dataset is alreadly cached, then we do not need to cache the internal rdd (like kmeans)

## How was this patch tested?
existing test

Closes #24919 from zhengruifeng/gmm_fix_double_caching.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-25 06:50:34 -05:00
zhengruifeng 83b96f6b30 [SPARK-28117][ML] LDA and BisectingKMeans cache the input dataset if necessary
## What changes were proposed in this pull request?
cache dataset in BisectingKMeans
cache dataset in LDA if Online solver is chosen.

## How was this patch tested?
existing test

Closes #24920 from zhengruifeng/bikm_cache.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-25 06:47:06 -05:00
zhengruifeng c397b06924 [SPARK-28045][ML][PYTHON] add missing RankingEvaluator
## What changes were proposed in this pull request?
add missing RankingEvaluator

## How was this patch tested?
added testsuites

Closes #24869 from zhengruifeng/ranking_eval.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-25 06:44:06 -05:00
Xingbo Jiang 731a60c92e [SPARK-27823][CORE][DOC][FOLLOWUP] Update doc of config spark.driver.resourcesFile
## What changes were proposed in this pull request?

Update doc of config `spark.driver.resourcesFile`

## How was this patch tested?

N/A

Closes #24954 from jiangxb1987/ResourceAllocation.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-25 13:47:47 +09:00
HyukjinKwon ac61f7d018 [SPARK-27893][SQL][PYTHON][FOLLOW-UP] Allow Scalar Pandas and Python UDFs can be tested with Scala test base
## What changes were proposed in this pull request?

After this PR, we can test Pandas and Python UDF as below **in Scala side**:

```scala
import IntegratedUDFTestUtils._
val pandasTestUDF = TestScalarPandasUDF("udf")
spark.range(10).select(pandasTestUDF($"id")).show()
```

## How was this patch tested?

Manually tested.

Closes #24945 from HyukjinKwon/SPARK-27893-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-25 12:00:05 +09:00
Hyukjin Kwon 1d36b892ab [SPARK-7721][INFRA][FOLLOW-UP] Remove cloned coverage repo after posting HTMLs
## What changes were proposed in this pull request?

This PR proposes to remove cloned `pyspark-coverage-site` repo.

it doesn't looks a problem in PR builder but somehow it's problematic in `spark-master-test-sbt-hadoop-2.7`.

## How was this patch tested?

Jenkins.

Closes #23729 from HyukjinKwon/followup-coverage.

Lead-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: shane knapp <incomplete@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-25 09:18:32 +09:00
Sean Owen 67042e90e7 [MINOR][BUILD] Exclude pyspark-coverage-site/ dir from RAT
## What changes were proposed in this pull request?

Looks like a directory `pyspark-site-coverage/` is now (?) generated and fails RAT checks. It should just be excluded. See: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/6029/console

## How was this patch tested?

N/A

Closes #24950 from srowen/pysparkcoveragesite.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 14:07:41 -05:00
Li Jin d0fbc4da3b [SPARK-28003][PYTHON] Allow NaT values when creating Spark dataframe from pandas with Arrow
## What changes were proposed in this pull request?

This patch removes `fillna(0)` when creating ArrowBatch from a pandas Series.

With `fillna(0)`, the original code would turn a timestamp type into object type, which pyarrow will complain later:
```
>>> s = pd.Series([pd.NaT, pd.Timestamp('2015-01-01')])
>>> s.dtypes
dtype('<M8[ns]')
>>> s.fillna(0)
0                      0
1    2015-01-01 00:00:00
dtype: object
```

## How was this patch tested?

Added `test_timestamp_nat`

Closes #24844 from icexelloss/SPARK-28003-arrow-nat.

Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-24 11:15:21 -07:00
Sean Owen 9df7587eea [MINOR][CORE] Fix line too long in TransportClientFactory
## What changes were proposed in this pull request?

See https://github.com/apache/spark/pull/24702/files#r296765487 -- this just fixes a Java style error. I'm not clear why the PR build didn't catch it.

## How was this patch tested?

N/A

Closes #24951 from srowen/SPARK-27989.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 11:17:54 -05:00
zhengruifeng 6064368415 [SPARK-27018][CORE] Fix incorrect removal of checkpointed file in PeriodicCheckpointer
## What changes were proposed in this pull request?
remove the oldest checkpointed file only if next checkpoint exists.
I think this patch needs back-porting.

## How was this patch tested?
existing test

local check in spark-shell with following suite:
```
import org.apache.spark.ml.linalg.Vectors
import org.apache.spark.ml.classification.GBTClassifier

case class Row(features: org.apache.spark.ml.linalg.Vector, label: Int)

sc.setCheckpointDir("/checkpoints")
val trainingData = sc.parallelize(1 to 2426874, 256).map(x => Row(Vectors.dense(x, x + 1, x * 2 % 10), if (x % 5 == 0) 1 else 0)).toDF
val classifier = new GBTClassifier()
  .setLabelCol("label")
  .setFeaturesCol("features")
  .setProbabilityCol("probability")
  .setMaxIter(100)
  .setMaxDepth(10)
  .setCheckpointInterval(2)

classifier.fit(trainingData)
```

Closes #24870 from zhengruifeng/ck_update.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 09:34:01 -05:00
Jose Luis Pedrosa 0671395966 [SPARK-27989][CORE] Added retries on the connection to the driver for k8s
Disabled negative dns caching for docker images

Improved logging on DNS resolution, convenient for slow k8s clusters

## What changes were proposed in this pull request?
Added retries when building the connection to the driver in K8s.
In some scenarios DNS reslution can take more than the timeout.
Also openjdk-8 by default has negative dns caching enabled, which means even retries may not help depending on the times.

## How was this patch tested?
This patch was tested agains an specific k8s cluster with slow response time in DNS to ensure it woks.

Closes #24702 from jlpedrosa/feature/kuberetries.

Authored-by: Jose Luis Pedrosa <jlpedrosa@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 09:25:43 -05:00
Jungtaek Lim (HeartSaVioR) 85e95b7d27 [SPARK-28142][SS] Use CaseInsensitiveStringMap for KafkaContinuousStream
## What changes were proposed in this pull request?

This patch addresses a missing spot which Map should be passed as CaseInsensitiveStringMap - KafkaContinuousStream seems to be only the missed one.

Before this fix, it has a relevant bug where `pollTimeoutMs` is always set to default value, as the value of `KafkaSourceProvider.CONSUMER_POLL_TIMEOUT` is `kafkaConsumer.pollTimeoutMs` which key-lowercased map has been provided as `sourceOptions`.

## How was this patch tested?

N/A.

Closes #24942 from HeartSaVioR/MINOR-use-case-insensitive-map-for-kafka-continuous-source.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-24 22:20:32 +09:00
Gengliang Wang b5e183cdc7 [SPARK-28108][SQL][test-hadoop3.2] Simplify OrcFilters
## What changes were proposed in this pull request?

In #24068, IvanVergiliev fixes the issue that OrcFilters.createBuilder has exponential complexity in the height of the filter tree due to the way the check-and-build pattern is implemented.

Comparing to the approach in #24068, I propose a simple solution for the issue:
1. separate the logic of building a convertible filter tree and the actual SearchArgument builder, since the two procedures are different and their return types are different. Thus the new introduced class `ActionType`,`TrimUnconvertibleFilters` and `BuildSearchArgument`  in #24068 can be dropped. The code is more readable.
2. For most of the leaf nodes, the convertible result is always Some(node), we can abstract it like this PR.
3. The code is actually small changes on the previous code. See https://github.com/apache/spark/pull/24783

## How was this patch tested?
Run the benchmark provided in #24068:
```
val schema = StructType.fromDDL("col INT")
(20 to 30).foreach { width =>
  val whereFilter = (1 to width).map(i => EqualTo("col", i)).reduceLeft(Or)
  val start = System.currentTimeMillis()
  OrcFilters.createFilter(schema, Seq(whereFilter))
  println(s"With $width filters, conversion takes ${System.currentTimeMillis() - start} ms")
}
```
Result:
```
With 20 filters, conversion takes 6 ms
With 21 filters, conversion takes 0 ms
With 22 filters, conversion takes 0 ms
With 23 filters, conversion takes 0 ms
With 24 filters, conversion takes 0 ms
With 25 filters, conversion takes 0 ms
With 26 filters, conversion takes 0 ms
With 27 filters, conversion takes 0 ms
With 28 filters, conversion takes 0 ms
With 29 filters, conversion takes 0 ms
With 30 filters, conversion takes 0 ms
```

Also verified with Unit tests.

Closes #24910 from gengliangwang/refactorOrcFilters.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-24 12:23:52 +08:00
HyukjinKwon 7c05f61514 [SPARK-28130][PYTHON] Print pretty messages for skipped tests when xmlrunner is available in PySpark
## What changes were proposed in this pull request?

Currently, pretty skipped message added by f7435bec6a mechanism seems not working when xmlrunner is installed apparently.

This PR fixes two things:

1. When `xmlrunner` is installed, seems `xmlrunner` does not respect `vervosity` level in unittests (default is level 1).

    So the output looks as below

    ```
    Running tests...
     ----------------------------------------------------------------------
    SSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSS
    ----------------------------------------------------------------------
    ```

    So it is not caught by our message detection mechanism.

2. If we manually set the `vervocity` level to `xmlrunner`, it prints messages as below:

    ```
    test_mixed_udf (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s)
    test_mixed_udf_and_sql (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s)
    ...
    ```

    This is different in our Jenkins machine:

    ```
    test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.'
    test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.'
    ...
    ```

    Note that last `SKIP` is different. This PR fixes the regular expression to catch `SKIP` case as well.

## How was this patch tested?

Manually tested.

**Before:**

```
Starting test(python2.7): pyspark....
Finished test(python2.7): pyspark.... (0s)
...
Tests passed in 562 seconds

========================================================================
...
```

**After:**

```
Starting test(python2.7): pyspark....
Finished test(python2.7): pyspark.... (48s) ... 93 tests were skipped
...
Tests passed in 560 seconds

Skipped tests pyspark.... with python2.7:
      pyspark...(...) ... SKIP (0.000s)
...

========================================================================
...
```

Closes #24927 from HyukjinKwon/SPARK-28130.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-24 09:58:17 +09:00
Liang-Chi Hsieh a00774afea [SPARK-28054][SQL] Fix error when insert Hive partitioned table dynamically where partition name is upper case
## What changes were proposed in this pull request?

When we use upper case partition name in Hive table, like:

```
CREATE TABLE src (KEY STRING, VALUE STRING) PARTITIONED BY (DS STRING)
```

Then, `insert into table` query doesn't work
```
INSERT INTO TABLE src PARTITION(ds) SELECT 'k' key, 'v' value, '1' ds
// or
INSERT INTO TABLE src PARTITION(DS) SELECT 'k' KEY, 'v' VALUE, '1' DS
```

```
[info]   org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.Table.ValidationFailureSemanticException: Partition spec {ds=, DS=1} contains non-partition columns;
```

As Hive metastore is not case preserving and keeps partition columns with lower cased names, we lowercase column names in partition spec before passing to Hive client. But we write upper case column names in partition paths.

However, when calling `loadDynamicPartitions` to do `insert into table` for dynamic partition, Hive calculates full path spec for partition paths. So it calculates a partition spec like `{ds=, DS=1}` in above case and fails partition column validation. This patch is proposed to fix the issue by lowercasing the column names in written partition paths for Hive partitioned table.

This fix touchs `saveAsHiveFile` method, which is used in `InsertIntoHiveDirCommand` and `InsertIntoHiveTable` commands. Among them, only `InsertIntoHiveTable` passes `partitionAttributes` parameter. So I think this change only affects `InsertIntoHiveTable` command.

## How was this patch tested?

Added test.

Closes #24886 from viirya/SPARK-28054.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-24 09:44:38 +09:00
Gabor Somogyi 1a915bf20f [MINOR][SQL][DOCS] failOnDataLoss has effect on batch queries so fix the doc
## What changes were proposed in this pull request?

According to the [Kafka integration document](https://spark.apache.org/docs/2.4.0/structured-streaming-kafka-integration.html) `failOnDataLoss` has effect only on streaming queries. While I was implementing the DSv2 Kafka batch sources I've realized it's not true. This feature is covered in [KafkaDontFailOnDataLossSuite](54da3bbfb2/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala (L180)).

In this PR I've updated the doc to reflect this behavior.

## How was this patch tested?

```
cd docs/
SKIP_API=1 jekyll build
```
Manual webpage check.

Closes #24932 from gaborgsomogyi/failOnDataLoss.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-23 19:23:57 -05:00
Parth Chandra 5a7aa6f4df [SPARK-27100][SQL] Use Array instead of Seq in FilePartition to prevent StackOverflowError
## What changes were proposed in this pull request?

ShuffleMapTask's partition field is a FilePartition and FilePartition's 'files' field is a Stream$cons which is essentially a linked list. It is therefore serialized recursively.
If the number of files in each partition is, say, 10000 files, recursing into a linked list of length 10000 overflows the stack

The problem is only in Bucketed partitions. The corresponding implementation for non Bucketed partitions uses a StreamBuffer. The proposed change applies the same for Bucketed partitions.

## How was this patch tested?

Existing unit tests. Added new unit test. The unit test fails without the patch. Manual testing on dataset used to reproduce the problem.

Closes #24865 from parthchandra/SPARK-27100.

Lead-authored-by: Parth Chandra <parthc@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-06-23 07:47:32 +00:00
Yuming Wang 929d313568 [SPARK-28059][SQL][TEST] Port int4.sql
## What changes were proposed in this pull request?

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

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

When porting the test cases, found two PostgreSQL specific features that do not exist in Spark SQL:
[SPARK-28023](https://issues.apache.org/jira/browse/SPARK-28023): Trim the string when cast string type to other types
[SPARK-28027](https://issues.apache.org/jira/browse/SPARK-28027): Add bitwise shift left/right operators

Also, found a bug:
[SPARK-28024](https://issues.apache.org/jira/browse/SPARK-28024): Incorrect value when out of range

Also, found four inconsistent behavior:
[SPARK-27923](https://issues.apache.org/jira/browse/SPARK-27923): Invalid input syntax for integer: "34.5" at PostgreSQL
[SPARK-28027](https://issues.apache.org/jira/browse/SPARK-28027) Our operator `!` and `!!` has different meanings
[SPARK-28028](https://issues.apache.org/jira/browse/SPARK-28028): Cast numeric to integral type need round
[SPARK-2659](https://issues.apache.org/jira/browse/SPARK-2659): HiveQL: Division operator should always perform fractional division, for example:
```sql
select 1/2;
```

## How was this patch tested?

N/A

Closes #24877 from wangyum/SPARK-28059.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-22 23:59:30 -07:00
Yuming Wang 0768fad777 [SPARK-28126][SQL] Support TRIM(trimStr FROM str) syntax
## What changes were proposed in this pull request?
[PostgreSQL](7c850320d8/src/test/regress/sql/strings.sql (L624)) support  another trim pattern: `TRIM(trimStr FROM str)`:

Function | Return Type | Description | Example | Result
--- | --- | --- | --- | ---
trim([leading \| trailing \| both] [characters] from string) | text | Remove the longest string containing only characters from characters (a space by default) from the start, end, or both ends (both is the default) of string | trim(both 'xyz' from 'yxTomxx') | Tom

This pr add support this trim pattern. After this pr. We can support all standard syntax except `TRIM(FROM str)` because it conflicts with our Literals:
```sql
Literals of type 'FROM' are currently not supported.(line 1, pos 12)

== SQL ==
SELECT TRIM(FROM ' SPARK SQL ')
```

PostgreSQL, Vertica and MySQL support this pattern. Teradata, Oracle, DB2, SQL Server, Hive and Presto
**PostgreSQL**:
```
postgres=# SELECT substr(version(), 0, 16), trim('xyz' FROM 'yxTomxx');
     substr      | btrim
-----------------+-------
 PostgreSQL 11.3 | Tom
(1 row)
```
**Vertica**:
```
dbadmin=> SELECT version(), trim('xyz' FROM 'yxTomxx');
              version               | btrim
------------------------------------+-------
 Vertica Analytic Database v9.1.1-0 | Tom
(1 row)
```
**MySQL**:
```
mysql> SELECT version(), trim('xyz' FROM 'yxTomxx');
+-----------+----------------------------+
| version() | trim('xyz' FROM 'yxTomxx') |
+-----------+----------------------------+
| 5.7.26    | yxTomxx                    |
+-----------+----------------------------+
1 row in set (0.00 sec)
```

More details:
https://www.postgresql.org/docs/11/functions-string.html

## How was this patch tested?

unit tests

Closes #24924 from wangyum/SPARK-28075-2.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-22 23:10:09 -07:00
Yuming Wang 870f972dcc [SPARK-28104][SQL] Implement Spark's own GetColumnsOperation
## What changes were proposed in this pull request?

[SPARK-24196](https://issues.apache.org/jira/browse/SPARK-24196) and [SPARK-24570](https://issues.apache.org/jira/browse/SPARK-24570) implemented Spark's own `GetSchemasOperation` and `GetTablesOperation`. This pr implements Spark's own `GetColumnsOperation`.

## How was this patch tested?

unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/59745367-3a7d6180-92a7-11e9-862d-96bc494c5f00.png)

Closes #24906 from wangyum/SPARK-28104.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-22 09:15:07 -07:00
Bryan Cutler 5ad1053f3e [SPARK-28128][PYTHON][SQL] Pandas Grouped UDFs skip empty partitions
## What changes were proposed in this pull request?

When running FlatMapGroupsInPandasExec or AggregateInPandasExec the shuffle uses a default number of partitions of 200 in "spark.sql.shuffle.partitions". If the data is small, e.g. in testing, many of the partitions will be empty but are treated just the same.

This PR checks the `mapPartitionsInternal` iterator to be non-empty before calling `ArrowPythonRunner` to start computation on the iterator.

## How was this patch tested?

Existing tests. Ran the following benchmarks a simple example where most partitions are empty:

```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.types import *

df = spark.createDataFrame(
     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
     ("id", "v"))

pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP)
def normalize(pdf):
    v = pdf.v
    return pdf.assign(v=(v - v.mean()) / v.std())

df.groupby("id").apply(normalize).count()
```

**Before**
```
In [4]: %timeit df.groupby("id").apply(normalize).count()
1.58 s ± 62.8 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

In [5]: %timeit df.groupby("id").apply(normalize).count()
1.52 s ± 29.5 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

In [6]: %timeit df.groupby("id").apply(normalize).count()
1.52 s ± 37.8 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)
```

**After this Change**
```
In [2]: %timeit df.groupby("id").apply(normalize).count()
646 ms ± 89.9 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

In [3]: %timeit df.groupby("id").apply(normalize).count()
408 ms ± 84.6 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

In [4]: %timeit df.groupby("id").apply(normalize).count()
381 ms ± 29.9 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)
```

Closes #24926 from BryanCutler/pyspark-pandas_udf-map-agg-skip-empty-parts-SPARK-28128.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-22 11:20:35 +09:00
HyukjinKwon 113f8c8d13 [SPARK-28132][PYTHON] Update document type conversion for Pandas UDFs (pyarrow 0.13.0, pandas 0.24.2, Python 3.7)
## What changes were proposed in this pull request?

This PR updates the chart generated at SPARK-25666. We deprecated Python 2. It's better to use Python 3.

We don't have to test `unicode` and `long` anymore in Python 3. So it was removed.

Use this code to generate the chart:

```python
from pyspark.sql.types import *
from pyspark.sql.functions import pandas_udf

columns = [
    ('none', 'object(NoneType)'),
    ('bool', 'bool'),
    ('int8', 'int8'),
    ('int16', 'int16'),
    ('int32', 'int32'),
    ('int64', 'int64'),
    ('uint8', 'uint8'),
    ('uint16', 'uint16'),
    ('uint32', 'uint32'),
    ('uint64', 'uint64'),
    ('float64', 'float16'),
    ('float64', 'float32'),
    ('float64', 'float64'),
    ('date', 'datetime64[ns]'),
    ('tz_aware_dates', 'datetime64[ns, US/Eastern]'),
    ('string', 'object(string)'),
    ('decimal', 'object(Decimal)'),
    ('array', 'object(array[int32])'),
    ('float128', 'float128'),
    ('complex64', 'complex64'),
    ('complex128', 'complex128'),
    ('category', 'category'),
    ('tdeltas', 'timedelta64[ns]'),
]

def create_dataframe():
    import pandas as pd
    import numpy as np
    import decimal
    pdf = pd.DataFrame({
        'none': [None, None],
        'bool': [True, False],
        'int8': np.arange(1, 3).astype('int8'),
        'int16': np.arange(1, 3).astype('int16'),
        'int32': np.arange(1, 3).astype('int32'),
        'int64': np.arange(1, 3).astype('int64'),
        'uint8': np.arange(1, 3).astype('uint8'),
        'uint16': np.arange(1, 3).astype('uint16'),
        'uint32': np.arange(1, 3).astype('uint32'),
        'uint64': np.arange(1, 3).astype('uint64'),
        'float16': np.arange(1, 3).astype('float16'),
        'float32': np.arange(1, 3).astype('float32'),
        'float64': np.arange(1, 3).astype('float64'),
        'float128': np.arange(1, 3).astype('float128'),
        'complex64': np.arange(1, 3).astype('complex64'),
        'complex128': np.arange(1, 3).astype('complex128'),
        'string': list('ab'),
        'array': pd.Series([np.array([1, 2, 3], dtype=np.int32), np.array([1, 2, 3], dtype=np.int32)]),
        'decimal': pd.Series([decimal.Decimal('1'), decimal.Decimal('2')]),
        'date': pd.date_range('19700101', periods=2).values,
        'category': pd.Series(list("AB")).astype('category')})
    pdf['tdeltas'] = [pdf.date.diff()[1], pdf.date.diff()[0]]
    pdf['tz_aware_dates'] = pd.date_range('19700101', periods=2, tz='US/Eastern')
    return pdf

types =  [
    BooleanType(),
    ByteType(),
    ShortType(),
    IntegerType(),
    LongType(),
    FloatType(),
    DoubleType(),
    DateType(),
    TimestampType(),
    StringType(),
    DecimalType(10, 0),
    ArrayType(IntegerType()),
    MapType(StringType(), IntegerType()),
    StructType([StructField("_1", IntegerType())]),
    BinaryType(),
]

df = spark.range(2).repartition(1)
results = []
count = 0
total = len(types) * len(columns)
values = []
spark.sparkContext.setLogLevel("FATAL")
for t in types:
    result = []
    for column, pandas_t in columns:
        v = create_dataframe()[column][0]
        values.append(v)
        try:
            row = df.select(pandas_udf(lambda _: create_dataframe()[column], t)(df.id)).first()
            ret_str = repr(row[0])
        except Exception:
            ret_str = "X"
        result.append(ret_str)
        progress = "SQL Type: [%s]\n  Pandas Value(Type): %s(%s)]\n  Result Python Value: [%s]" % (
            t.simpleString(), v, pandas_t, ret_str)
        count += 1
        print("%s/%s:\n  %s" % (count, total, progress))
    results.append([t.simpleString()] + list(map(str, result)))

schema = ["SQL Type \\ Pandas Value(Type)"] + list(map(lambda values_column: "%s(%s)" % (values_column[0], values_column[1][1]), zip(values, columns)))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: "    # %s  # noqa" % line, strings.strip().split("\n"))))
```

## How was this patch tested?

Manually.

Closes #24930 from HyukjinKwon/SPARK-28132.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-21 10:47:54 -07:00
HyukjinKwon 9b9d81b821 [SPARK-28131][PYTHON] Update document type conversion between Python data and SQL types in normal UDFs (Python 3.7)
## What changes were proposed in this pull request?

This PR updates the chart generated at SPARK-25666. We deprecated Python 2. It's better to use Python 3.

We don't have to test `unicode` and `long` anymore in Python 3. So it was removed.

Use this code to generate the chart:

```python
import sys
import array
import datetime
from decimal import Decimal

from pyspark.sql import Row
from pyspark.sql.types import *
from pyspark.sql.functions import udf

data = [
    None,
    True,
    1,
    "a",
    datetime.date(1970, 1, 1),
    datetime.datetime(1970, 1, 1, 0, 0),
    1.0,
    array.array("i", [1]),
    [1],
    (1,),
    bytearray([65, 66, 67]),
    Decimal(1),
    {"a": 1},
    Row(kwargs=1),
    Row("namedtuple")(1),
]

types =  [
    BooleanType(),
    ByteType(),
    ShortType(),
    IntegerType(),
    LongType(),
    StringType(),
    DateType(),
    TimestampType(),
    FloatType(),
    DoubleType(),
    ArrayType(IntegerType()),
    BinaryType(),
    DecimalType(10, 0),
    MapType(StringType(), IntegerType()),
    StructType([StructField("_1", IntegerType())]),
]

df = spark.range(1)
results = []
count = 0
total = len(types) * len(data)
spark.sparkContext.setLogLevel("FATAL")
for t in types:
    result = []
    for v in data:
        try:
            row = df.select(udf(lambda: v, t)()).first()
            ret_str = repr(row[0])
        except Exception:
            ret_str = "X"
        result.append(ret_str)
        progress = "SQL Type: [%s]\n  Python Value: [%s(%s)]\n  Result Python Value: [%s]" % (
            t.simpleString(), str(v), type(v).__name__, ret_str)
        count += 1
        print("%s/%s:\n  %s" % (count, total, progress))
    results.append([t.simpleString()] + list(map(str, result)))

schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: "    # %s  # noqa" % line, strings.strip().split("\n"))))
```

## How was this patch tested?

Manually.

Closes #24929 from HyukjinKwon/SPARK-28131.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-21 10:27:18 -07:00
Yesheng Ma 54da3bbfb2 [SPARK-28127][SQL] Micro optimization on TreeNode's mapChildren method
## What changes were proposed in this pull request?

The `mapChildren` method in the TreeNode class is commonly used across the whole Spark SQL codebase. In this method, there's a if statement that checks non-empty children. However, there's a cached lazy val `containsChild`, which can avoid unnecessary computation since `containsChild` is used in other methods and therefore constructed anyway.

Benchmark showed that this optimization can improve the whole TPC-DS planning time by 6.8%. There is no regression on any TPC-DS query.

## How was this patch tested?

Existing UTs.

Closes #24925 from yeshengm/treenode-children.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-20 19:45:59 -07:00
Dongjoon Hyun 47f54b1ec7 [SPARK-28118][CORE] Add spark.eventLog.compression.codec configuration
## What changes were proposed in this pull request?

Event logs are different from the other data in terms of the lifetime. It would be great to have a new configuration for Spark event log compression like `spark.eventLog.compression.codec` .
This PR adds this new configuration as an optional configuration. So, if `spark.eventLog.compression.codec` is not given, `spark.io.compression.codec` will be used.

## How was this patch tested?

Pass the Jenkins with the newly added test case.

Closes #24921 from dongjoon-hyun/SPARK-28118.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-06-21 00:43:38 +00:00
Xingbo Jiang d98a5ce34d [SPARK-28115][CORE][TEST] Fix flaky test: SparkContextSuite.test resource scheduling under local-cluster mode
## What changes were proposed in this pull request?

The test `SparkContextSuite.test resource scheduling under local-cluster mode` has been flaky, because it expects the size of `sc.statusTracker.getExecutorInfos` be the same as the number of executors, while the returned list contains both the driver and executors.

## How was this patch tested?

Updated existing tests.

Closes #24917 from jiangxb1987/getExecutorInfos.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-20 13:23:29 -07:00
tools4origins 25c5d57883 [MINOR][DOC] Fix python variance() documentation
## What changes were proposed in this pull request?

The Python documentation incorrectly says that `variance()` acts as `var_pop` whereas it acts like `var_samp` here: https://spark.apache.org/docs/latest/api/python/pyspark.sql.html#pyspark.sql.functions.variance

It was not the case in Spark 1.6 doc but it is in Spark 2.0 doc:
https://spark.apache.org/docs/1.6.0/api/java/org/apache/spark/sql/functions.html
https://spark.apache.org/docs/2.0.0/api/java/org/apache/spark/sql/functions.html

The Scala documentation is correct: https://spark.apache.org/docs/latest/api/java/org/apache/spark/sql/functions.html#variance-org.apache.spark.sql.Column-

The alias is set on this line:
https://github.com/apache/spark/blob/v2.4.3/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L786

## How was this patch tested?
Using variance() in pyspark 2.4.3 returns:
```
>>> spark.createDataFrame([(1, ), (2, ), (3, )], "a: int").select(variance("a")).show()
+-----------+
|var_samp(a)|
+-----------+
|        1.0|
+-----------+
```

Closes #24895 from tools4origins/patch-1.

Authored-by: tools4origins <tools4origins@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-20 08:10:19 -07:00
Dongjoon Hyun ea0e119f84 [SPARK-28111][BUILD] Upgrade xbean-asm7-shaded to 4.14
## What changes were proposed in this pull request?

This PR aims to update `xbean-asm7-shaded` to bring [XBEAN-318](https://issues.apache.org/jira/browse/XBEAN-318) which is helpful to log the class definition reading failures.
- https://issues.apache.org/jira/projects/XBEAN/versions/12345220

## How was this patch tested?

Pass the Jenkins.

Closes #24914 from dongjoon-hyun/SPARK-28111.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-20 07:59:59 -07:00
sychen d9697fedf5 [SPARK-28012][SQL] Hive UDF supports struct type foldable expression
## What changes were proposed in this pull request?

Currently using hive udf, the parameter is struct type, there will be an exception thrown.

No handler for Hive UDF 'xxxUDF': java.lang.RuntimeException: Hive doesn't support the constant type [StructType(StructField(name,StringType,true), StructField(value,DecimalType(3,1),true))]

## How was this patch tested?
added new UT

Closes #24846 from cxzl25/hive_udf_literal_struct_type.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-20 14:36:01 +09:00
Yuming Wang 4968f87168 [SPARK-23263][TEST] CTAS should update stat if autoUpdate statistics is enabled
## What changes were proposed in this pull request?
The [SPARK-27403](https://issues.apache.org/jira/browse/SPARK-27403) fixed CTAS cannot update statistics even if `spark.sql.statistics.size.autoUpdate.enabled` is enabled, as mentioned in [SPARK-23263](https://issues.apache.org/jira/browse/SPARK-23263).

This pr adds tests for that fix.

## How was this patch tested?

N/A

Closes #20430 from wangyum/SPARK-23263.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-20 14:19:10 +09:00
Gengliang Wang f5107614d6 [SPARK-28089][SQL] File source v2: support reading output of file streaming Sink
## What changes were proposed in this pull request?

File source V1 supports reading output of FileStreamSink as batch. https://github.com/apache/spark/pull/11897
We should support this in file source V2 as well. When reading with paths, we first check if there is metadata log of FileStreamSink. If yes, we use `MetadataLogFileIndex` for listing files; Otherwise, we use `InMemoryFileIndex`.

## How was this patch tested?

Unit test

Closes #24900 from gengliangwang/FileStreamV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-20 12:57:13 +08:00
WeichenXu b276788d57 [SPARK-27990][SQL][ML] Provide a way to recursively load data from datasource
## What changes were proposed in this pull request?

Provide a way to recursively load data from datasource.
I add a "recursiveFileLookup" option.

When "recursiveFileLookup" option turn on, then partition inferring is turned off and all files from the directory will be loaded recursively.

If some datasource explicitly specify the partitionSpec, then if user turn on "recursive" option, then exception will be thrown.

## How was this patch tested?

Unit tests.

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

Closes #24830 from WeichenXu123/recursive_ds.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-20 12:43:01 +08:00
Josh Rosen ec032cea4f [SPARK-28112][TEST] Fix Kryo exception perf. bottleneck in tests due to absence of ML/MLlib classes
## What changes were proposed in this pull request?
In a nutshell, it looks like the absence of ML / MLlib classes on the classpath causes code in KryoSerializer to throw and catch ClassNotFoundExceptions whenever instantiating a new serializer in newInstance(). This isn't a performance problem in production (since MLlib is on the classpath there) but it's a huge issue in tests and appears to account for an enormous amount of test time

We can address this problem by reducing the total number of ClassNotFoundExceptions by performing the class existence checks once and storing the results in KryoSerializer instances rather than repeating the checks on each newInstance() call.

## How was this patch tested?
The existing tests.

Authored-by: Josh Rosen <joshrosendatabricks.com>

Closes #24916 from gatorsmile/kryoException.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-06-19 19:06:22 -07:00
Josh Rosen 6b27ad5ea1 [SPARK-28102][CORE] Avoid performance problems when lz4-java JNI libraries fail to initialize
## What changes were proposed in this pull request?

This PR fixes a performance problem in environments where `lz4-java`'s native JNI libraries fail to initialize.

Spark's uses `lz4-java` for LZ4 compression. Under the hood, the `LZ4BlockInputStream` and `LZ4BlockOutputStream` constructors call `LZ4Factory.fastestInstance()`, which attempts to load JNI libraries and falls back on Java implementations in case the JNI library cannot be loaded or initialized.

If the LZ4 JNI libraries are present on the library load path (`Native.isLoaded()`) but cannot be initialized (e.g. due to breakage caused by shading) then an exception will be thrown and caught, triggering fallback to `fastestJavaInstance()` (a non-JNI implementation).

Unfortunately, the LZ4 library does not cache the fact that the JNI library failed during initialization, so every call to `LZ4Factory.fastestInstance()` re-attempts (and fails) to initialize the native code. These initialization attempts are performed in a `static synchronized` method, so exceptions from failures are thrown while holding shared monitors and this causes monitor-contention performance issues. Here's an example stack trace showing the problem:

```java

java.lang.Throwable.fillInStackTrace(Native Method)
java.lang.Throwable.fillInStackTrace(Throwable.java:783) => holding Monitor(java.lang.NoClassDefFoundError441628568})
java.lang.Throwable.<init>(Throwable.java:265)
java.lang.Error.<init>(Error.java:70)
java.lang.LinkageError.<init>(LinkageError.java:55)
java.lang.NoClassDefFoundError.<init>(NoClassDefFoundError.java:59)
shaded.net.jpountz.lz4.LZ4JNICompressor.compress(LZ4JNICompressor.java:36)
shaded.net.jpountz.lz4.LZ4Factory.<init>(LZ4Factory.java:200)
shaded.net.jpountz.lz4.LZ4Factory.instance(LZ4Factory.java:51)
shaded.net.jpountz.lz4.LZ4Factory.nativeInstance(LZ4Factory.java:84) => holding Monitor(java.lang.Class1475983836})
shaded.net.jpountz.lz4.LZ4Factory.fastestInstance(LZ4Factory.java:157)
shaded.net.jpountz.lz4.LZ4BlockOutputStream.<init>(LZ4BlockOutputStream.java:135)
org.apache.spark.io.LZ4CompressionCodec.compressedOutputStream(CompressionCodec.scala:122)
org.apache.spark.serializer.SerializerManager.wrapForCompression(SerializerManager.scala:156)
org.apache.spark.serializer.SerializerManager.wrapStream(SerializerManager.scala:131)
org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:120)
org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:249)
org.apache.spark.shuffle.sort.ShuffleExternalSorter.writeSortedFile(ShuffleExternalSorter.java:211)
[...]
```

To avoid this problem, this PR modifies Spark's `LZ4CompressionCodec` to call `fastestInstance()` itself and cache the result (which is safe because these factories [are thread-safe](https://github.com/lz4/lz4-java/issues/82)).

## How was this patch tested?

Existing unit tests.

Closes #24905 from JoshRosen/lz4-factory-flags.

Lead-authored-by: Josh Rosen <rosenville@gmail.com>
Co-authored-by: Josh Rosen <joshrosen@stripe.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-06-19 15:26:26 -07:00