Commit graph

7244 commits

Author SHA1 Message Date
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
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
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
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
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
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
Xiangrui Meng 7056e004ee [SPARK-27823][CORE] Refactor resource handling code
## What changes were proposed in this pull request?

Continue the work from https://github.com/apache/spark/pull/24821. Refactor resource handling code to make the code more readable. Major changes:

* Moved resource-related classes to `spark.resource` from `spark`.
* Added ResourceUtils and helper classes so we don't need to directly deal with Spark conf.
 * ResourceID: resource identifier and it provides conf keys
 * ResourceRequest/Allocation: abstraction for requested and allocated resources
* Added `TestResourceIDs` to reference commonly used resource IDs in tests like `spark.executor.resource.gpu`.

cc: tgravescs jiangxb1987 Ngone51

## How was this patch tested?

Unit tests for added utils and existing unit tests.

Closes #24856 from mengxr/SPARK-27823.

Lead-authored-by: Xiangrui Meng <meng@databricks.com>
Co-authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-06-18 17:18:17 -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
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
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
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
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
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
Thomas Graves d30284b5a5 [SPARK-27760][CORE] Spark resources - change user resource config from .count to .amount
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Unit tests and manually verified on yarn and local cluster mode

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

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-06-06 14:16:05 -05:00
Marcelo Vanzin b312033bd3 [SPARK-20286][CORE] Improve logic for timing out executors in dynamic allocation.
This change refactors the portions of the ExecutorAllocationManager class that
track executor state into a new class, to achieve a few goals:

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

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

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

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

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

Closes #24704 from vanzin/SPARK-20286.

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

Use objects in `ResourceName` to represent resource names.

## How was this patch tested?

Existing tests.

Closes #24799 from jiangxb1987/ResourceName.

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

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

## How was this patch tested?

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

Closes #24374 from jiangxb1987/gpu.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-04 16:57:47 -07:00
HyukjinKwon 8b18ef5c7b [MINOR] Avoid hardcoded py4j-0.10.8.1-src.zip in Scala
## What changes were proposed in this pull request?

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

## How was this patch tested?

N/A

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

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 21:23:17 -07:00
HyukjinKwon f5317f10b2 [SPARK-27893][SQL][PYTHON] Create an integrated test base for Python, Scalar Pandas, Scala UDF by sql files
## What changes were proposed in this pull request?

This PR targets to add an integrated test base for various UDF test cases so that Scalar UDF, Python UDF and Scalar Pandas UDFs can be tested in SBT & Maven tests.

### Problem

One of the problems we face is that: `ExtractPythonUDFs` (for Python UDF and Scalar Pandas UDF) has unevaluable expressions that always has to be wrapped with special plans. This special rule seems producing many issues, for instance, SPARK-27803, SPARK-26147, SPARK-26864, SPARK-26293, SPARK-25314 and SPARK-24721.

### Why do we have less test cases dedicated for SQL and plans with Python UDFs?

We have virtually no such SQL (or plan) dedicated tests in PySpark to catch such issues because:
  - A developer should know all the analyzer, the optimizer, SQL, PySpark, Py4J and version differences in Python to write such good test cases
  - To test plans, we should access to plans in JVM via Py4J which is tricky, messy and duplicates Scala test cases
  - Usually we just add end-to-end test cases in PySpark therefore there are not so many dedicated examples to refer to write in PySpark

It is also a non-trivial overhead to switch test base and method (IMHO).

### How does this PR fix?

This PR adds Python UDF and Scalar Pandas UDF into our `*.sql` file based test base in runtime of SBT / Maven test cases. It generates Python-pickled instance (consisting of return type and Python native function) that is used in Python or Scalar Pandas UDF and directly brings into JVM.

After that, (we don't interact via Py4J) run the tests directly in JVM - we can just register and run Python UDF and Scalar Pandas UDF in JVM.

Currently, I only integrated this change into SQL file based testing. This is how works with test files under `udf` directory:

After the test files under 'inputs/udf' directory are detected, it creates three test cases:
  - Scala UDF test case with a Scalar UDF registered named 'udf'.
  - Python UDF test case with a Python UDF registered named 'udf' iff Python executable and pyspark are available.
  - Scalar Pandas UDF test case with a Scalar Pandas UDF registered named 'udf' iff Python executable, pandas, pyspark and pyarrow are available.

Therefore, UDF test cases should have single input and output files but executed by three different types of UDFs.

For instance,

```sql
CREATE TEMPORARY VIEW ta AS
SELECT udf(a) AS a, udf('a') AS tag FROM t1
UNION ALL
SELECT udf(a) AS a, udf('b') AS tag FROM t2;

CREATE TEMPORARY VIEW tb AS
SELECT udf(a) AS a, udf('a') AS tag FROM t3
UNION ALL
SELECT udf(a) AS a, udf('b') AS tag FROM t4;

SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag;
```

will be ran 3 times with Scalar UDF, Python UDF and Scalar Pandas UDF each.

### Appendix

Plus, this PR adds `IntegratedUDFTestUtils` which enables to test and execute Python UDF and Scalar Pandas UDFs as below:

To register Python UDF in SQL:

```scala
IntegratedUDFTestUtils.registerTestUDF(TestPythonUDF(name = "udf"), spark)
```

To register Scalar Pandas UDF in SQL:

```scala
IntegratedUDFTestUtils.registerTestUDF(TestScalarPandasUDF(name = "udf"), spark)
```

 To use it in Scala API:

```scala
spark.select(expr("udf(1)").show()
```

 To use it in SQL:

```scala
sql("SELECT udf(1)").show()
```

This util could be used in the future for better coverage with Scala API combinations as well.

## How was this patch tested?

Tested via the command below:

```bash
build/sbt "sql/test-only *SQLQueryTestSuite -- -z udf/udf-inner-join.sql"
```

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (5 seconds, 47 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF (4 seconds, 335 milliseconds)
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF (5 seconds, 423 milliseconds)
```

[python] unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 577 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF is skipped because [pyton] and/or pyspark were not available. !!! IGNORED !!!
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [pyton]. !!! IGNORED !!!
```

pyspark unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 991 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF is skipped because [python] and/or pyspark were not available. !!! IGNORED !!!
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pyspark,pandas and/or pyarrow were not available in [python]. !!! IGNORED !!!
```

pandas and/or pyarrow unavailable:

```
[info] SQLQueryTestSuite:
[info] - udf/udf-inner-join.sql - Scala UDF (4 seconds, 713 milliseconds)
[info] - udf/udf-inner-join.sql - Python UDF (3 seconds, 89 milliseconds)
[info] - udf/udf-inner-join.sql - Scalar Pandas UDF is skipped because pandas and/or pyarrow were not available in [python]. !!! IGNORED !!!
```

Closes #24752 from HyukjinKwon/udf-tests.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-03 10:03:36 +09:00
gengjiaan 8feb80ad86 [SPARK-27811][CORE][DOCS] Improve docs about spark.driver.memoryOverhead and spark.executor.memoryOverhead.
## What changes were proposed in this pull request?

I found the docs of `spark.driver.memoryOverhead` and `spark.executor.memoryOverhead` exists a little ambiguity.
For example, the origin docs of `spark.driver.memoryOverhead` start with `The amount of off-heap memory to be allocated per driver in cluster mode`.
But `MemoryManager` also managed a memory area named off-heap used to allocate memory in tungsten mode.
So I think the description of `spark.driver.memoryOverhead` always make confused.

`spark.executor.memoryOverhead` has the same confused with `spark.driver.memoryOverhead`.

## How was this patch tested?

Exists UT.

Closes #24671 from beliefer/improve-docs-of-overhead.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-01 08:19:50 -05:00
Thomas Graves 1277f8fa92 [SPARK-27362][K8S] Resource Scheduling support for k8s
## What changes were proposed in this pull request?

Add ability to map the spark resource configs spark.{executor/driver}.resource.{resourceName} to kubernetes Container builder so that we request resources (gpu,s/fpgas/etc) from kubernetes.
Note that the spark configs will overwrite any resource configs users put into a pod template.
I added a generic vendor config which is only used by kubernetes right now.  I intentionally didn't put it into the kubernetes config namespace just to avoid adding more config prefixes.

I will add more documentation for this under jira SPARK-27492. I think it will be easier to do all at once to get cohesive story.

## How was this patch tested?

Unit tests and manually testing on k8s cluster.

Closes #24703 from tgravescs/SPARK-27362.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-31 15:26:14 -05:00
Izek Greenfield c647f9011c [SPARK-27862][BUILD] Move to json4s 3.6.6
## What changes were proposed in this pull request?
Move to json4s version 3.6.6
Add scala-xml 1.2.0

## How was this patch tested?

Pass the Jenkins

Closes #24736 from igreenfield/master.

Authored-by: Izek Greenfield <igreenfield@axiomsl.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-30 19:42:56 -05:00
Thomas Graves 6748b486a9 [SPARK-27835][CORE] Resource Scheduling: change driver config from addresses
## What changes were proposed in this pull request?

Change the Driver resource discovery argument for standalone mode to be a file rather then separate address configs per resource. This makes it consistent with how the Executor is doing it and makes it more flexible in the future, and it makes for less configs if you have multiple resources.

## How was this patch tested?

Unit tests and basic manually testing to make sure files were parsed properly.

Closes #24730 from tgravescs/SPARK-27835-driver-resourcesFile.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-30 07:51:06 -05:00
Yuming Wang db3e746b64 [SPARK-27875][CORE][SQL][ML][K8S] Wrap all PrintWriter with Utils.tryWithResource
## What changes were proposed in this pull request?

This pr wrap all `PrintWriter` with `Utils.tryWithResource` to prevent resource leak.

## How was this patch tested?

Existing test

Closes #24739 from wangyum/SPARK-27875.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-30 19:54:32 +09:00
ShuMingLi 7d2fec31e7 [MINOR][DOC] Fix spelling mistake
## What changes were proposed in this pull request?

`A very little spelling mistake.`

## How was this patch tested?
No

Closes #24710 from LiShuMing/minor-spelling.

Authored-by: ShuMingLi <ming.moriarty@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-29 18:21:47 -05:00
Aaruna bfa7f112e3 [SPARK-27869][CORE] Redact sensitive information in System Properties from UI
## What changes were proposed in this pull request?

Currently system properties are not redacted. This PR fixes that, so that any credentials passed as System properties are redacted as well.

## How was this patch tested?

Manual test. Run the following and see the UI.
```
bin/spark-shell --conf 'spark.driver.extraJavaOptions=-DMYSECRET=app'
```

Closes #24733 from aaruna/27869.

Authored-by: Aaruna <aaruna.godthi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-29 10:30:16 -07:00
wuyi 826ee6074e [SPARK-23191][CORE] Warn rather than terminate when duplicate worker register happens
## What changes were proposed in this pull request?

### Standalone HA Background

In Spark Standalone HA mode, we'll have multiple masters running at the same time. But, there's only one master leader, which actively serving scheduling requests. Once this master leader crashes, other masters would compete for the leader and only one master is guaranteed to be elected as new master leader, which would reconstruct the state from the original master leader and continute to serve scheduling requests.

### Related Issues

#2828 firstly introduces the bug of *duplicate Worker registration*, and #3447 fixed it. But there're still corner cases(see SPARK-23191 for details) where #3447 can not cover it:

* CASE 1
(1) Initially, Worker registered with Master A.
(2) After a while, the connection channel between Master A and Worker becomes inactive(e.g. due to network drop), and Worker is notified about that by calling `onDisconnected` from NettyRpcEnv
(3) When Worker invokes `onDisconnected`, then, it will attempt to reconnect to all masters(including Master A)
(4) At the meanwhile, network between Worker and Master A recover,  Worker successfully register to Master A again
(5) Master A response with `RegisterWorkerFailed("Duplicate worker ID")`
(6) Worker receives that msg, exit

* CASE 2
(1) Master A lost leadership(sends `RevokedLeadership` to itself). Master B takes over and recovery everything from master A(which would  register workers for the first time in Master B) and sends `MasterChanged` to Worker
(2) Before Master A receives `RevokedLeadership`, it receives a late `HeartBeat` from Worker(which had been removed in Master A due to heartbeat timeout previously), so it sends `ReconnectWorker`  to worker
(3) Worker receives `MasterChanged` before `ReconnectWorker` , changing masterRef to Master B
(4) Subsequently, Worker receives `ReconnectWorker` from Master A, then it reconnects to all masters
(5) Master B receives register request again from the Worker,  response with `RegisterWorkerFailed("Duplicate worker ID")`
(6) Worker receives that msg, exit

In CASE 1, it is difficult for the Worker to know Master A's state. Normally, Worker thinks Master A has already died and is impossible that Master A would response with Worker's re-connect request.

In CASE 2, we can see race condition between `RevokedLeadership` and `HeartBeat`. Actually, Master A has already been revoked leadership while processing `HeartBeat` msg. That's means the state between Master and Zookeeper could be out of sync for a while.

### Solutions

In this PR, instead of exiting Worker process when *duplicate Worker registration* happens, we suggest to log warn about it. This would be fine since Master actually perform no-op when it receives duplicate registration from a Worker. In turn, Worker could continue living with that Master normally without any side effect.

## How was this patch tested?

Tested Manually.

I followed the steps as  Neeraj Gupta suggested in JIRA SPARK-23191 to reproduce the case 1.

Before this pr, Worker would be DEAD from UI.
After this pr, Worker just warn the duplicate register behavior (as you can see the second last row in log snippet below), and still be ALIVE from UI.

```
19/05/09 20:58:32 ERROR Worker: Connection to master failed! Waiting for master to reconnect...
19/05/09 20:58:32 INFO Worker: wuyi.local:7077 Disassociated !
19/05/09 20:58:32 INFO Worker: Connecting to master wuyi.local:7077...
19/05/09 20:58:32 ERROR Worker: Connection to master failed! Waiting for master to reconnect...
19/05/09 20:58:32 INFO Worker: Not spawning another attempt to register with the master, since there is an attempt scheduled already.
19/05/09 20:58:37 WARN TransportClientFactory: DNS resolution for wuyi.local/127.0.0.1:7077 took 5005 ms
19/05/09 20:58:37 INFO TransportClientFactory: Found inactive connection to wuyi.local/127.0.0.1:7077, creating a new one.
19/05/09 20:58:37 INFO TransportClientFactory: Successfully created connection to wuyi.local/127.0.0.1:7077 after 3 ms (0 ms spent in bootstraps)
19/05/09 20:58:37 WARN Worker: Duplicate registration at master spark://wuyi.local:7077
19/05/09 20:58:37 INFO Worker: Successfully registered with master spark://wuyi.local:7077
```

Closes #24569 from Ngone51/fix-worker-dup-register-error.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-28 11:59:29 +08:00
Tom van Bussel 00a8c85a44 [SPARK-27071][CORE] Expose additional metrics in status.api.v1.StageData
## What changes were proposed in this pull request?

This PR exposes additional metrics in `status.api.v1.StageData`. These metrics were already computed for `LiveStage`, but they were never exposed to the user. This includes extra metrics about the JVM GC, executor (de)serialization times, shuffle reads and writes, and more.

## How was this patch tested?

Existing tests.

cc hvanhovell

Closes #24011 from tomvanbussel/SPARK-27071.

Authored-by: Tom van Bussel <tom.vanbussel@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-05-27 17:36:22 +02:00
Yuanjian Li 8949bc7a3c [SPARK-27665][CORE] Split fetch shuffle blocks protocol from OpenBlocks
## What changes were proposed in this pull request?

As the current approach in OneForOneBlockFetcher, we reuse the OpenBlocks protocol to describe the fetch request for shuffle blocks, and it causes the extension work for shuffle fetching like #19788 and #24110 very awkward.
In this PR, we split the fetch request for shuffle blocks from OpenBlocks which named FetchShuffleBlocks. It's a loose bind with ShuffleBlockId and can easily extend by adding new fields in this protocol.

## How was this patch tested?

Existing and new added UT.

Closes #24565 from xuanyuanking/SPARK-27665.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-27 22:19:31 +08:00
10087686 3df86a2581 [SPARK-27147][CORE][TEST] Add SortShuffleWriterSuite
## What changes were proposed in this pull request?
There are no unit test cases for this SortShuffleWriter,so add new test cases;

## How was this patch tested?
new test cases

Closes #24080 from wangjiaochun/UtestForSortShuffleWriter.

Authored-by: 10087686 <wang.jiaochun@zte.com.cn>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-25 21:03:10 -07:00
Dongjoon Hyun 0a1e01c30e [SPARK-27830][CORE][UI] Show Spark version at app lists of Spark History UI
## What changes were proposed in this pull request?

This PR aims to show Spark version at application lists of Spark History UI.

From the following, the first `Version` column is added. When the application has multiple attempts, this will show the first attempt's version number.

**COMPLETED APPLICATION LIST**
![Screen Shot 2019-05-23 at 11 42 39 PM](https://user-images.githubusercontent.com/9700541/58308045-49aae580-7db5-11e9-93bd-63cb2d359fb0.png)

**INCOMPLETE APPLICATION LIST**
![Screen Shot 2019-05-23 at 11 42 48 PM](https://user-images.githubusercontent.com/9700541/58308336-2c2a4b80-7db6-11e9-873a-2868b9dbb835.png)

## How was this patch tested?

Manually launch Spark history server and see the UI. Please use *Private New Window (Safari)* or *New Incognito Window (Chrome)* to avoid browser caching.

```
sbin/start-history-server.sh
```

Closes #24694 from dongjoon-hyun/SPARK-27830.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-24 17:40:46 -07:00
“attilapiros” 1e87694f2b [SPARK-27677][CORE] Disable by default fetching of disk persisted RDD blocks via external shuffle service
## What changes were proposed in this pull request?

In the PR the config `spark.shuffle.service.fetch.rdd.enabled` default is changed to **false** to avoid breaking any compatibility with older external shuffle service installations. As external shuffle service is deployed separately and disk persisted RDD block fetching had even introduced new network messages (`RemoveBlocks` and `BlocksRemoved`) and changed the behaviour of the already existing fetching: extended it for RDD blocks.

## How was this patch tested?

With existing unit tests.

Closes #24697 from attilapiros/minor-ext-shuffle-fetch-disabled.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-24 11:58:26 -07:00
“attilapiros” e9f3f62b2c [SPARK-27677][CORE] Serve local disk persisted blocks by the external service after releasing executor by dynamic allocation
# What changes were proposed in this pull request?

## Problem statement

An executor which has persisted blocks does not consider to be idle and this way ready to be released by dynamic allocation after the regular timeout `spark.dynamicAllocation.executorIdleTimeout` but there is separate configuration `spark.dynamicAllocation.cachedExecutorIdleTimeout` which defaults to `Integer.MAX_VALUE`. This is because releasing the executor also means losing the persisted blocks (as the metadata for individual blocks called `BlockInfo` are kept in memory) and when the RDD is referenced latter on this lost blocks will be recomputed.
On the other hand keeping the executors too long without any task to work on is also a waste of resources (as executors are reserved for the application by the resource manager).

## Solution

This PR focuses on the first part of SPARK-25888: it extends the external shuffle service with the capability to serve RDD blocks which are persisted on the local disk store by the executors. Moreover when this feature is enabled by setting the `spark.shuffle.service.fetch.rdd.enabled` config to true and a block is reported to be persisted on to disk the external shuffle service instance running on the same host as the executor is also registered (along with the reporting block manager) as a possible location for fetching it.

## Some implementation detail

Some explanation about the decisions made during the development:
- the location list to fetch a block was randomized but the groups (same host, same rack, others) order was kept. In this PR the order of groups are kept and external shuffle service added to the end of the each group.
- `BlockManagerInfo` is not introduced for external shuffle service but only a lightweight solution is taken. A hash map from `BlockId` to `BlockStatus` is introduced. A type alias would make the source more readable but I know it is discouraged. On the other hand a new class wrapping this hash map would introduce unnecessary indirection.
- when this feature is on the cleanup triggered during removing of executors (which is handled in `ExternalShuffleBlockResolver`) is modified to keep the disk persisted RDD blocks. This cleanup is triggered in standalone mode when the `spark.storage.cleanupFilesAfterExecutorExit` config is set.
- the unpersisting of an RDD is extended to use the external shuffle service for disk persisted RDD blocks when the original executor which created the blocks are already released. New block transport messages are introduced to support this: `RemoveBlocks` and `BlocksRemoved`.

# How was this patch tested?

## Unit tests

### ExternalShuffleServiceSuite

Here the complete use case is tested by the "SPARK-25888: using external shuffle service fetching disk persisted blocks" with a tiny difference: here the executor is killed manually, this way the test is a bit faster than waiting for the idle timeout.

### ExternalShuffleBlockHandlerSuite

Tests the fetching of the RDD blocks via the external shuffle service.

### BlockManagerInfoSuite

This a new suite. As the `BlockManagerInfo` behaviour depends very much on whether the external shuffle service enabled or not all the tests are executed with and without it.

### BlockManagerSuite

Tests the sorting of the block locations.

## Manually on YARN

Spark App was:

~~~scala
package com.mycompany

import org.apache.spark.rdd.RDD
import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.storage.StorageLevel

object TestAppDiskOnlyLevel {
  def main(args: Array[String]): Unit = {
    val conf = new SparkConf().setAppName("test-app")

    println("Attila: START")
    val sc = new SparkContext(conf)
    val rdd = sc.parallelize(0 until 100, 10)
      .map { i =>
        println(s"Attila: calculate first rdd i=$i")
        Thread.sleep(1000)
        i
      }

    rdd.persist(StorageLevel.DISK_ONLY)
    rdd.count()

    println("Attila: First RDD is processed, waiting for 60 sec")

    Thread.sleep(60 * 1000)

    println("Attila: Num executors must be 0 as executorIdleTimeout is way over")

    val rdd2 = sc.parallelize(0 until 10, 1)
      .map(i => (i, 1))
      .persist(StorageLevel.DISK_ONLY)

    rdd2.count()

    println("Attila: Second RDD with one partition (only one executors must be alive)")

    // reduce runs as user code to detect the empty seq (empty blocks)
    println("Calling collect on the first RDD: " + rdd.collect().reduce(_ + _))

    println("Attila: STOP")
  }
}
~~~

I have submitted with the following configuration:

~~~bash
spark-submit --master yarn \
  --conf spark.dynamicAllocation.enabled=true \
  --conf spark.dynamicAllocation.executorIdleTimeout=30 \
  --conf spark.dynamicAllocation.cachedExecutorIdleTimeout=90 \
  --class com.mycompany.TestAppDiskOnlyLevel dyn_alloc_demo-core_2.11-0.1.0-SNAPSHOT-jar-with-dependencies.jar
~~~

Checked the result by filtering for the side effect of the task calculations:

~~~bash
[userserver ~]$ yarn logs -applicationId application_1556299359453_0001 | grep "Attila: calculate" | wc -l
WARNING: YARN_OPTS has been replaced by HADOOP_OPTS. Using value of YARN_OPTS.
19/04/26 10:31:59 INFO client.RMProxy: Connecting to ResourceManager at apiros-1.gce.company.com/172.31.115.165:8032
100
~~~

So it is only 100 task execution and not 200 (which would be the case for re-computation).

Moreover from the submit/launcher log we can see executors really stopped in between (see the new total is 0 before the last line):
~~~
[userserver ~]$ grep "Attila: Num executors must be 0" -B 2 spark-submit.log
19/04/26 10:24:27 INFO cluster.YarnScheduler: Executor 9 on apiros-3.gce.company.com killed by driver.
19/04/26 10:24:27 INFO spark.ExecutorAllocationManager: Existing executor 9 has been removed (new total is 0)
Attila: Num executors must be 0 as executorIdleTimeout is way over
~~~

[Full spark submit log](https://github.com/attilapiros/spark/files/3122465/spark-submit.log)

I have done a test also after changing the `DISK_ONLY` storage level to `MEMORY_ONLY` for the first RDD. After this change during the 60sec waiting no executor was removed.

Closes #24499 from attilapiros/SPARK-25888-final.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-23 13:15:39 -07:00
Thomas Graves 74e5e41eeb [SPARK-27488][CORE] Driver interface to support GPU resources
## What changes were proposed in this pull request?

Added the driver functionality to get the resources.

The user interface is: SparkContext.resources  - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it.

There are 2 ways the driver can discover what resources it has.
  1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment.
  2) read the config spark.driver.resource.resourceName.addresses.  The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config.

When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X  where x is the number they want.  If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else.  We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user.  I'll wait for the jira that implements that to decide if we can remove.

- This PR also has changes to be consistent about using resourceName everywhere.
- change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark
- Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first.

## How was this patch tested?

Unit tests and manually test the interface.

Closes #24615 from tgravescs/SPARK-27488.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 11:46:13 -07:00
Jose Torres 5fae8f7b1d [SPARK-27711][CORE] Unset InputFileBlockHolder at the end of tasks
## What changes were proposed in this pull request?

Unset InputFileBlockHolder at the end of tasks to stop the file name from leaking over to other tasks in the same thread. This happens in particular in Pyspark because of its complex threading model.

## How was this patch tested?

new pyspark test

Closes #24605 from jose-torres/fix254.

Authored-by: Jose Torres <torres.joseph.f+github@gmail.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-05-22 18:35:50 -07:00
Stavros Kontopoulos 5e74570c8f [SPARK-23153][K8S] Support client dependencies with a Hadoop Compatible File System
## What changes were proposed in this pull request?
- solves the current issue with --packages in cluster mode (there is no ticket for it). Also note of some [issues](https://issues.apache.org/jira/browse/SPARK-22657) of the past here when hadoop libs are used at the spark submit side.
- supports spark.jars, spark.files, app jar.

It works as follows:
Spark submit uploads the deps to the HCFS. Then the driver serves the deps via the Spark file server.
No hcfs uris are propagated.

The related design document is [here](https://docs.google.com/document/d/1peg_qVhLaAl4weo5C51jQicPwLclApBsdR1To2fgc48/edit). the next option to add is the RSS but has to be improved given the discussion in the past about it (Spark 2.3).
## How was this patch tested?

- Run integration test suite.
- Run an example using S3:

```
 ./bin/spark-submit \
...
 --packages com.amazonaws:aws-java-sdk:1.7.4,org.apache.hadoop:hadoop-aws:2.7.6 \
 --deploy-mode cluster \
 --name spark-pi \
 --class org.apache.spark.examples.SparkPi \
 --conf spark.executor.memory=1G \
 --conf spark.kubernetes.namespace=spark \
 --conf spark.kubernetes.authenticate.driver.serviceAccountName=spark-sa \
 --conf spark.driver.memory=1G \
 --conf spark.executor.instances=2 \
 --conf spark.sql.streaming.metricsEnabled=true \
 --conf "spark.driver.extraJavaOptions=-Divy.cache.dir=/tmp -Divy.home=/tmp" \
 --conf spark.kubernetes.container.image.pullPolicy=Always \
 --conf spark.kubernetes.container.image=skonto/spark:k8s-3.0.0 \
 --conf spark.kubernetes.file.upload.path=s3a://fdp-stavros-test \
 --conf spark.hadoop.fs.s3a.access.key=... \
 --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \
 --conf spark.hadoop.fs.s3a.fast.upload=true \
 --conf spark.kubernetes.executor.deleteOnTermination=false \
 --conf spark.hadoop.fs.s3a.secret.key=... \
 --conf spark.files=client:///...resolv.conf \
file:///my.jar **
```
Added integration tests based on [Ceph nano](https://github.com/ceph/cn). Looks very [active](http://www.sebastien-han.fr/blog/2019/02/24/Ceph-nano-is-getting-better-and-better/).
Unfortunately minio needs hadoop >= 2.8.

Closes #23546 from skonto/support-client-deps.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Erik Erlandson <eerlands@redhat.com>
2019-05-22 16:15:42 -07:00
wenxuanguan e7443d6412 [SPARK-27774][CORE][MLLIB] Avoid hardcoded configs
## What changes were proposed in this pull request?

avoid hardcoded configs in `SparkConf` and `SparkSubmit` and test

## How was this patch tested?

N/A

Closes #24631 from wenxuanguan/minor-fix.

Authored-by: wenxuanguan <choose_home@126.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-22 10:45:11 +09:00
David Navas 9e73be38a5 [SPARK-27726][CORE] Fix performance of ElementTrackingStore deletes when using InMemoryStore under high loads
The details of the PR are explored in-depth in the sub-tasks of the umbrella jira SPARK-27726.
Briefly:
  1. Stop issuing asynchronous requests to cleanup elements in the tracking store when a request is already pending
  2. Fix a couple of thread-safety issues (mutable state and mis-ordered updates)
  3. Move Summary deletion outside of Stage deletion loop like Tasks already are
  4. Reimplement multi-delete in a removeAllKeys call which allows InMemoryStore to implement it in a performant manner.
  5. Some generic typing and exception handling cleanup

We see about five orders of magnitude improvement in the deletion code, which for us is the difference between a server that needs restarting daily, and one that is stable over weeks.

Unit tests for the fire-once asynchronous code and the removeAll calls in both LevelDB and InMemoryStore are supplied.  It was noted that the testing code for the LevelDB and InMemoryStore is highly repetitive, and should probably be merged, but we did not attempt that in this PR.

A version of this code was run in our production 2.3.3 and we were able to sustain higher throughput without going into GC overload (which was happening on a daily basis some weeks ago).

A version of this code was also put under a purpose-built Performance Suite of tests to verify performance under both types of Store implementations for both before and after code streams and for both total and partial delete cases (this code is not included in this PR).

Closes #24616 from davidnavas/PentaBugFix.

Authored-by: David Navas <davidn@clearstorydata.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-21 10:22:21 -07:00
hustfeiwang d90c460c48 [SPARK-27637][SHUFFLE] For nettyBlockTransferService, if IOException occurred while fetching data, check whether relative executor is alive before retry
## What changes were proposed in this pull request?

There are several kinds of shuffle client, blockTransferService and externalShuffleClient.

For the externalShuffleClient,  there are relative external shuffle service, which guarantees  the shuffle block data and regardless the  state of executors.

For the blockTransferService, it is used to fetch broadcast block, and fetch the shuffle data when external shuffle service is not enabled.

When fetching data by using blockTransferService, the shuffle client would connect relative executor's blockManager, so if the relative executor is dead, it would never fetch successfully.

When spark.shuffle.service.enabled is true and spark.dynamicAllocation.enabled is true,  the executor will be removed while it has been idle  for more than idleTimeout.

If a blockTransferService create connection to relative executor successfully, but the relative executor is removed when beginning to fetch broadcast block, it would retry (see RetryingBlockFetcher), which is Ineffective.

If the spark.shuffle.io.retryWait and spark.shuffle.io.maxRetries is big,  such as 30s and 10 times, it would waste 5 minutes.

In this PR, we check whether relative executor is alive before retry.
## How was this patch tested?
Unit test.

Closes #24533 from turboFei/SPARK-27637.

Authored-by: hustfeiwang <wangfei3@corp.netease.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-21 13:45:42 +08:00
Gabor Somogyi efa303581a [SPARK-27687][SS] Rename Kafka consumer cache capacity conf and document caching
## What changes were proposed in this pull request?

Kafka related Spark parameters has to start with `spark.kafka.` and not with `spark.sql.`. Because of this I've renamed `spark.sql.kafkaConsumerCache.capacity`.

Since Kafka consumer caching is not documented I've added this also.

## How was this patch tested?

Existing + added unit test.

```
cd docs
SKIP_API=1 jekyll build
```
and manual webpage check.

Closes #24590 from gaborgsomogyi/SPARK-27687.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-15 10:42:09 -07:00
Marcelo Vanzin d14e2d7874 [SPARK-27678][UI] Allow user impersonation in the UI.
This feature allows proxy servers to identify the actual request user
using a request parameter, and performs access control checks against
that user instead of the authenticated user. Impersonation is only
allowed if the authenticated user is configured as an admin.

The request parameter used ("doAs") matches the one currently used by
Knox, but it should be easy to change / customize if different proxy
servers use a different way of identifying the original user.

Tested with updated unit tests and also with a live server behind Knox.

Closes #24582 from vanzin/SPARK-27678.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-15 09:58:12 -07:00
Sean Owen bfb3ffe9b3 [SPARK-27682][CORE][GRAPHX][MLLIB] Replace use of collections and methods that will be removed in Scala 2.13 with work-alikes
## What changes were proposed in this pull request?

This replaces use of collection classes like `MutableList` and `ArrayStack` with workalikes that are available in 2.12, as they will be removed in 2.13. It also removes use of `.to[Collection]` as its uses was superfluous anyway. Removing `collection.breakOut` will have to wait until 2.13

## How was this patch tested?

Existing tests

Closes #24586 from srowen/SPARK-27682.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-15 09:29:12 -05:00
Sean Owen a10608cb82 [SPARK-27680][CORE][SQL][GRAPHX] Remove usage of Traversable
## What changes were proposed in this pull request?

This removes usage of `Traversable`, which is removed in Scala 2.13. This is mostly an internal change, except for the change in the `SparkConf.setAll` method. See additional comments below.

## How was this patch tested?

Existing tests.

Closes #24584 from srowen/SPARK-27680.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-14 09:14:56 -05:00
pgandhi 695dbe27ce [SPARK-25719][UI] : Search functionality in datatables in stages page should search over formatted data rather than the raw data
The Pull Request to add datatables to stage page SPARK-21809 got merged. The search functionality in those datatables being a great improvement for searching through a large number of tasks, also performs search over the raw data rather than the formatted data displayed in the tables. It would be great if the search can happen for the formatted data as well.

## What changes were proposed in this pull request?

Added code to enable searching over displayed data in tables e.g. searching on "165.7 MiB" or "0.3 ms" will now return the search results. Also, earlier we were missing search for two columns in the task table "Shuffle Read Bytes" as well as "Shuffle Remote Reads", which I have added here.

## How was this patch tested?

Manual Tests

Closes #24419 from pgandhi999/SPARK-25719.

Authored-by: pgandhi <pgandhi@verizonmedia.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-14 09:05:13 -05:00
Thomas Graves db2e3c4341 [SPARK-27024] Executor interface for cluster managers to support GPU and other resources
## What changes were proposed in this pull request?

Add in GPU and generic resource type allocation to the executors.

Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow :

   - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this.
  -  cluster manager allocates a container or set of resources (standalone mode)
-    When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them.
  -  Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources

In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects.

The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use.

Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615

## How was this patch tested?

Added unit tests and manually tested.

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

Closes #24406 from tgravescs/gpu-sched-executor-clean.

Authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 08:41:41 -05:00
jiafu.zhang@intel.com fa5dc0a45a [SPARK-26632][CORE] Separate Thread Configurations of Driver and Executor
## What changes were proposed in this pull request?

For the below three thread configuration items applied to both driver and executor,
spark.rpc.io.serverThreads
spark.rpc.io.clientThreads
spark.rpc.netty.dispatcher.numThreads,
we separate them to driver specifics and executor specifics.
spark.driver.rpc.io.serverThreads                     < - > spark.executor.rpc.io.serverThreads
spark.driver.rpc.io.clientThreads                      < - > spark.executor.rpc.io.clientThreads
spark.driver.rpc.netty.dispatcher.numThreads < - > spark.executor.rpc.netty.dispatcher.numThreads

Spark reads these specifics first and fall back to the common configurations.

## How was this patch tested?
We ran the SimpleMap app without shuffle for benchmark purpose to test Spark's scalability in HPC with omini-path NIC which has higher bandwidth than normal ethernet NIC.

Spark's base version is 2.4.0.
Spark ran in the Standalone mode. Driver was in a standalone node.
After the separation, the performance is improved a lot in 256 nodes and 512 nodes. see below test results of SimpleMapTask before and after the enhancement. You can view the tables in the  [JIRA](https://issues.apache.org/jira/browse/SPARK-26632) too.

ds: spark.driver.rpc.io.serverThreads
dc: spark.driver.rpc.io.clientThreads
dd: spark.driver.rpc.netty.dispatcher.numThreads
ed: spark.executor.rpc.netty.dispatcher.numThreads
time: Overall Time (s)
old time: Overall Time without Separation (s)

**Before:**

 nodes | ds | dc | dd | ed | time
-- |-- | -- | -- | -- | --
128 nodes | 8 | 8 | 8 | 8 | 108
256 nodes | 8 | 8 | 8 | 8 | 196
512 nodes | 8 | 8 | 8 | 8 | 377

**After:**

nodes | ds | dc | dd | ed | time | improvement
-- | -- | -- | -- | -- | -- | --
128 nodes | 15 | 15 | 10 | 30 | 107 | 0.9%
256 nodes | 12 | 15 | 10 | 30 | 159 | 18.8%
512 nodes | 12 | 15 | 10 | 30 | 283 | 24.9%

Closes #23560 from zjf2012/thread_conf_separation.

Authored-by: jiafu.zhang@intel.com <jiafu.zhang@intel.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-10 10:42:43 -07:00
Xingbo Jiang e63fbfcf20 [SPARK-25139][SPARK-18406][CORE] Avoid NonFatals to kill the Executor in PythonRunner
## What changes were proposed in this pull request?

Python uses a prefetch approach to read the result from upstream and serve them in another thread, thus it's possible that if the children operator doesn't consume all the data then the Task cleanup may happen before Python side read process finishes, this in turn create a race condition that the block read locks are freed during Task cleanup and then the reader try to release the read lock it holds and find it has been released, in this case we shall hit a AssertionError.

We shall catch the AssertionError in PythonRunner and prevent this kill the Executor.

## How was this patch tested?

Hard to write a unit test case for this case, manually verified with failed job.

Closes #24542 from jiangxb1987/pyError.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-08 13:47:26 +09:00
Bryan Cutler 5e79ae3b40 [SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?

This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.

Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.

The change here introduces a protocol for PySpark local iterators that work as follows:

1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.

Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.

## How was this patch tested?

Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7  and 3.6.

Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 14:47:39 -07:00
Gabor Somogyi 2f55809425 [SPARK-27294][SS] Add multi-cluster Kafka delegation token
## What changes were proposed in this pull request?

The actual implementation doesn't support multi-cluster Kafka connection with delegation token. In this PR I've added this functionality.

What this PR contains:
* New way of configuration
* Multiple delegation token obtain/store/use functionality
* Documentation
* The change works on DStreams also

## How was this patch tested?

Existing + additional unit tests.
Additionally tested on cluster.

Test scenario:

* 2 * 4 node clusters
* The 4-4 nodes are in different kerberos realms
* Cross-Realm trust between the 2 realms
* Yarn
* Kafka broker version 2.1.0
* security.protocol = SASL_SSL
* sasl.mechanism = SCRAM-SHA-512
* Artificial exceptions during processing
* Source reads from realm1 sink writes to realm2

Kafka broker settings:

* delegation.token.expiry.time.ms=600000 (10 min)
* delegation.token.max.lifetime.ms=1200000 (20 min)
* delegation.token.expiry.check.interval.ms=300000 (5 min)

Closes #24305 from gaborgsomogyi/SPARK-27294.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-07 11:40:43 -07:00
Adi Muraru 8ef4da753d [SPARK-27610][YARN] Shade netty native libraries
## What changes were proposed in this pull request?

Fixed the `spark-<version>-yarn-shuffle.jar` artifact packaging to shade the native netty libraries:
- shade the `META-INF/native/libnetty_*` native libraries when packagin
the yarn shuffle service jar. This is required as netty library loader
derives that based on shaded package name.
- updated the `org/spark_project` shade package prefix to `org/sparkproject`
(i.e. removed underscore) as the former breaks the netty native lib loading.

This was causing the yarn external shuffle service to fail
when spark.shuffle.io.mode=EPOLL

## How was this patch tested?
Manual tests

Closes #24502 from amuraru/SPARK-27610_master.

Authored-by: Adi Muraru <amuraru@adobe.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-07 10:47:36 -07:00
Wenchen Fan d124ce9c7e [SPARK-27590][CORE] do not consider skipped tasks when scheduling speculative tasks
## What changes were proposed in this pull request?

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

When `TaskSetManager` skips a task because its corresponding partition is already completed by other `TaskSetManager`s, we should not consider the duration of the task that is finished by other `TaskSetManager`s to schedule the speculative tasks of this `TaskSetManager`.

## How was this patch tested?

updated test case

Closes #24485 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-05-07 12:02:08 -05:00
Liang-Chi Hsieh d9bcacf94b [SPARK-27629][PYSPARK] Prevent Unpickler from intervening each unpickling
## What changes were proposed in this pull request?

In SPARK-27612, one correctness issue was reported. When protocol 4 is used to pickle Python objects, we found that unpickled objects were wrong. A temporary fix was proposed by not using highest protocol.

It was found that Opcodes.MEMOIZE was appeared in the opcodes in protocol 4. It is suspect to this issue.

A deeper dive found that Opcodes.MEMOIZE stores objects into internal map of Unpickler object. We use single Unpickler object to unpickle serialized Python bytes. Stored objects intervenes next round of unpickling, if the map is not cleared.

We has two options:

1. Continues to reuse Unpickler, but calls its close after each unpickling.
2. Not to reuse Unpickler and create new Unpickler object in each unpickling.

This patch takes option 1.

## How was this patch tested?

Passing the test added in SPARK-27612 (#24519).

Closes #24521 from viirya/SPARK-27629.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 13:21:08 +09:00
wuyi 51de86baed [SPARK-27510][CORE] Avoid Master falls into dead loop while launching executor failed in Worker
## What changes were proposed in this pull request?

This is a long standing issue which I met before and I've seen other people got trouble with it:
[test cases stuck on "local-cluster mode" of ReplSuite?](http://apache-spark-developers-list.1001551.n3.nabble.com/test-cases-stuck-on-quot-local-cluster-mode-quot-of-ReplSuite-td3086.html)
[Spark tests hang on local machine due to "testGuavaOptional" in JavaAPISuite](http://apache-spark-developers-list.1001551.n3.nabble.com/Spark-tests-hang-on-local-machine-due-to-quot-testGuavaOptional-quot-in-JavaAPISuite-tc10999.html)

When running test under local-cluster mode with wrong SPARK_HOME(spark.test.home), test just get stuck and no response forever. After looking into SPARK_WORKER_DIR, I found there's endless executor directories under it. So, this explains what happens during test getting stuck.

The whole process looks like:

1. Driver submits an app to Master and asks for N executors
2. Master inits executor state with LAUNCHING and sends `LaunchExecutor` to Worker
3. Worker receives `LaunchExecutor`, launches ExecutorRunner asynchronously and sends `ExecutorStateChanged(state=RUNNING)` to Mater immediately
4. Master receives `ExecutorStateChanged(state=RUNNING)` and reset `_retyCount` to 0.
5. ExecutorRunner throws exception during executor launching, sends `ExecutorStateChanged(state=FAILED)` to Worker, Worker forwards the msg to Master
6. Master receives `ExecutorStateChanged(state=FAILED)`. Since Master always reset `_retyCount` when it receives RUNNING msg, so, event if a Worker fails to launch executor for continuous many times, ` _retryCount` would never exceed `maxExecutorRetries`. So, Master continue to launch executor and fall into the dead loop.

The problem exists in step 3. Worker sends `ExecutorStateChanged(state=RUNNING)` to Master immediately while executor is still launching. And, when Master receive that msg, it believes the executor has launched successfully, and reset `_retryCount` subsequently. However, that's not true.

This pr suggests to remove step 3 and requires Worker only send `ExecutorStateChanged(state=RUNNING)` after executor has really launched successfully.

## How was this patch tested?

Tested Manually.

Closes #24408 from Ngone51/fix-dead-loop.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-05-03 15:48:37 -07:00
Xingbo Jiang f950e53930 [MINOR][CORE] Update taskName in PythonRunner
## What changes were proposed in this pull request?

Update taskName in PythonRunner so it keeps align with that in Executor.

## How was this patch tested?

N/A

Closes #24510 from jiangxb1987/pylog.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-05-02 15:37:25 -07:00
Jungtaek Lim (HeartSaVioR) 75b40a53d3 [SPARK-27575][CORE][YARN] Yarn file-related confs should merge new value with existing value
## What changes were proposed in this pull request?

This patch fixes a bug which YARN file-related configurations are being overwritten when there're some values to assign - e.g. if `--file` is specified as an argument, `spark.yarn.dist.files` is overwritten with the value of argument. After this patch the existing value and new value will be merged before assigning to the value of configuration.

## How was this patch tested?

Added UT, and manually tested with below command:

> ./bin/spark-submit --verbose --files /etc/spark2/conf/spark-defaults.conf.template --master yarn-cluster --class org.apache.spark.examples.SparkPi examples/jars/spark-examples_2.11-2.4.0.jar 10

where the spark conf file has

`spark.yarn.dist.files=file:/etc/spark2/conf/atlas-application.properties.yarn#atlas-application.properties`

```
Spark config:
...
(spark.yarn.dist.files,file:/etc/spark2/conf/atlas-application.properties.yarn#atlas-application.properties,file:///etc/spark2/conf/spark-defaults.conf.template)
...
```

Closes #24465 from HeartSaVioR/SPARK-27575.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-29 10:14:59 -07:00
Gabor Somogyi fb6b19ab7c [SPARK-23014][SS] Fully remove V1 memory sink.
## What changes were proposed in this pull request?

There is a MemorySink v2 already so v1 can be removed. In this PR I've removed it completely.
What this PR contains:
* V1 memory sink removal
* V2 memory sink renamed to become the only implementation
* Since DSv2 sends exceptions in a chained format (linking them with cause field) I've made python side compliant
* Adapted all the tests

## How was this patch tested?

Existing unit tests.

Closes #24403 from gaborgsomogyi/SPARK-23014.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-29 09:44:23 -07:00
Sean Owen a6716d3f03 [SPARK-27571][CORE][YARN][EXAMPLES] Avoid scala.language.reflectiveCalls
## What changes were proposed in this pull request?

This PR avoids usage of reflective calls in Scala. It removes the import that suppresses the warnings and rewrites code in small ways to avoid accessing methods that aren't technically accessible.

## How was this patch tested?

Existing tests.

Closes #24463 from srowen/SPARK-27571.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:16:45 -05:00
Sean Owen 8a17d26784 [SPARK-27536][CORE][ML][SQL][STREAMING] Remove most use of scala.language.existentials
## What changes were proposed in this pull request?

I want to get rid of as much use of `scala.language.existentials` as possible for 3.0. It's a complicated language feature that generates warnings unless this value is imported. It might even be on the way out of Scala: https://contributors.scala-lang.org/t/proposal-to-remove-existential-types-from-the-language/2785

For Spark, it comes up mostly where the code plays fast and loose with generic types, not the advanced situations you'll often see referenced where this feature is explained. For example, it comes up in cases where a function returns something like `(String, Class[_])`. Scala doesn't like matching this to any other instance of `(String, Class[_])` because doing so requires inferring the existence of some type that satisfies both. Seems obvious if the generic type is a wildcard, but, not technically something Scala likes to let you get away with.

This is a large PR, and it only gets rid of _most_ instances of `scala.language.existentials`. The change should be all compile-time and shouldn't affect APIs or logic.

Many of the changes simply touch up sloppiness about generic types, making the known correct value explicit in the code.

Some fixes involve being more explicit about the existence of generic types in methods. For instance, `def foo(arg: Class[_])` seems innocent enough but should really be declared `def foo[T](arg: Class[T])` to let Scala select and fix a single type when evaluating calls to `foo`.

For kind of surprising reasons, this comes up in places where code evaluates a tuple of things that involve a generic type, but is OK if the two parts of the tuple are evaluated separately.

One key change was altering `Utils.classForName(...): Class[_]` to the more correct `Utils.classForName[T](...): Class[T]`. This caused a number of small but positive changes to callers that otherwise had to cast the result.

In several tests, `Dataset[_]` was used where `DataFrame` seems to be the clear intent.

Finally, in a few cases in MLlib, the return type `this.type` was used where there are no subclasses of the class that uses it. This really isn't needed and causes issues for Scala reasoning about the return type. These are just changed to be concrete classes as return types.

After this change, we have only a few classes that still import `scala.language.existentials` (because modifying them would require extensive rewrites to fix) and no build warnings.

## How was this patch tested?

Existing tests.

Closes #24431 from srowen/SPARK-27536.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:02:01 -05:00
Wenchen Fan 05b85eb8cb [SPARK-27474][CORE] avoid retrying a task failed with CommitDeniedException many times
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-25250 reports a bug that, a task which is failed with `CommitDeniedException` gets retried many times.

This can happen when a stage has 2 task set managers, one is zombie, one is active. A task from the zombie TSM completes, and commits to a central coordinator(assuming it's a file writing task). Then the corresponding task from the active TSM will fail with `CommitDeniedException`. `CommitDeniedException.countTowardsTaskFailures` is false, so the active TSM will keep retrying this task, until the job finishes. This wastes resource a lot.

#21131 firstly implements that a previous successful completed task from zombie `TaskSetManager` could mark the task of the same partition completed in the active `TaskSetManager`. Later #23871 improves the implementation to cover a corner case that, an active `TaskSetManager` hasn't been created when a previous task succeed.

However, #23871 has a bug and was reverted in #24359. With hindsight, #23781 is fragile because we need to sync the states between `DAGScheduler` and `TaskScheduler`, about which partitions are completed.

This PR proposes a new fix:
1. When `DAGScheduler` gets a task success event from an earlier attempt, notify the `TaskSchedulerImpl` about it
2. When `TaskSchedulerImpl` knows a partition is already completed, ask the active `TaskSetManager` to mark the corresponding task as finished, if the task is not finished yet.

This fix covers the corner case, because:
1. If `DAGScheduler` gets the task completion event from zombie TSM before submitting the new stage attempt, then `DAGScheduler` knows that this partition is completed, and it will exclude this partition when creating task set for the new stage attempt. See `DAGScheduler.submitMissingTasks`
2. If `DAGScheduler` gets the task completion event from zombie TSM after submitting the new stage attempt, then the active TSM is already created.

Compared to the previous fix, the message loop becomes longer, so it's likely that, the active task set manager has already retried the task multiple times. But this failure window won't be too big, and we want to avoid the worse case that retries the task many times until the job finishes. So this solution is acceptable.

## How was this patch tested?

a new test case.

Closes #24375 from cloud-fan/fix2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-29 14:20:58 +08:00
gatorsmile cd4a284030 [SPARK-27460][FOLLOW-UP][TESTS] Fix flaky tests
## What changes were proposed in this pull request?

This patch makes several test flakiness fixes.

## How was this patch tested?
N/A

Closes #24434 from gatorsmile/fixFlakyTest.

Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-24 17:36:29 +08:00
Sean Owen 596a5ff273 [MINOR][BUILD] Update genjavadoc to 0.13
## What changes were proposed in this pull request?

Kind of related to https://github.com/gatorsmile/spark/pull/5 - let's update genjavadoc to see if it generates fewer spurious javadoc errors to begin with.

## How was this patch tested?

Existing docs build

Closes #24443 from srowen/genjavadoc013.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-24 13:44:48 +09:00
uncleGen ecfdffcb35 [SPARK-27503][DSTREAM] JobGenerator thread exit for some fatal errors but application keeps running
## What changes were proposed in this pull request?

In some corner cases, `JobGenerator` thread (including some other EventLoop threads) may exit for some fatal error, like OOM, but Spark Streaming job keep running with no batch job generating. Currently, we only report any non-fatal error.

```
override def run(): Unit = {
      try {
        while (!stopped.get) {
          val event = eventQueue.take()
          try {
            onReceive(event)
          } catch {
            case NonFatal(e) =>
              try {
                onError(e)
              } catch {
                case NonFatal(e) => logError("Unexpected error in " + name, e)
              }
          }
        }
      } catch {
        case ie: InterruptedException => // exit even if eventQueue is not empty
        case NonFatal(e) => logError("Unexpected error in " + name, e)
      }
    }
```

In this PR, we double check if event thread alive when post Event

## How was this patch tested?

existing unit tests

Closes #24400 from uncleGen/SPARK-27503.

Authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-23 07:11:58 -07:00
Shixiong Zhu 009059e3c2 [SPARK-27496][CORE] Fatal errors should also be sent back to the sender
## What changes were proposed in this pull request?

When a fatal error (such as StackOverflowError) throws from "receiveAndReply", we should try our best to notify the sender. Otherwise, the sender will hang until timeout.

In addition, when a MessageLoop is dying unexpectedly, it should resubmit a new one so that Dispatcher is still working.

## How was this patch tested?

New unit tests.

Closes #24396 from zsxwing/SPARK-27496.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-21 17:00:07 -07:00
Shahid 16bbe0f798 [SPARK-27486][CORE][TEST] Enable History server storage information test in the HistoryServerSuite
## What changes were proposed in this pull request?

We have disabled a test related to storage in the History server suite after SPARK-13845. But, after SPARK-22050, we can store the information about block updated events to eventLog, if we enable "spark.eventLog.logBlockUpdates.enabled=true".

   So, we can enable the test, by adding an eventlog corresponding to the application, which has enabled the configuration, "spark.eventLog.logBlockUpdates.enabled=true"

## How was this patch tested?
Existing UTs

Closes #24390 from shahidki31/enableRddStorageTest.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-19 08:12:20 -07:00
pengbo 54b0d1e0ef [SPARK-27416][SQL] UnsafeMapData & UnsafeArrayData Kryo serialization …
## What changes were proposed in this pull request?
Finish the rest work of https://github.com/apache/spark/pull/24317, https://github.com/apache/spark/pull/9030
a. Implement Kryo serialization for UnsafeArrayData
b. fix UnsafeMapData Java/Kryo Serialization issue when two machines have different Oops size
c. Move the duplicate code "getBytes()" to Utils.

## How was this patch tested?
According Units has been added & tested

Closes #24357 from pengbo/SPARK-27416_new.

Authored-by: pengbo <bo.peng1019@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 13:03:00 +08:00
shivusondur 88d9de26dd [SPARK-27464][CORE] Added Constant instead of referring string literal used from many places
## What changes were proposed in this pull request?

Added Constant instead of referring the same String literal "spark.buffer.pageSize" from many places
## How was this patch tested?
Run the corresponding Unit Test Cases manually.

Closes #24368 from shivusondur/Constant.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-16 09:30:46 -05:00
Kazuaki Ishizaki 257d01a6b8 [SPARK-27397][CORE] Take care of OpenJ9 JVM in Spark
## What changes were proposed in this pull request?

This PR supports `OpenJ9` in addition to `IBM JDK` and `OpenJDK` in Spark by handling `System.getProperty("java.vendor") = "Eclipse OpenJ9"`.

In `inferDefaultMemory()` and `getKrb5LoginModuleName()`, this PR uses non `IBM` way.

```
$ ~/jdk-11.0.2+9_openj9-0.12.1/bin/jshell
|  Welcome to JShell -- Version 11.0.2
|  For an introduction type: /help intro

jshell> System.out.println(System.getProperty("java.vendor"))
Eclipse OpenJ9

jshell> System.out.println(System.getProperty("java.vm.info"))
JRE 11 Linux amd64-64-Bit Compressed References 20190204_127 (JIT enabled, AOT enabled)
OpenJ9   - 90dd8cb40
OMR      - d2f4534b
JCL      - 289c70b6844 based on jdk-11.0.2+9

jshell> System.out.println(Class.forName("com.ibm.lang.management.OperatingSystemMXBean").getDeclaredMethod("getTotalPhysicalMemory"))
public abstract long com.ibm.lang.management.OperatingSystemMXBean.getTotalPhysicalMemory()

jshell> System.out.println(Class.forName("com.sun.management.OperatingSystemMXBean").getDeclaredMethod("getTotalPhysicalMemorySize"))
public abstract long com.sun.management.OperatingSystemMXBean.getTotalPhysicalMemorySize()

jshell> System.out.println(Class.forName("com.ibm.security.auth.module.Krb5LoginModule"))
|  Exception java.lang.ClassNotFoundException: com.ibm.security.auth.module.Krb5LoginModule
|        at Class.forNameImpl (Native Method)
|        at Class.forName (Class.java:339)
|        at (#1:1)

jshell> System.out.println(Class.forName("com.sun.security.auth.module.Krb5LoginModule"))
class com.sun.security.auth.module.Krb5LoginModule
```

## How was this patch tested?

Existing test suites
Manual testing with OpenJ9.

Closes #24308 from kiszk/SPARK-27397.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-16 09:11:47 -05:00
Sean Owen 8718367e2e [SPARK-27470][PYSPARK] Update pyrolite to 4.23
## What changes were proposed in this pull request?

 Update pyrolite to 4.23 to pick up bug and security fixes.

## How was this patch tested?

Existing tests.

Closes #24381 from srowen/SPARK-27470.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-16 19:41:40 +09:00
Wenchen Fan 0bb716bac3 Revert [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions
## What changes were proposed in this pull request?

Our customer has a very complicated job. Sometimes it successes and sometimes it fails with
```
Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: ShuffleMapStage 4  has failed the maximum allowable number of times: 4.
Most recent failure reason: org.apache.spark.shuffle.FetchFailedException
```

However, with the patch https://github.com/apache/spark/pull/23871 , the job hangs forever.

When I investigated it, I found that `DAGScheduler` and `TaskSchedulerImpl` define stage completion differently. `DAGScheduler` thinks a stage is completed if all its partitions are marked as completed ([result stage](https://github.com/apache/spark/blob/v2.4.1/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1362-L1368) and [shuffle stage](https://github.com/apache/spark/blob/v2.4.1/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1400)). `TaskSchedulerImpl` thinks a stage's task set is completed when all tasks finish (see the [code](https://github.com/apache/spark/blob/v2.4.1/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L779-L784)).

Ideally this two definition should be consistent, but #23871 breaks it. In our customer's Spark log, I found that, a stage's task set completes, but the stage never completes. More specifically, `DAGScheduler` submits a task set for stage 4.1 with 1000 tasks, but the `TaskSetManager` skips to run the first 100 tasks. Later on, `TaskSetManager` finishes 900 tasks and marks the task set as completed. However, `DAGScheduler` doesn't agree with it and hangs forever, waiting for more task completion events of stage 4.1.

With hindsight, I think `TaskSchedulerIImpl.stageIdToFinishedPartitions` is fragile. We need to pay more effort to make sure this is consistent with `DAGScheduler`'s knowledge. When `DAGScheduler` marks some partitions from finished to unfinished, `TaskSchedulerIImpl.stageIdToFinishedPartitions` should be updated as well.

This PR reverts #23871, let's think of a more robust idea later.

## How was this patch tested?

N/A

Closes #24359 from cloud-fan/revert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-14 16:57:41 +08:00
Bago Amirbekian eea3f55a31 [SPARK-27446][R] Use existing spark conf if available.
## What changes were proposed in this pull request?

The RBackend and RBackendHandler create new conf objects that don't pick up conf values from the existing SparkSession and therefore always use the default conf values instead of values specified by the user.

In this fix we check to see if the spark env already exists, and get the conf from there. We fall back to creating a new conf. This follows the pattern used in other places including this: 3725b1324f/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala (L261)

## How was this patch tested?

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

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

Closes #24353 from MrBago/r-backend-use-existing-conf.

Authored-by: Bago Amirbekian <bago@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-14 17:09:12 +09:00
Sean Owen 4ec7f631aa [SPARK-27404][CORE][SQL][STREAMING][YARN] Fix build warnings for 3.0: postfixOps edition
## What changes were proposed in this pull request?

Fix build warnings -- see some details below.

But mostly, remove use of postfix syntax where it causes warnings without the `scala.language.postfixOps` import. This is mostly in expressions like "120000 milliseconds". Which, I'd like to simplify to things like "2.minutes" anyway.

## How was this patch tested?

Existing tests.

Closes #24314 from srowen/SPARK-27404.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-11 13:43:44 -05:00
Maxim Gekk d33ae2e9ed [SPARK-26953][CORE][TEST] Disable result checking in the test: java.lang.ArrayIndexOutOfBoundsException in TimSort
## What changes were proposed in this pull request?

I propose to disable (comment) result checking in `SorterSuite`.`java.lang.ArrayIndexOutOfBoundsException in TimSort` because:
1. The check is optional, and correctness of TimSort is checked by another tests. Purpose of the test is to check that TimSort doesn't fail with `ArrayIndexOutOfBoundsException`.
2. Significantly drops execution time of the test.

Here are timing of running the test locally:
```
Sort: 1.4 seconds
Result checking: 15.6 seconds
```

## How was this patch tested?

By `SorterSuite`.

Closes #24343 from MaxGekk/timsort-test-speedup.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-11 07:58:57 -07:00
Shixiong Zhu 5ff39cd5ee [SPARK-27394][WEBUI] Flush LiveEntity if necessary when receiving SparkListenerExecutorMetricsUpdate
## What changes were proposed in this pull request?

This PR updates `AppStatusListener` to flush `LiveEntity` if necessary when receiving `SparkListenerExecutorMetricsUpdate`. This will ensure the staleness of Spark UI doesn't last more than the executor heartbeat interval.

## How was this patch tested?

The new unit test.

Closes #24303 from zsxwing/SPARK-27394.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-09 08:26:00 -07:00
LantaoJin 52838e74af [SPARK-13704][CORE][YARN] Reduce rack resolution time
## What changes were proposed in this pull request?

When you submit a stage on a large cluster, rack resolving takes a long time when initializing TaskSetManager because a script is invoked to resolve the rack of each host, one by one.
Based on current implementation, it takes 30~40 seconds to resolve the racks in our 5000 nodes' cluster. After applied the patch, it decreased to less than 15 seconds.

YARN-9332 has added an interface to handle multiple hosts in one invocation to save time. But before upgrading to the newest Hadoop, we could construct the same tool in Spark to resolve this issue.

## How was this patch tested?

UT and manually testing on a 5000 node cluster.

Closes #24245 from squito/SPARK-13704_update.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-04-08 10:47:06 -05:00
liulijia 39f75b4588 [SPARK-27192][CORE] spark.task.cpus should be less or equal than spark.executor.cores
## What changes were proposed in this pull request?
check spark.task.cpus before creating TaskScheduler in SparkContext

## How was this patch tested?
UT

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

Closes #24261 from liutang123/SPARK-27192.

Authored-by: liulijia <liutang123@yeah.net>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-05 13:55:57 -05:00
Dongjoon Hyun 982c4c8e3c [SPARK-27390][CORE][SQL][TEST] Fix package name mismatch
## What changes were proposed in this pull request?

This PR aims to clean up package name mismatches.

## How was this patch tested?

Pass the Jenkins.

Closes #24300 from dongjoon-hyun/SPARK-27390.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-05 11:50:37 -07:00
Sean Owen 23bde44797 [SPARK-27358][UI] Update jquery to 1.12.x to pick up security fixes
## What changes were proposed in this pull request?

Update jquery -> 1.12.4, datatables -> 1.10.18, mustache -> 2.3.12.
Add missing mustache license

## How was this patch tested?

I manually tested the UI locally with the javascript console open and didn't observe any problems or JS errors. The only 'risky' change seems to be mustache, but on reading its release notes, don't think the changes from 0.8.1 to 2.x would affect Spark's simple usage.

Closes #24288 from srowen/SPARK-27358.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-05 12:54:01 -05:00
Wenchen Fan b56e433b54 [SPARK-27338][CORE][FOLLOWUP] remove trailing space
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/24265 breaks the lint check, because it has trailing space. (not sure why it passed jenkins). This PR fixes it.

## How was this patch tested?

N/A

Closes #24289 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 11:43:20 +08:00
Venkata krishnan Sowrirajan 6c4552c650 [SPARK-27338][CORE] Fix deadlock in UnsafeExternalSorter.SpillableIterator when locking both UnsafeExternalSorter.SpillableIterator and TaskMemoryManager
## What changes were proposed in this pull request?

In `UnsafeExternalSorter.SpillableIterator#loadNext()` takes lock on the `UnsafeExternalSorter` and calls `freePage` once the `lastPage` is consumed which needs to take a lock on `TaskMemoryManager`. At the same time, there can be another MemoryConsumer using `UnsafeExternalSorter` as part of sorting can try to `allocatePage` needs to get lock on `TaskMemoryManager` which can cause spill to happen which requires lock on `UnsafeExternalSorter` again causing deadlock. This is a classic deadlock situation happening similar to the SPARK-26265.

To fix this, we can move the `freePage` call in `loadNext` outside of `Synchronized` block similar to the fix in SPARK-26265

## How was this patch tested?

Manual tests were being done and will also try to add a test.

Closes #24265 from venkata91/deadlock-sorter.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@qubole.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 09:58:05 +08:00
LantaoJin 69dd44af19 [SPARK-27216][CORE] Upgrade RoaringBitmap to 0.7.45 to fix Kryo unsafe ser/dser issue
## What changes were proposed in this pull request?

HighlyCompressedMapStatus uses RoaringBitmap to record the empty blocks. But RoaringBitmap couldn't be ser/deser with unsafe KryoSerializer.

It's a bug of RoaringBitmap-0.5.11 and fixed in latest version.

This is an update of #24157

## How was this patch tested?

Add a UT

Closes #24264 from LantaoJin/SPARK-27216.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: Lantao Jin <jinlantao@gmail.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-04-03 20:09:50 -05:00
Gabor Somogyi 57aff93886 [SPARK-26998][CORE] Remove SSL configuration from executors
## What changes were proposed in this pull request?

Different SSL passwords shown up as command line argument on executor side in standalone mode:
* keyStorePassword
* keyPassword
* trustStorePassword

In this PR I've removed SSL configurations from executors.

## How was this patch tested?

Existing + additional unit tests.
Additionally tested with standalone mode and checked the command line arguments:
```
[gaborsomogyi:~/spark] SPARK-26998(+4/-0,3)+ ± jps
94803 CoarseGrainedExecutorBackend
94818 Jps
90149 RemoteMavenServer
91925 Nailgun
94793 SparkSubmit
94680 Worker
94556 Master
398
[gaborsomogyi:~/spark] SPARK-26998(+4/-1,3)+ ± ps -ef | egrep "94556|94680|94793|94803"
  502 94556     1   0  2:02PM ttys007    0:07.39 /Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/bin/java -cp /Users/gaborsomogyi/spark/conf/:/Users/gaborsomogyi/spark/assembly/target/scala-2.12/jars/* -Xmx1g org.apache.spark.deploy.master.Master --host gsomogyi-MBP.local --port 7077 --webui-port 8080 --properties-file conf/spark-defaults.conf
  502 94680     1   0  2:02PM ttys007    0:07.27 /Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/bin/java -cp /Users/gaborsomogyi/spark/conf/:/Users/gaborsomogyi/spark/assembly/target/scala-2.12/jars/* -Xmx1g org.apache.spark.deploy.worker.Worker --webui-port 8081 --properties-file conf/spark-defaults.conf spark://gsomogyi-MBP.local:7077
  502 94793 94782   0  2:02PM ttys007    0:35.52 /Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/bin/java -cp /Users/gaborsomogyi/spark/conf/:/Users/gaborsomogyi/spark/assembly/target/scala-2.12/jars/* -Dscala.usejavacp=true -Xmx1g org.apache.spark.deploy.SparkSubmit --master spark://gsomogyi-MBP.local:7077 --class org.apache.spark.repl.Main --name Spark shell spark-shell
  502 94803 94680   0  2:03PM ttys007    0:05.20 /Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/bin/java -cp /Users/gaborsomogyi/spark/conf/:/Users/gaborsomogyi/spark/assembly/target/scala-2.12/jars/* -Xmx1024M -Dspark.ssl.ui.port=0 -Dspark.driver.port=60902 org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url spark://CoarseGrainedScheduler172.30.65.186:60902 --executor-id 0 --hostname 172.30.65.186 --cores 8 --app-id app-20190326140311-0000 --worker-url spark://Worker172.30.65.186:60899
  502 94910 57352   0  2:05PM ttys008    0:00.00 egrep 94556|94680|94793|94803
```

Closes #24170 from gaborgsomogyi/SPARK-26998.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-02 09:18:43 -07:00
Sean Owen d4420b455a [SPARK-27323][CORE][SQL][STREAMING] Use Single-Abstract-Method support in Scala 2.12 to simplify code
## What changes were proposed in this pull request?

Use Single Abstract Method syntax where possible (and minor related cleanup). Comments below. No logic should change here.

## How was this patch tested?

Existing tests.

Closes #24241 from srowen/SPARK-27323.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-02 07:37:05 -07:00
“attilapiros” 9eb896cc3b [SPARK-27333][TEST] Update thread audit whitelist to skip broadcast-exchange-.*, process reaper and StatisticsDataReferenceCleaner threads
## What changes were proposed in this pull request?

Update thread audit whitelist to skip threads of the global broadcast exchange thread pool, process reaper and Hadoop FS statistics data reference cleaner thread.

## How was this patch tested?

Via existing UT using broadcast exchange via `sbt` i.e:

```
> project sql
> testOnly *.SessionStateSuite -- -z "fork new sessions and run query on inherited table"
```

Before (wrapped long line for manually to save horizontal scrolling for reviewers):

```
===== POSSIBLE THREAD LEAK IN SUITE o.a.s.sql.SessionStateSuite,
thread names: broadcast-exchange-6, broadcast-exchange-0,
broadcast-exchange-2, broadcast-exchange-5, broadcast-exchange-7,
broadcast-exchange-4, broadcast-exchange-1, process reaper, broadcast-exchange-3,
 org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner =====
```

After this change no possible thread leak detected.

Closes #24244 from attilapiros/thread-audit-minor.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-31 17:33:31 -07:00
gatorsmile 92b6f86f6d [SPARK-27244][CORE][TEST][FOLLOWUP] toDebugString redacts sensitive information
## What changes were proposed in this pull request?
This PR is a FollowUp of https://github.com/apache/spark/pull/24196. It improves the test case by using the parameters that are being used in the actual scenarios.

## How was this patch tested?
N/A

Closes #24257 from gatorsmile/followupSPARK-27244.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-30 22:58:28 -07:00
Sean Owen 754f820035 [SPARK-26918][DOCS] All .md should have ASF license header
## What changes were proposed in this pull request?

Add AL2 license to metadata of all .md files.
This seemed to be the tidiest way as it will get ignored by .md renderers and other tools. Attempts to write them as markdown comments revealed that there is no such standard thing.

## How was this patch tested?

Doc build

Closes #24243 from srowen/SPARK-26918.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-30 19:49:45 -05:00
Dongjoon Hyun 88ea319871 Revert "[SPARK-27192][CORE] spark.task.cpus should be less or equal than spark.executor.cores"
This reverts commit f8fa564dec.
2019-03-30 16:35:34 -07:00
liulijia f8fa564dec [SPARK-27192][CORE] spark.task.cpus should be less or equal than spark.executor.cores
## What changes were proposed in this pull request?
spark.task.cpus should be less or equal than spark.executor.cores when use static executor allocation
## How was this patch tested?
manual

Closes #24131 from liutang123/SPARK-27192.

Authored-by: liulijia <liutang123@yeah.net>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-30 12:38:05 -05:00
Ninad Ingole dbc7ce18b9 [SPARK-27244][CORE] Redact Passwords While Using Option logConf=true
## What changes were proposed in this pull request?

When logConf is set to true, config keys that contain password were printed in cleartext in driver log. This change uses the already present redact method in Utils, to redact all the passwords based on redact pattern in SparkConf and then print the conf to driver log thus ensuring that sensitive information like passwords is not printed in clear text.

## How was this patch tested?
This patch was tested through `SparkConfSuite` & then entire unit test through sbt

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

Closes #24196 from ninadingole/SPARK-27244.

Authored-by: Ninad Ingole <robert.wallis@example.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-29 14:16:53 -05:00
Wenchen Fan e4a968d829 [MINOR][CORE] Remove import scala.collection.Set in TaskSchedulerImpl
## What changes were proposed in this pull request?

I was playing with the scheduler and found this weird thing. In `TaskSchedulerImpl` we import `scala.collection.Set` without any reason. This is bad in practice, as it silently changes the actual class when we simply type `Set`, which by default should point to the immutable set.

This change only affects one method: `getExecutorsAliveOnHost`. I checked all the caller side and none of them need a general `Set` type.

## How was this patch tested?

N/A

Closes #24231 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-28 21:12:18 +09:00
Sean Owen 3a8398df5c [SPARK-26660][FOLLOWUP] Raise task serialized size warning threshold to 1000 KiB
## What changes were proposed in this pull request?

Raise the threshold size for serialized task size at which a warning is generated from 100KiB to 1000KiB.

As several people have noted, the original change for this JIRA highlighted that this threshold is low. Test output regularly shows:

```
- sorting on StringType with nullable=false, sortOrder=List('a DESC NULLS LAST)
22:47:53.320 WARN org.apache.spark.scheduler.TaskSetManager: Stage 80 contains a task of very large size (755 KiB). The maximum recommended task size is 100 KiB.
22:47:53.348 WARN org.apache.spark.scheduler.TaskSetManager: Stage 81 contains a task of very large size (755 KiB). The maximum recommended task size is 100 KiB.
22:47:53.417 WARN org.apache.spark.scheduler.TaskSetManager: Stage 83 contains a task of very large size (755 KiB). The maximum recommended task size is 100 KiB.
22:47:53.444 WARN org.apache.spark.scheduler.TaskSetManager: Stage 84 contains a task of very large size (755 KiB). The maximum recommended task size is 100 KiB.

...

- SPARK-20688: correctly check analysis for scalar sub-queries
22:49:10.314 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 150.8 KiB
- SPARK-21835: Join in correlated subquery should be duplicateResolved: case 1
22:49:10.595 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 150.7 KiB
22:49:10.744 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 150.7 KiB
22:49:10.894 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 150.7 KiB
- SPARK-21835: Join in correlated subquery should be duplicateResolved: case 2
- SPARK-21835: Join in correlated subquery should be duplicateResolved: case 3
- SPARK-23316: AnalysisException after max iteration reached for IN query
22:49:11.559 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 154.2 KiB
```

It seems that a larger threshold of about 1MB is more suitable.

## How was this patch tested?

Existing tests.

Closes #24226 from srowen/SPARK-26660.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-27 10:42:26 +09:00
Stavros Kontopoulos 05168e725d [SPARK-24793][K8S] Enhance spark-submit for app management
- supports `--kill` & `--status` flags.
- supports globs which is useful in general check this long standing [issue](https://github.com/kubernetes/kubernetes/issues/17144#issuecomment-272052461) for kubectl.

Manually against running apps. Example output:

Submission Id reported at launch time:

```
2019-01-20 23:47:56 INFO  Client:58 - Waiting for application spark-pi with submissionId spark:spark-pi-1548020873671-driver to finish...
```

Killing the app:

```
./bin/spark-submit --kill spark:spark-pi-1548020873671-driver --master  k8s://https://192.168.2.8:8443
2019-01-20 23:48:07 WARN  Utils:70 - Your hostname, universe resolves to a loopback address: 127.0.0.1; using 192.168.2.8 instead (on interface wlp2s0)
2019-01-20 23:48:07 WARN  Utils:70 - Set SPARK_LOCAL_IP if you need to bind to another address

```

App terminates with 143 (SIGTERM, since we have tiny this should lead to [graceful shutdown](https://cloud.google.com/solutions/best-practices-for-building-containers)):

```
2019-01-20 23:48:08 INFO  LoggingPodStatusWatcherImpl:58 - State changed, new state:
	 pod name: spark-pi-1548020873671-driver
	 namespace: spark
	 labels: spark-app-selector -> spark-e4730c80e1014b72aa77915a2203ae05, spark-role -> driver
	 pod uid: 0ba9a794-1cfd-11e9-8215-a434d9270a65
	 creation time: 2019-01-20T21:47:55Z
	 service account name: spark-sa
	 volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm
	 node name: minikube
	 start time: 2019-01-20T21:47:55Z
	 phase: Running
	 container status:
		 container name: spark-kubernetes-driver
		 container image: skonto/spark:k8s-3.0.0
		 container state: running
		 container started at: 2019-01-20T21:48:00Z
2019-01-20 23:48:09 INFO  LoggingPodStatusWatcherImpl:58 - State changed, new state:
	 pod name: spark-pi-1548020873671-driver
	 namespace: spark
	 labels: spark-app-selector -> spark-e4730c80e1014b72aa77915a2203ae05, spark-role -> driver
	 pod uid: 0ba9a794-1cfd-11e9-8215-a434d9270a65
	 creation time: 2019-01-20T21:47:55Z
	 service account name: spark-sa
	 volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm
	 node name: minikube
	 start time: 2019-01-20T21:47:55Z
	 phase: Failed
	 container status:
		 container name: spark-kubernetes-driver
		 container image: skonto/spark:k8s-3.0.0
		 container state: terminated
		 container started at: 2019-01-20T21:48:00Z
		 container finished at: 2019-01-20T21:48:08Z
		 exit code: 143
		 termination reason: Error
2019-01-20 23:48:09 INFO  LoggingPodStatusWatcherImpl:58 - Container final statuses:
	 container name: spark-kubernetes-driver
	 container image: skonto/spark:k8s-3.0.0
	 container state: terminated
	 container started at: 2019-01-20T21:48:00Z
	 container finished at: 2019-01-20T21:48:08Z
	 exit code: 143
	 termination reason: Error
2019-01-20 23:48:09 INFO  Client:58 - Application spark-pi with submissionId spark:spark-pi-1548020873671-driver finished.
2019-01-20 23:48:09 INFO  ShutdownHookManager:58 - Shutdown hook called
2019-01-20 23:48:09 INFO  ShutdownHookManager:58 - Deleting directory /tmp/spark-f114b2e0-5605-4083-9203-a4b1c1f6059e

```

Glob scenario:

```
./bin/spark-submit --status spark:spark-pi* --master  k8s://https://192.168.2.8:8443
2019-01-20 22:27:44 WARN  Utils:70 - Your hostname, universe resolves to a loopback address: 127.0.0.1; using 192.168.2.8 instead (on interface wlp2s0)
2019-01-20 22:27:44 WARN  Utils:70 - Set SPARK_LOCAL_IP if you need to bind to another address
Application status (driver):
	 pod name: spark-pi-1547948600328-driver
	 namespace: spark
	 labels: spark-app-selector -> spark-f13f01702f0b4503975ce98252d59b94, spark-role -> driver
	 pod uid: c576e1c6-1c54-11e9-8215-a434d9270a65
	 creation time: 2019-01-20T01:43:22Z
	 service account name: spark-sa
	 volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm
	 node name: minikube
	 start time: 2019-01-20T01:43:22Z
	 phase: Running
	 container status:
		 container name: spark-kubernetes-driver
		 container image: skonto/spark:k8s-3.0.0
		 container state: running
		 container started at: 2019-01-20T01:43:27Z
Application status (driver):
	 pod name: spark-pi-1547948792539-driver
	 namespace: spark
	 labels: spark-app-selector -> spark-006d252db9b24f25b5069df357c30264, spark-role -> driver
	 pod uid: 38375b4b-1c55-11e9-8215-a434d9270a65
	 creation time: 2019-01-20T01:46:35Z
	 service account name: spark-sa
	 volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm
	 node name: minikube
	 start time: 2019-01-20T01:46:35Z
	 phase: Succeeded
	 container status:
		 container name: spark-kubernetes-driver
		 container image: skonto/spark:k8s-3.0.0
		 container state: terminated
		 container started at: 2019-01-20T01:46:39Z
		 container finished at: 2019-01-20T01:46:56Z
		 exit code: 0
		 termination reason: Completed

```

Closes #23599 from skonto/submit_ops_extension.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-26 11:55:03 -07:00
Ajith b61dce23d2 [SPARK-26961][CORE] Enable parallel classloading capability
## What changes were proposed in this pull request?

As per https://docs.oracle.com/javase/8/docs/api/java/lang/ClassLoader.html
``Class loaders that support concurrent loading of classes are known as parallel capable class loaders and are required to register themselves at their class initialization time by invoking the ClassLoader.registerAsParallelCapable method. Note that the ClassLoader class is registered as parallel capable by default. However, its subclasses still need to register themselves if they are parallel capable. ``
 i.e we can have finer class loading locks by registering classloaders as parallel capable.  (Refer to deadlock due to macro lock  https://issues.apache.org/jira/browse/SPARK-26961).
All the classloaders we have are wrapper of URLClassLoader which by itself is parallel capable.
But this cannot be achieved by scala code due to static registration Refer https://github.com/scala/bug/issues/11429

## How was this patch tested?

All Existing UT must pass

Closes #24126 from ajithme/driverlock.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 19:07:30 -05:00
liuxian e4b36df2c0 [SPARK-27256][CORE][SQL] If the configuration is used to set the number of bytes, we'd better use bytesConf'.
## What changes were proposed in this pull request?
Currently, if we want to configure `spark.sql.files.maxPartitionBytes` to 256 megabytes, we must set  `spark.sql.files.maxPartitionBytes=268435456`, which is very unfriendly to users.

And if we set it like this:`spark.sql.files.maxPartitionBytes=256M`, we will  encounter this exception:
```
Exception in thread "main" java.lang.IllegalArgumentException:
 spark.sql.files.maxPartitionBytes should be long, but was 256M
        at org.apache.spark.internal.config.ConfigHelpers$.toNumber(ConfigBuilder.scala)
```
This PR use `bytesConf` to replace `longConf` or `intConf`,  if the configuration is used to set the number of bytes.
Configuration change list:
`spark.files.maxPartitionBytes`
`spark.files.openCostInBytes`
`spark.shuffle.sort.initialBufferSize`
`spark.shuffle.spill.initialMemoryThreshold`
`spark.sql.autoBroadcastJoinThreshold`
`spark.sql.files.maxPartitionBytes`
`spark.sql.files.openCostInBytes`
`spark.sql.defaultSizeInBytes`
## How was this patch tested?
1.Existing unit tests
2.Manual testing

Closes #24187 from 10110346/bytesConf.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-25 14:47:40 -07:00
Luca Canali 4b2b3da766 [SPARK-26928][CORE][FOLLOWUP] Fix JVMCPUSource file name and minor updates to doc
## What changes were proposed in this pull request?

This applies some minor updates/cleaning following up SPARK-26928, notably renaming JVMCPU.scala to JVMCPUSource.scala.

## How was this patch tested?

Manually tested

Closes #24201 from LucaCanali/fixupSPARK-26928.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 15:35:24 -05:00
Sean Owen 8bc304f97e [SPARK-26132][BUILD][CORE] Remove support for Scala 2.11 in Spark 3.0.0
## What changes were proposed in this pull request?

Remove Scala 2.11 support in build files and docs, and in various parts of code that accommodated 2.11. See some targeted comments below.

## How was this patch tested?

Existing tests.

Closes #23098 from srowen/SPARK-26132.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 10:46:42 -05:00
Maxim Gekk 027ed2d11b [SPARK-23643][CORE][SQL][ML] Shrinking the buffer in hashSeed up to size of the seed parameter
## What changes were proposed in this pull request?

The hashSeed method allocates 64 bytes instead of 8. Other bytes are always zeros (thanks to default behavior of ByteBuffer). And they could be excluded from hash calculation because they don't differentiate inputs.

## How was this patch tested?

By running the existing tests - XORShiftRandomSuite

Closes #20793 from MaxGekk/hash-buff-size.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-23 11:26:09 -05:00
hehuiyuan 68abf77b1a [SPARK-27184][CORE] Avoid hardcoded 'spark.jars', 'spark.files', 'spark.submit.pyFiles' and 'spark.submit.deployMode'
## What changes were proposed in this pull request?

For [SPARK-27184](https://issues.apache.org/jira/browse/SPARK-27184)

In the `org.apache.spark.internal.config`, we define the variables of `FILES` and `JARS`, we can use them instead of "spark.jars" and "spark.files".

```scala
private[spark] val JARS = ConfigBuilder("spark.jars")
  .stringConf
  .toSequence
  .createWithDefault(Nil)
```

```scala
private[spark] val FILES = ConfigBuilder("spark.files")
  .stringConf
  .toSequence
  .createWithDefault(Nil)
```

Other :
In the `org.apache.spark.internal.config`, we define the variables of `SUBMIT_PYTHON_FILES ` and `SUBMIT_DEPLOY_MODE `, we can use them instead of "spark.submit.pyFiles" and "spark.submit.deployMode".
```scala
private[spark] val SUBMIT_PYTHON_FILES = ConfigBuilder("spark.submit.pyFiles")
    .stringConf
    .toSequence
    .createWithDefault(Nil)

```
```scala
private[spark] val SUBMIT_DEPLOY_MODE = ConfigBuilder("spark.submit.deployMode")
    .stringConf
    .createWithDefault("client")
```

Closes #24123 from hehuiyuan/hehuiyuan-patch-6.

Authored-by: hehuiyuan <hehuiyuan@ZBMAC-C02WD3K5H.local>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-23 09:43:00 +09:00
Jungtaek Lim (HeartSaVioR) 8a9eb05137 [SPARK-26606][CORE] Handle driver options properly when submitting to standalone cluster mode via legacy Client
## What changes were proposed in this pull request?

This patch fixes the issue that ClientEndpoint in standalone cluster doesn't recognize about driver options which are passed to SparkConf instead of system properties. When `Client` is executed via cli they should be provided as system properties, but with `spark-submit` they can be provided as SparkConf. (SpartSubmit will call `ClientApp.start` with SparkConf which would contain these options.)

## How was this patch tested?

Manually tested via following steps:

1) setup standalone cluster (launch master and worker via `./sbin/start-all.sh`)

2) submit one of example app with standalone cluster mode

```
./bin/spark-submit --class org.apache.spark.examples.SparkPi --master "spark://localhost:7077" --conf "spark.driver.extraJavaOptions=-Dfoo=BAR" --deploy-mode "cluster" --num-executors 1 --driver-memory 512m --executor-memory 512m --executor-cores 1 examples/jars/spark-examples*.jar 10
```

3) check whether `foo=BAR` is provided in system properties in Spark UI

<img width="877" alt="Screen Shot 2019-03-21 at 8 18 04 AM" src="https://user-images.githubusercontent.com/1317309/54728501-97db1700-4bc1-11e9-89da-078445c71e9b.png">

Closes #24163 from HeartSaVioR/SPARK-26606.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-22 15:07:49 -07:00
Jungtaek Lim (HeartSaVioR) 174531c183 [MINOR][CORE] Leverage modified Utils.classForName to reduce scalastyle off for Class.forName
## What changes were proposed in this pull request?

This patch modifies Utils.classForName to have optional parameters - initialize, noSparkClassLoader - to let callers of Class.forName with thread context classloader to use it instead. This helps to reduce scalastyle off for Class.forName.

## How was this patch tested?

Existing UTs.

Closes #24148 from HeartSaVioR/MINOR-reduce-scalastyle-off-for-class-forname.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-22 05:28:46 -05:00
maryannxue 9f58d3b436 [SPARK-27236][TEST] Refactor log-appender pattern in tests
## What changes were proposed in this pull request?

Refactored code in tests regarding the "withLogAppender()" pattern by creating a general helper method in SparkFunSuite.

## How was this patch tested?

Passed existing tests.

Closes #24172 from maryannxue/log-appender.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-21 19:18:30 -07:00
Jungtaek Lim (HeartSaVioR) a8d9531edc [SPARK-27205][CORE] Remove complicated logic for just leaving warning log when main class is scala.App
## What changes were proposed in this pull request?

[SPARK-26977](https://issues.apache.org/jira/browse/SPARK-26977) introduced very strange bug which spark-shell is no longer able to load classes which are provided via `--packages`. TBH I don't know about the details why it is broken, but looks like initializing `object class` brings the weirdness (maybe due to static initialization done twice?).

This patch removes the logic to leave warning log when main class is scala.App, to not deal with such complexity for just leaving warning message.

## How was this patch tested?

Manual test: suppose we run spark-shell with `--packages` option like below:

```
./bin/spark-shell --verbose   --master "local[*]" --packages org.apache.spark:spark-sql-kafka-0-10_2.11:2.4.0
```

Before this patch, importing class in transitive dependency fails:

```
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-1553005771597).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_191)
Type in expressions to have them evaluated.
Type :help for more information.

scala> import org.apache.kafka
<console>:23: error: object kafka is not a member of package org.apache
       import org.apache.kafka
```

After this patch, importing class in transitive dependency succeeds:

```
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-1553004095542).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_191)
Type in expressions to have them evaluated.
Type :help for more information.

scala> import org.apache.kafka
import org.apache.kafka
```

Closes #24147 from HeartSaVioR/SPARK-27205.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-20 17:55:48 -05:00
Lantao Jin 93c6d2a198 [SPARK-27215][CORE] Correct the kryo configurations
## What changes were proposed in this pull request?

```scala
val KRYO_USE_UNSAFE = ConfigBuilder("spark.kyro.unsafe")
    .booleanConf
    .createWithDefault(false)

  val KRYO_USE_POOL = ConfigBuilder("spark.kyro.pool")
    .booleanConf
    .createWithDefault(true)
```
**kyro should be kryo**

## How was this patch tested?

no need

Closes #24156 from LantaoJin/SPARK-27215.

Authored-by: Lantao Jin <jinlantao@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-20 14:27:05 -07:00
Gengliang Wang ef2d63bfb1 [SPARK-27201][WEBUI] Toggle full job description on click
## What changes were proposed in this pull request?

Previously, in https://github.com/apache/spark/pull/6646 there was an improvement to show full job description after double clicks.
I think this is a bit hard to be noticed by some users. I suggest changing the event to one click.
Also, after the full description is shown, another click should be able to hide the overflow text again.

Before click:
![short](https://user-images.githubusercontent.com/1097932/54608784-79bfca80-4a8c-11e9-912b-30799be0d6cb.png)

After click:
![full](https://user-images.githubusercontent.com/1097932/54608790-7b898e00-4a8c-11e9-9251-86061158db68.png)

Click again:
![short](https://user-images.githubusercontent.com/1097932/54608784-79bfca80-4a8c-11e9-912b-30799be0d6cb.png)

## How was this patch tested?

Manually check.

Closes #24145 from gengliangwang/showDescriptionDetail.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 21:29:13 +09:00
Ajith 1f692e522c [SPARK-27200][WEBUI][HISTORYSERVER] History Environment tab must sort Configurations/Properties by default
Environment Page in SparkUI have all the configuration sorted by key. But this is not the case in History server case, to keep UX same, we can have it sorted in history server too

## What changes were proposed in this pull request?

On render of Env page the properties are sorted before creating page

## How was this patch tested?

Manually tested in UI

Closes #24143 from ajithme/historyenv.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 20:16:17 +09:00
weixiuli 8b0aa59218 [SPARK-26288][CORE] add initRegisteredExecutorsDB
## What changes were proposed in this pull request?

As we all know that spark on Yarn uses DB https://github.com/apache/spark/pull/7943 to record RegisteredExecutors information which can be reloaded and used again when the ExternalShuffleService is restarted .

The RegisteredExecutors information can't be recorded both in the mode of spark's standalone and spark on k8s , which will cause the RegisteredExecutors information to be lost ,when the ExternalShuffleService is restarted.

To solve the problem above, a method is proposed and is committed .

## How was this patch tested?
new  unit tests

Closes #23393 from weixiuli/SPARK-26288.

Authored-by: weixiuli <weixiuli@jd.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-19 16:16:43 -05:00
pgandhi 7043aee1ba [SPARK-27112][CORE] : Create a resource ordering between threads to resolve the deadlocks encountered …
…when trying to kill executors either due to dynamic allocation or blacklisting

## What changes were proposed in this pull request?

There are two deadlocks as a result of the interplay between three different threads:

**task-result-getter thread**

**spark-dynamic-executor-allocation thread**

**dispatcher-event-loop thread(makeOffers())**

The fix ensures ordering synchronization constraint by acquiring lock on `TaskSchedulerImpl` before acquiring lock on `CoarseGrainedSchedulerBackend` in `makeOffers()` as well as killExecutors() method. This ensures resource ordering between the threads and thus, fixes the deadlocks.

## How was this patch tested?

Manual Tests

Closes #24072 from pgandhi999/SPARK-27112-2.

Authored-by: pgandhi <pgandhi@verizonmedia.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-18 10:33:51 -05:00
Ajith fc88d3df5c [SPARK-27164][CORE] RDD.countApprox on empty RDDs schedules jobs which never complete
## What changes were proposed in this pull request?

When Result stage has zero tasks, the Job End event is never fired, hence the Job is always running in UI. Example: sc.emptyRDD[Int].countApprox(1000) never finishes even it has no tasks to launch

## How was this patch tested?

Added UT

Closes #24100 from ajithme/emptyRDD.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 12:56:41 -05:00
fitermay 1bc481b779 [SPARK-27070] Improve performance of DefaultPartitionCoalescer
This time tested against Scala 2.11 as well

Closes #24116 from fitermay/master.

Authored-by: fitermay <fiterman@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 11:47:14 -05:00
Ajith c324e1da9d [SPARK-27122][CORE] Jetty classes must not be return via getters in org.apache.spark.ui.WebUI
## What changes were proposed in this pull request?

When we run YarnSchedulerBackendSuite, the class path seems to be made from the classes folder(resource-managers/yarn/target/scala-2.12/classes) instead of jar (resource-managers/yarn/target/spark-yarn_2.12-3.0.0-SNAPSHOT.jar) . ui.getHandlers is in spark-core and its loaded from spark-core.jar which is shaded and hence refers to org.spark_project.jetty.servlet.ServletContextHandler

Here in  org.apache.spark.scheduler.cluster.YarnSchedulerBackend, as its not shaded, it expects org.eclipse.jetty.servlet.ServletContextHandler
Refer discussion  https://issues.apache.org/jira/browse/SPARK-27122?focusedCommentId=16792318&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16792318

Hence as a fix, org.apache.spark.ui.WebUI must only return a wrapper class instance or references so that Jetty classes can be avoided in getters which are accessed outside spark-core

## How was this patch tested?

Existing UT can pass

Closes #24088 from ajithme/shadebug.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 06:44:02 -05:00
lichaoqun 4132c989db [MINOR][CORE] spark.diskStore.subDirectories <= 0 should throw Exception
## What changes were proposed in this pull request?
this pr add check this spark.diskStore.subDirectories > 0.This value need to be checked before it can be used.

## How was this patch tested?
N/A

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

Closes #24024 from lcqzte10192193/wid-lcq-190308.

Authored-by: lichaoqun <li.chaoqun@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 06:43:14 -05:00
Liupengcheng cad475dcc9 [SPARK-26941][YARN] Fix incorrect computation of maxNumExecutorFailures in ApplicationMaster for streaming
## What changes were proposed in this pull request?

Currently, when enabled streaming dynamic allocation for streaming applications, the maxNumExecutorFailures in ApplicationMaster is still computed with `spark.dynamicAllocation.maxExecutors`.

Actually, we should consider `spark.streaming.dynamicAllocation.maxExecutors` instead.

Related codes:
f87153a3ac/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (L101)

## How was this patch tested?

NA

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

Closes #23845 from liupc/Fix-incorrect-maxNumExecutorFailures-for-streaming.

Lead-authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Co-authored-by: liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-16 19:45:05 -05:00
SongYadong ec11790580 [CORE][MINOR] Correct the comment to show heartbeat interval is configurable
## What changes were proposed in this pull request?

Executor heartbeat interval is configurable by `"spark.executor.heartbeatInterval"`. But in a comment, heartbeat interval is presented as a constant `10s`. This pr tries to correct the description.

## How was this patch tested?

Existing unit tests.

Closes #24101 from SongYadong/heartbeat_interval_comment.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-15 20:30:36 -05:00
Dongjoon Hyun 4bab69b22a Revert "[SPARK-27070] Fix performance bug in DefaultPartitionCoalescer"
This reverts commit 21db4336b0.
2019-03-15 14:56:08 -07:00
fitermay 21db4336b0 [SPARK-27070] Fix performance bug in DefaultPartitionCoalescer
When trying to coalesce a UnionRDD of two large FileScanRDDs
(each with a few million partitions) into around 8k partitions
the driver can stall for over an hour.

Profiler shows that over 90% of the time is spent in TimSort
which is invoked by `pickBin`.  This patch replaces sorting with a more
efficient `min` for the purpose of finding the least occupied
PartitionGroup

Closes #23986 from fitermay/SPARK-27070.

Authored-by: fitermay <fiterman@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-14 20:13:18 -05:00
Ajith 2a04de52dd [SPARK-26152] Synchronize Worker Cleanup with Worker Shutdown
## What changes were proposed in this pull request?

The race between org.apache.spark.deploy.DeployMessages.WorkDirCleanup event and  org.apache.spark.deploy.worker.Worker#onStop. Here its possible that while the WorkDirCleanup event is being processed, org.apache.spark.deploy.worker.Worker#cleanupThreadExecutor was shutdown. hence any submission after ThreadPoolExecutor will result in java.util.concurrent.RejectedExecutionException

## How was this patch tested?

Manually

Closes #24056 from ajithme/workercleanup.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-14 09:16:29 -05:00
Jungtaek Lim (HeartSaVioR) f57af2286f [MINOR][CORE] Use https for bintray spark-packages repository
## What changes were proposed in this pull request?

This patch changes the schema of url from http to https for bintray spark-packages repository. Looks like we already changed the schema of repository url for pom.xml but missed inside the code.

## How was this patch tested?

Manually ran the `--package` via `./bin/spark-shell --verbose  --packages "RedisLabs:spark-redis:0.3.2"`

```
...
Ivy Default Cache set to: /Users/jlim/.ivy2/cache
The jars for the packages stored in: /Users/jlim/.ivy2/jars
:: loading settings :: url = jar:file:/Users/jlim/WorkArea/ScalaProjects/spark/dist/jars/ivy-2.4.0.jar!/org/apache/ivy/core/settings/ivysettings.xml
RedisLabs#spark-redis added as a dependency
:: resolving dependencies :: org.apache.spark#spark-submit-parent-2fee2e18-7832-4a4d-9e97-7b3d0fef766d;1.0
	confs: [default]
	found RedisLabs#spark-redis;0.3.2 in spark-packages
	found redis.clients#jedis;2.7.2 in central
	found org.apache.commons#commons-pool2;2.3 in central
downloading https://dl.bintray.com/spark-packages/maven/RedisLabs/spark-redis/0.3.2/spark-redis-0.3.2.jar ...
	[SUCCESSFUL ] RedisLabs#spark-redis;0.3.2!spark-redis.jar (824ms)
downloading https://repo1.maven.org/maven2/redis/clients/jedis/2.7.2/jedis-2.7.2.jar ...
	[SUCCESSFUL ] redis.clients#jedis;2.7.2!jedis.jar (576ms)
downloading https://repo1.maven.org/maven2/org/apache/commons/commons-pool2/2.3/commons-pool2-2.3.jar ...
	[SUCCESSFUL ] org.apache.commons#commons-pool2;2.3!commons-pool2.jar (150ms)
:: resolution report :: resolve 4586ms :: artifacts dl 1555ms
	:: modules in use:
	RedisLabs#spark-redis;0.3.2 from spark-packages in [default]
	org.apache.commons#commons-pool2;2.3 from central in [default]
	redis.clients#jedis;2.7.2 from central in [default]
	---------------------------------------------------------------------
	|                  |            modules            ||   artifacts   |
	|       conf       | number| search|dwnlded|evicted|| number|dwnlded|
	---------------------------------------------------------------------
	|      default     |   3   |   3   |   3   |   0   ||   3   |   3   |
	---------------------------------------------------------------------
```

Closes #24061 from HeartSaVioR/MINOR-use-https-to-bintray-repository.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 18:01:16 -05:00
Liupengcheng d5cfe08fdc [SPARK-26927][CORE] Ensure executor is active when processing events in dynamic allocation manager.
## What changes were proposed in this pull request?

There is a race condition in the `ExecutorAllocationManager` that the `SparkListenerExecutorRemoved` event is posted before the `SparkListenerTaskStart` event, which will cause the incorrect result of `executorIds`. Then, when some executor idles, the real executors will be removed even actual executor number is equal to `minNumExecutors` due to the incorrect computation of `newExecutorTotal`(may greater than the `minNumExecutors`), thus may finally causing zero available executors but a wrong positive number of executorIds was kept in memory.

What's more, even the `SparkListenerTaskEnd` event can not make the fake `executorIds` released, because later idle event for the fake executors can not cause the real removal of these executors, as they are already removed and they are not exist in the `executorDataMap`  of `CoaseGrainedSchedulerBackend`, so that the `onExecutorRemoved` method will never be called again.

For details see https://issues.apache.org/jira/browse/SPARK-26927

This PR is to fix this problem.

## How was this patch tested?

existUT and added UT

Closes #23842 from liupc/Fix-race-condition-that-casues-dyanmic-allocation-not-working.

Lead-authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Co-authored-by: liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-12 13:53:42 -07:00
ankurgupta 688b0c01fa [SPARK-26089][CORE] Handle corruption in large shuffle blocks
## What changes were proposed in this pull request?

SPARK-4105 added corruption detection in shuffle blocks but that was limited to blocks which are
smaller than maxBytesInFlight/3. This commit adds upon that by adding corruption check for large
blocks. There are two changes/improvements that are made in this commit:

1. Large blocks are checked upto maxBytesInFlight/3 size in a similar way as smaller blocks, so if a
large block is corrupt in the starting, that block will be re-fetched and if that also fails,
FetchFailureException will be thrown.
2. If large blocks are corrupt after size maxBytesInFlight/3, then any IOException thrown while
reading the stream will be converted to FetchFailureException.  This is slightly more aggressive
than was originally intended but since the consumer of the stream may have already read some records and processed them, we can't just re-fetch the block, we need to fail the whole task. Additionally, we also thought about maybe adding a new type of TaskEndReason, which would re-try the task couple of times before failing the previous stage, but given the complexity involved in that solution we decided to not proceed in that direction.

Thanks to squito for direction and support.

## How was this patch tested?

Changed the junit test for big blocks to check for corruption.

Closes #23453 from ankuriitg/ankurgupta/SPARK-26089.

Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-12 14:27:44 -05:00
shivusondur 4b6d39d85d [SPARK-27090][CORE] Removing old LEGACY_DRIVER_IDENTIFIER ("<driver>")
## What changes were proposed in this pull request?
LEGACY_DRIVER_IDENTIFIER and its reference are removed.
corresponding references test are updated.

## How was this patch tested?
tested  UT test cases

Closes #24026 from shivusondur/newjira2.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 13:29:39 -05:00
hongdongdong 1029bf9c35 Use variable instead of function to keep the format uniform
## What changes were proposed in this pull request?

The change just use  variable(_taskScheduler) instead of function(taskScheduler) to keep the format uniform in different situation.

## How was this patch tested?

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

Closes #24048 from hddong/Use-variable-instead-of-function.

Authored-by: hongdongdong <hongdongdong@cmss.chinamobile.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 19:00:26 -05:00
Hyukjin Kwon 3725b1324f [SPARK-26923][SQL][R] Refactor ArrowRRunner and RRunner to share one BaseRRunner
## What changes were proposed in this pull request?

This PR proposes to have one base R runner.

In the high level,

Previously, it had `ArrowRRunner` and it inherited `RRunner`:

```
└── RRunner
    └── ArrowRRunner
```

After this PR, now it has a `BaseRRunner`, and `ArrowRRunner` and `RRunner` inherit `BaseRRunner`:

```
└── BaseRRunner
    ├── ArrowRRunner
    └── RRunner
```

This way is consistent with Python's.

In more details, see below:

```scala
class BaseRRunner[IN, OUT] {

  def compute: Iterator[OUT] = {
    ...
    newWriterThread(...).start()
    ...
    newReaderIterator(...)
    ...
  }

  // Make a thread that writes data from JVM to R process
  abstract protected def newWriterThread(..., iter: Iterator[IN], ...): WriterThread

  // Make an iterator that reads data from the R process to JVM
  abstract protected def newReaderIterator(...): ReaderIterator

  abstract class WriterThread(..., iter: Iterator[IN], ...) extends Thread {
    override def run(): Unit {
      ...
      writeIteratorToStream(...)
      ...
    }

    // Actually writing logic to the socket stream.
    abstract protected def writeIteratorToStream(dataOut: DataOutputStream): Unit
  }

  abstract class ReaderIterator extends Iterator[OUT] {
    override def hasNext(): Boolean = {
      ...
      read(...)
      ...
    }

    override def next(): OUT = {
      ...
      hasNext()
      ...
    }

    // Actually reading logic from the socket stream.
    abstract protected def read(...): OUT
  }
}
```

```scala
case [Arrow]RRunner extends BaseRRunner {
  override def newWriterThread(...) {
    new WriterThread(...) {
      override def writeIteratorToStream(...) {
        ...
      }
    }
  }

  override def newReaderIterator(...) {
    new ReaderIterator(...) {
      override def read(...) {
        ...
      }
    }
  }
}
```

## How was this patch tested?

Manually tested and existing tests should cover.

Closes #23977 from HyukjinKwon/SPARK-26923.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-12 08:45:29 +09:00
Marcelo Vanzin f1e223bfa3 [SPARK-27004][CORE] Remove stale HTTP auth code.
This code is from the era when Spark used an HTTP server to distribute
dependencies, which is long gone. Nowadays it only causes problems when
someone is using dependencies from an HTTP server with Spark auth on.

Closes #24033 from vanzin/SPARK-27004.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-11 12:27:25 -07:00
Ajith b98922abf2 [SPARK-27116] Environment tab must sort Hadoop Configuration by default
## What changes were proposed in this pull request?

Environment tab in SparkUI do not have Hadoop Configuration sorted. All other tables in the same page like Spark Configrations, System Configuration etc are sorted by keys by default

## How was this patch tested?

Manually tested on SparkUI

Closes #24038 from ajithme/sqluisort.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 08:43:49 -05:00
Gabor Somogyi 29d9021245 [SPARK-24621][WEBUI] Show secure URLs on web pages
## What changes were proposed in this pull request?

Web UI URLs are pointing to `http://` targets even if SSL is enabled. In this PR I've changed the code to point to `https://` URLs.

## How was this patch tested?

Existing unit tests + manually by starting standalone master/worker/spark-shell. Please see jira.

Closes #23991 from gaborgsomogyi/SPARK-24621.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-10 19:28:35 -05:00
Hyukjin Kwon 28d003097b [SPARK-27102][R][PYTHON][CORE] Remove the references to Python's Scala codes in R's Scala codes
## What changes were proposed in this pull request?

Currently, R's Scala codes happened to refer Python's Scala codes for code deduplications. It's a bit odd. For instance, when we face an exception from R, it shows python related code path, which makes confusing to debug. It should rather have one code base and R's and Python's should share.

This PR proposes:

1. Make a `SocketAuthServer` and move `PythonServer` so that `PythonRDD` and `RRDD` can share it.
2. Move `readRDDFromFile` and `readRDDFromInputStream` into `JavaRDD`.
3. Reuse `RAuthHelper` and remove `RSocketAuthHelper` in `RRDD`.
4. Rename `getEncryptionEnabled` to `isEncryptionEnabled` while I am here.

So, now, the places below:

- `sql/core/src/main/scala/org/apache/spark/sql/api/r`
- `core/src/main/scala/org/apache/spark/api/r`
- `mllib/src/main/scala/org/apache/spark/ml/r`

don't refer Python's Scala codes.

## How was this patch tested?

Existing tests should cover this.

Closes #24023 from HyukjinKwon/SPARK-27102.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-10 15:08:23 +09:00
Kris Mok 57ae251f75 [SPARK-27097] Avoid embedding platform-dependent offsets literally in whole-stage generated code
## What changes were proposed in this pull request?

Spark SQL performs whole-stage code generation to speed up query execution. There are two steps to it:
- Java source code is generated from the physical query plan on the driver. A single version of the source code is generated from a query plan, and sent to all executors.
  - It's compiled to bytecode on the driver to catch compilation errors before sending to executors, but currently only the generated source code gets sent to the executors. The bytecode compilation is for fail-fast only.
- Executors receive the generated source code and compile to bytecode, then the query runs like a hand-written Java program.

In this model, there's an implicit assumption about the driver and executors being run on similar platforms. Some code paths accidentally embedded platform-dependent object layout information into the generated code, such as:
```java
Platform.putLong(buffer, /* offset */ 24, /* value */ 1);
```
This code expects a field to be at offset +24 of the `buffer` object, and sets a value to that field.
But whole-stage code generation generally uses platform-dependent information from the driver. If the object layout is significantly different on the driver and executors, the generated code can be reading/writing to wrong offsets on the executors, causing all kinds of data corruption.

One code pattern that leads to such problem is the use of `Platform.XXX` constants in generated code, e.g. `Platform.BYTE_ARRAY_OFFSET`.

Bad:
```scala
val baseOffset = Platform.BYTE_ARRAY_OFFSET
// codegen template:
s"Platform.putLong($buffer, $baseOffset, $value);"
```
This will embed the value of `Platform.BYTE_ARRAY_OFFSET` on the driver into the generated code.

Good:
```scala
val baseOffset = "Platform.BYTE_ARRAY_OFFSET"
// codegen template:
s"Platform.putLong($buffer, $baseOffset, $value);"
```
This will generate the offset symbolically -- `Platform.putLong(buffer, Platform.BYTE_ARRAY_OFFSET, value)`, which will be able to pick up the correct value on the executors.

Caveat: these offset constants are declared as runtime-initialized `static final` in Java, so they're not compile-time constants from the Java language's perspective. It does lead to a slightly increased size of the generated code, but this is necessary for correctness.

NOTE: there can be other patterns that generate platform-dependent code on the driver which is invalid on the executors. e.g. if the endianness is different between the driver and the executors, and if some generated code makes strong assumption about endianness, it would also be problematic.

## How was this patch tested?

Added a new test suite `WholeStageCodegenSparkSubmitSuite`. This test suite needs to set the driver's extraJavaOptions to force the driver and executor use different Java object layouts, so it's run as an actual SparkSubmit job.

Authored-by: Kris Mok <kris.mokdatabricks.com>

Closes #24031 from gatorsmile/cherrypickSPARK-27097.

Lead-authored-by: Kris Mok <kris.mok@databricks.com>
Co-authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-09 01:20:32 +00:00
Wenchen Fan cb20fbc43e [SPARK-27065][CORE] avoid more than one active task set managers for a stage
## What changes were proposed in this pull request?

This is another attempt to fix the more-than-one-active-task-set-managers bug.

https://github.com/apache/spark/pull/17208 is the first attempt. It marks the TSM as zombie before sending a task completion event to DAGScheduler. This is necessary, because when the DAGScheduler gets the task completion event, and it's for the last partition, then the stage is finished. However, if it's a shuffle stage and it has missing map outputs, DAGScheduler will resubmit it(see the [code](https://github.com/apache/spark/blob/v2.4.0/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1416-L1422)) and create a new TSM for this stage. This leads to more than one active TSM of a stage and fail.

This fix has a hole: Let's say a stage has 10 partitions and 2 task set managers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 10 and it completes. TSM2 finishes tasks for partitions 1-9, and thinks he is still active because he hasn't finished partition 10 yet. However, DAGScheduler gets task completion events for all the 10 partitions and thinks the stage is finished. Then the same problem occurs: DAGScheduler may resubmit the stage and cause more than one actice TSM error.

https://github.com/apache/spark/pull/21131 fixed this hole by notifying all the task set managers when a task finishes. For the above case, TSM2 will know that partition 10 is already completed, so he can mark himself as zombie after partitions 1-9 are completed.

However, #21131 still has a hole: TSM2 may be created after the task from TSM1 is completed. Then TSM2 can't get notified about the task completion, and leads to the more than one active TSM error.

#22806 and #23871 are created to fix this hole. However the fix is complicated and there are still ongoing discussions.

This PR proposes a simple fix, which can be easy to backport: mark all existing task set managers as zombie when trying to create a new task set manager.

After this PR, #21131 is still necessary, to avoid launching unnecessary tasks and fix [SPARK-25250](https://issues.apache.org/jira/browse/SPARK-25250 ). #22806 and #23871 are its followups to fix the hole.

## How was this patch tested?

existing tests.

Closes #23927 from cloud-fan/scheduler.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-06 12:00:33 -06:00
wuyi e5c61436a5 [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions
## What changes were proposed in this pull request?

This is an optional solution for #22806 .

#21131 firstly implement that a previous successful completed task from zombie TaskSetManager could also succeed the active TaskSetManager, which based on an assumption that an active TaskSetManager always exists for that stage when this happen.  But that's not always true as an active TaskSetManager may haven't been created when a previous task succeed, and this is the reason why #22806 hit the issue.

This pr extends #21131 's behavior by adding `stageIdToFinishedPartitions` into TaskSchedulerImpl, which recording the finished partition whenever a task(from zombie or active) succeed. Thus, a later created active TaskSetManager could also learn about the finished partition by looking into `stageIdToFinishedPartitions ` and won't launch any duplicate tasks.

## How was this patch tested?

Add.

Closes #23871 from Ngone51/dev-23433-25250.

Lead-authored-by: wuyi <ngone_5451@163.com>
Co-authored-by: Ngone51 <ngone_5451@163.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-06 11:53:07 -06:00
moqimoqidea 3fcbc7fb9f [MINOR] Spelling mistake: forword -> forward
## What changes were proposed in this pull request?

Spelling mistake: forword -> forward

## How was this patch tested?

This is a private function, there is no place to call this function outside of this file.

Closes #23978 from moqimoqidea/master.

Authored-by: moqimoqidea <39821951+moqimoqidea@users.noreply.github.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-06 16:29:07 +09:00
“attilapiros” 5668c42edf [SPARK-27021][CORE] Cleanup of Netty event loop group for shuffle chunk fetch requests
## What changes were proposed in this pull request?

Creating an Netty `EventLoopGroup` leads to creating a new Thread pool for handling the events. For stopping the threads of the pool the event loop group should be shut down which is properly done for transport servers and clients by calling for example the `shutdownGracefully()` method (for details see the `close()` method of `TransportClientFactory` and `TransportServer`). But there is a separate event loop group for shuffle chunk fetch requests which is in pipeline for handling fetch request (shared between the client and server) and owned by the `TransportContext` and this was never shut down.

## How was this patch tested?

With existing unittest.

This leak is in the production system too but its effect is spiking in the unittest.

Checking the core unittest logs before the PR:
```
$ grep "LEAK IN SUITE" unit-tests.log | grep -o shuffle-chunk-fetch-handler | wc -l
381
```

And after the PR without whitelisting in thread audit and with an extra `await` after the
` chunkFetchWorkers.shutdownGracefully()`:
```
$ grep "LEAK IN SUITE" unit-tests.log | grep -o shuffle-chunk-fetch-handler | wc -l
0
```

Closes #23930 from attilapiros/SPARK-27021.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-05 12:31:06 -08:00
Luca Canali 25d2850665 [SPARK-26928][CORE] Add driver CPU Time to the metrics system
## What changes were proposed in this pull request?

This proposes to add instrumentation for the driver's JVM CPU time via the Spark Dropwizard/Codahale metrics system. It follows directly from previous work SPARK-25228 and shares similar motivations: it is intended as an improvement to be used for Spark performance dashboards and monitoring tools/instrumentation.

Implementation details: this PR takes the code introduced in SPARK-25228 and moves it to a new separate Source JVMCPUSource, which is then used to register the jvmCpuTime gauge metric for both executor and driver.
The registration of the jvmCpuTime metric for the driver is conditional, a new configuration parameter `spark.metrics.cpu.time.driver.enabled` (proposed default: false) is introduced for this purpose.

## How was this patch tested?

Manually tested, using local mode and using YARN.

Closes #23838 from LucaCanali/addCPUTimeMetricDriver.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-05 10:47:39 -08:00
Ajith 6207360b00 [SPARK-27012][CORE] Storage tab shows rdd details even after executor ended
## What changes were proposed in this pull request?

After we cache a table, we can see its details in Storage Tab of spark UI. If the executor has shutdown ( graceful shutdown/ Dynamic executor scenario) UI still shows the rdd as cached and when we click the link it throws error. This is because on executor remove event, we fail to adjust rdd partition details  org.apache.spark.status.AppStatusListener#onExecutorRemoved

## How was this patch tested?

Have tested this fix in UI manually
Edit: Added UT

Closes #23920 from ajithme/cachestorage.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-05 10:40:38 -08:00
Yanbo Liang 7857c6d633 [SPARK-27051][CORE] Bump Jackson version to 2.9.8
## What changes were proposed in this pull request?
Fasterxml Jackson version before 2.9.8 is affected by multiple [CVEs](https://github.com/FasterXML/jackson-databind/issues/2186), we need to fix bump the dependent Jackson to 2.9.8.

## How was this patch tested?
Existing tests and offline benchmark.
I have run ```SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.datasources.json.JSONBenchmark"``` to check there is no performance degradation for this upgrade.

Closes #23965 from yanboliang/SPARK-27051.

Authored-by: Yanbo Liang <ybliang8@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-05 11:46:51 +09:00
Sean Owen 0deebd3820 [SPARK-26016][DOCS] Clarify that text DataSource read/write, and RDD methods that read text, always use UTF-8
## What changes were proposed in this pull request?

Clarify that text DataSource read/write, and RDD methods that read text, always use UTF-8 as they use Hadoop's implementation underneath. I think these are all the places that this needs a mention in the user-facing docs.

## How was this patch tested?

Doc tests.

Closes #23962 from srowen/SPARK-26016.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-05 08:03:39 +09:00
Yuming Wang 827d371877 [SPARK-25689][FOLLOW-UP][CORE] Get proxy user's delegation tokens
## What changes were proposed in this pull request?

This pr makes it get proxy user's delegation token, otherwise throws `AccessControlException`([full log](https://issues.apache.org/jira/browse/SPARK-25689?focusedCommentId=16780609&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16780609)):
```java
org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
...
at org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend.waitForApplication(YarnClientSchedulerBackend.scala:95)
at org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend.start(YarnClientSchedulerBackend.scala:62)
at org.apache.spark.scheduler.TaskSchedulerImpl.start(TaskSchedulerImpl.scala:185)
```

How to reproduce this issue:
```shell
$ ssh user_admspark-getaway-host1
$ export HADOOP_PROXY_USER=user_a
$ spark-sql --master yarn
```

## How was this patch tested?

Test on our production environment.

Closes #23922 from wangyum/SPARK-25689.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-04 13:21:24 -08:00
LantaoJin e5c502c596 [SPARK-25865][CORE] Add GC information to ExecutorMetrics
## What changes were proposed in this pull request?

Only memory usage without GC information could not help us to determinate the proper settings of memory. We need the GC metrics about frequency of major & minor GC. For example, two cases, their configured memory for executor are all 10GB and their usages are all near 10GB. So should we increase or decrease the configured memory for them? This metrics may be helpful. We can increase configured memory for the first one if it has very frequency major GC and decrease the second one if only some minor GC and none major GC.
GC metrics are only useful in entire lifetime of executors instead of separated stages.

## How was this patch tested?

Adding UT.

Closes #22874 from LantaoJin/SPARK-25865.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-04 14:26:02 -06:00
Jungtaek Lim (HeartSaVioR) d5bda2c9e8 [SPARK-26792][CORE] Apply custom log URL to Spark UI
## What changes were proposed in this pull request?

[SPARK-23155](https://issues.apache.org/jira/browse/SPARK-23155) enables SHS to set up custom executor log URLs. This patch proposes to extend this feature to to Spark UI as well.

Unlike the approach we did for SHS (replace executor log URLs when executor information is requested so it's like a change of view), here this patch replaces executor log URLs while registering executor, which also affects event log as well. In point of SHS's view, it will be treated as original log url when custom log url is applied to Spark UI.

## How was this patch tested?

Added UT.

Closes #23790 from HeartSaVioR/SPARK-26792.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-04 10:36:04 -08:00
manuzhang 81dd21fda9 [SPARK-26977][CORE] Fix warn against subclassing scala.App
## What changes were proposed in this pull request?

Fix warn against subclassing scala.App

## How was this patch tested?

Manual test

Closes #23903 from manuzhang/fix_submit_warning.

Authored-by: manuzhang <owenzhang1990@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-01 17:37:58 -06:00
SongYadong 86b25c4350 [SPARK-26967][CORE] Put MetricsSystem instance names together for clearer management
## What changes were proposed in this pull request?

`MetricsSystem` instance creations have a scattered distribution in the project code. So do their names. It may cause some inconvenience for browsing and management.
This PR tries to put them together. In this way, we can have a uniform location for adding or removing them, and have a overall view of `MetircsSystem `instances in current project.
It's also helpful for maintaining user documents by avoiding missing something.

## How was this patch tested?

Existing unit tests.

Closes #23869 from SongYadong/metrics_system_inst_manage.

Lead-authored-by: SongYadong <song.yadong1@zte.com.cn>
Co-authored-by: walter2001 <ydsong2007@163.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-01 11:49:43 -06:00
liuxian 02bbe977ab [MINOR] Remove unnecessary gets when getting a value from map.
## What changes were proposed in this pull request?

Redundant `get`  when getting a value from `Map` given a key.

## How was this patch tested?

N/A

Closes #23901 from 10110346/removegetfrommap.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-01 11:48:07 -06:00
Yifei Huang bc7592ba11 [SPARK-27009][TEST] Add Standard Deviation to benchmark results
## What changes were proposed in this pull request?

Add standard deviation to the stats taken during benchmark testing.

## How was this patch tested?

Manually ran a few benchmark tests locally and visually inspected the output

Closes #23914 from yifeih/spark-27009-stdev.

Authored-by: Yifei Huang <yifeih@palantir.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-02-28 20:55:55 -08:00
Imran Rashid c8e7eb1fa7 [SPARK-26774][CORE] Update some docs on TaskSchedulerImpl.
A couple of places in TaskSchedulerImpl could use a minor doc update on
threading concerns.  There is one bug fix here, but only in
sc.killTaskAttempt() which is probably not used much.

Closes #23874 from squito/SPARK-26774.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-28 11:30:20 -08:00
Hyukjin Kwon 6e31ccf2a1 [SPARK-26895][CORE][FOLLOW-UP] Uninitializing log after prepareSubmitEnvironment in SparkSubmit
## What changes were proposed in this pull request?

Currently, if I run `spark-shell` in my local, it started to show the logs as below:

```
$ ./bin/spark-shell
...
19/02/28 04:42:43 INFO SecurityManager: Changing view acls to: hkwon
19/02/28 04:42:43 INFO SecurityManager: Changing modify acls to: hkwon
19/02/28 04:42:43 INFO SecurityManager: Changing view acls groups to:
19/02/28 04:42:43 INFO SecurityManager: Changing modify acls groups to:
19/02/28 04:42:43 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(hkwon); groups with view permissions: Set(); users  with modify permissions: Set(hkwon); groups with modify permissions: Set()
19/02/28 04:42:43 INFO SignalUtils: Registered signal handler for INT
19/02/28 04:42:48 INFO SparkContext: Running Spark version 3.0.0-SNAPSHOT
19/02/28 04:42:48 INFO SparkContext: Submitted application: Spark shell
19/02/28 04:42:48 INFO SecurityManager: Changing view acls to: hkwon
```

Seems to be the cause is https://github.com/apache/spark/pull/23806 and `prepareSubmitEnvironment` looks actually reinitializing the logging again.

This PR proposes to uninitializing log later after `prepareSubmitEnvironment`.

## How was this patch tested?

Manually tested.

Closes #23911 from HyukjinKwon/SPARK-26895.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-27 17:01:30 -08:00