Commit graph

6920 commits

Author SHA1 Message Date
Liupengcheng 88b074f3f0 [SPARK-26501][CORE][TEST] Fix unexpected overriden of exitFn in SparkSubmitSuite
## What changes were proposed in this pull request?

The overriden of SparkSubmit's exitFn at some previous tests in SparkSubmitSuite may cause the following tests pass even they failed when they were run separately. This PR is to fix this problem.

## How was this patch tested?

unittest

Closes #23404 from liupc/Fix-SparkSubmitSuite-exitFn.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-03 10:26:14 -06:00
Sean Owen 4bdfda92a1 [SPARK-26507][CORE] Fix core tests for Java 11
## What changes were proposed in this pull request?

This should make tests in core modules pass for Java 11.

## How was this patch tested?

Existing tests, with modifications.

Closes #23419 from srowen/Java11.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-02 11:23:53 -06:00
Kazuaki Ishizaki 79b05481a2 [SPARK-26508][CORE][SQL] Address warning messages in Java reported at lgtm.com
## What changes were proposed in this pull request?

This PR addresses warning messages in Java files reported at [lgtm.com](https://lgtm.com).

[lgtm.com](https://lgtm.com) provides automated code review of Java/Python/JavaScript files for OSS projects. [Here](https://lgtm.com/projects/g/apache/spark/alerts/?mode=list&severity=warning) are warning messages regarding Apache Spark project.

This PR addresses the following warnings:

- Result of multiplication cast to wider type
- Implicit narrowing conversion in compound assignment
- Boxed variable is never null
- Useless null check

NOTE: `Potential input resource leak` looks false positive for now.

## How was this patch tested?

Existing UTs

Closes #23420 from kiszk/SPARK-26508.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-01 22:37:28 -06:00
Jungtaek Lim (HeartSaVioR) 993736154b [MINOR] Fix inconsistency log level among delegation token providers
## What changes were proposed in this pull request?

There's some inconsistency for log level while logging error messages in
delegation token providers. (DEBUG, INFO, WARNING)

Given that failing to obtain token would often crash the query, I guess
it would be nice to set higher log level for error log messages.

## How was this patch tested?

The patch just changed the log level.

Closes #23418 from HeartSaVioR/FIX-inconsistency-log-level-between-delegation-token-providers.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-01 09:14:23 +08:00
Marco Gaido b1a9b5eff5
[SPARK-26470][CORE] Use ConfigEntry for hardcoded configs for eventLog category
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.eventLog` configs to use `ConfigEntry` and put them in the `config` package.

## How was this patch tested?

existing tests

Closes #23395 from mgaido91/SPARK-26470.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-31 13:35:02 -08:00
Gengliang Wang 240817b7ae [SPARK-26363][WEBUI] Avoid duplicated KV store lookups in method taskList
## What changes were proposed in this pull request?

In the method `taskList`(since https://github.com/apache/spark/pull/21688),  the executor log value is queried in KV store  for every task(method `constructTaskData`).
This PR propose to use a hashmap for reducing duplicated KV store lookups in the method.

![image](https://user-images.githubusercontent.com/1097932/49946230-841c7680-ff29-11e8-8b83-d8f7553bfe5e.png)

## How was this patch tested?

Manual check

Closes #23310 from gengliangwang/removeExecutorLog.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-29 21:47:49 -06:00
Takuya UESHIN e6d3e7d0d8
[SPARK-26443][CORE] Use ConfigEntry for hardcoded configs for history category.
## What changes were proposed in this pull request?

This pr makes hardcoded "spark.history" configs to use `ConfigEntry` and put them in `History` config object.

## How was this patch tested?

Existing tests.

Closes #23384 from ueshin/issues/SPARK-26443/hardcoded_history_configs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-29 17:33:43 -08:00
seancxmao 5bef4fedfe [SPARK-26444][WEBUI] Stage color doesn't change with it's status
## What changes were proposed in this pull request?
On job page, in event timeline section, stage color doesn't change according to its status. Below are some screenshots.

ACTIVE:
<img width="550" alt="active" src="https://user-images.githubusercontent.com/12194089/50438844-c763e580-092a-11e9-84f6-6fc30e08d69b.png">
COMPLETE:
<img width="516" alt="complete" src="https://user-images.githubusercontent.com/12194089/50438847-ca5ed600-092a-11e9-9d2e-5d79807bc1ce.png">
FAILED:
<img width="325" alt="failed" src="https://user-images.githubusercontent.com/12194089/50438852-ccc13000-092a-11e9-9b6b-782b96b283b1.png">

This PR lets stage color change with it's status. The main idea is to make css style class name match the corresponding stage status.

## How was this patch tested?
Manually tested locally.

```
// active/complete stage
sc.parallelize(1 to 3, 3).map { n => Thread.sleep(10* 1000); n }.count
// failed stage
sc.parallelize(1 to 3, 3).map { n => Thread.sleep(10* 1000); throw new Exception() }.count
```

Note we need to clear browser cache to let new `timeline-view.css` take effect. Below are screenshots after this PR.

ACTIVE:
<img width="569" alt="active-after" src="https://user-images.githubusercontent.com/12194089/50439986-08f68f80-092f-11e9-85d9-be1c31aed13b.png">
COMPLETE:
<img width="567" alt="complete-after" src="https://user-images.githubusercontent.com/12194089/50439990-0bf18000-092f-11e9-8624-723958906e90.png">
FAILED:
<img width="352" alt="failed-after" src="https://user-images.githubusercontent.com/12194089/50439993-101d9d80-092f-11e9-8dfd-3e20536f2fa5.png">

Closes #23385 from seancxmao/timeline-stage-color.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-28 07:40:59 -06:00
wuqingxin f2adb61068
[SPARK-26446][CORE] Add cachedExecutorIdleTimeout docs at ExecutorAllocationManager
## What changes were proposed in this pull request?

Add docs to describe how remove policy act while considering the property `spark.dynamicAllocation.cachedExecutorIdleTimeout` in ExecutorAllocationManager

## How was this patch tested?
comment-only PR.

Closes #23386 from TopGunViper/SPARK-26446.

Authored-by: wuqingxin <wuqingxin@baidu.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-28 00:15:57 -08:00
Alessandro Bellina 0a02d5c36f [SPARK-26285][CORE] accumulator metrics sources for LongAccumulator and Doub…
…leAccumulator

## What changes were proposed in this pull request?

This PR implements metric sources for LongAccumulator and DoubleAccumulator, such that a user can register these accumulators easily and have their values be reported by the driver's metric namespace.

## How was this patch tested?

Unit tests, and manual tests.

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

Closes #23242 from abellina/SPARK-26285_accumulator_source.

Lead-authored-by: Alessandro Bellina <abellina@yahoo-inc.com>
Co-authored-by: Alessandro Bellina <abellina@oath.com>
Co-authored-by: Alessandro Bellina <abellina@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-12-22 09:03:02 -06:00
pgandhi 8dd29fe36b [SPARK-25642][YARN] Adding two new metrics to record the number of registered connections as well as the number of active connections to YARN Shuffle Service
Recently, the ability to expose the metrics for YARN Shuffle Service was added as part of [SPARK-18364](https://github.com/apache/spark/pull/22485). We need to add some metrics to be able to determine the number of active connections as well as open connections to the external shuffle service to benchmark network and connection issues on large cluster environments.

Added two more shuffle server metrics for Spark Yarn shuffle service: numRegisteredConnections which indicate the number of registered connections to the shuffle service and numActiveConnections which indicate the number of active connections to the shuffle service at any given point in time.

If these metrics are outputted to a file, we get something like this:

1533674653489 default.shuffleService: Hostname=server1.abc.com, openBlockRequestLatencyMillis_count=729, openBlockRequestLatencyMillis_rate15=0.7110833548897356, openBlockRequestLatencyMillis_rate5=1.657808981793011, openBlockRequestLatencyMillis_rate1=2.2404486061620474, openBlockRequestLatencyMillis_rateMean=0.9242558551196706,
numRegisteredConnections=35,
blockTransferRateBytes_count=2635880512, blockTransferRateBytes_rate15=2578547.6094160094, blockTransferRateBytes_rate5=6048721.726302424, blockTransferRateBytes_rate1=8548922.518223226, blockTransferRateBytes_rateMean=3341878.633637769, registeredExecutorsSize=5, registerExecutorRequestLatencyMillis_count=5, registerExecutorRequestLatencyMillis_rate15=0.0027973949328659836, registerExecutorRequestLatencyMillis_rate5=0.0021278007987206426, registerExecutorRequestLatencyMillis_rate1=2.8270296777387467E-6, registerExecutorRequestLatencyMillis_rateMean=0.006339206380043053, numActiveConnections=35

Closes #22498 from pgandhi999/SPARK-18364.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-21 11:28:33 -08:00
zhoukang 7c8f4756c3 [SPARK-24687][CORE] Avoid job hanging when generate task binary causes fatal error
## What changes were proposed in this pull request?
When NoClassDefFoundError thrown,it will cause job hang.
`Exception in thread "dag-scheduler-event-loop" java.lang.NoClassDefFoundError: Lcom/xxx/data/recommend/aggregator/queue/QueueName;
	at java.lang.Class.getDeclaredFields0(Native Method)
	at java.lang.Class.privateGetDeclaredFields(Class.java:2436)
	at java.lang.Class.getDeclaredField(Class.java:1946)
	at java.io.ObjectStreamClass.getDeclaredSUID(ObjectStreamClass.java:1659)
	at java.io.ObjectStreamClass.access$700(ObjectStreamClass.java:72)
	at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:480)
	at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:468)
	at java.security.AccessController.doPrivileged(Native Method)
	at java.io.ObjectStreamClass.<init>(ObjectStreamClass.java:468)
	at java.io.ObjectStreamClass.lookup(ObjectStreamClass.java:365)
	at java.io.ObjectOutputStream.writeClass(ObjectOutputStream.java:1212)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1119)
	at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
	at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
	at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
	at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
	at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
	at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
	at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
	at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
	at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
	at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1377)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1173)
	at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
	at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
	at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
	at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
	at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
	at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
	at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547)
	at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508)
	at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
	at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1377)`

It is caused by NoClassDefFoundError will not catch up during task seriazation.
`var taskBinary: Broadcast[Array[Byte]] = null
    try {
      // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep).
      // For ResultTask, serialize and broadcast (rdd, func).
      val taskBinaryBytes: Array[Byte] = stage match {
        case stage: ShuffleMapStage =>
          JavaUtils.bufferToArray(
            closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef))
        case stage: ResultStage =>
          JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef))
      }

      taskBinary = sc.broadcast(taskBinaryBytes)
    } catch {
      // In the case of a failure during serialization, abort the stage.
      case e: NotSerializableException =>
        abortStage(stage, "Task not serializable: " + e.toString, Some(e))
        runningStages -= stage

        // Abort execution
        return
      case NonFatal(e) =>
        abortStage(stage, s"Task serialization failed: $e\n${Utils.exceptionString(e)}", Some(e))
        runningStages -= stage
        return
    }`
image below shows that stage 33 blocked and never be scheduled.
<img width="1273" alt="2018-06-28 4 28 42" src="https://user-images.githubusercontent.com/26762018/42621188-b87becca-85ef-11e8-9a0b-0ddf07504c96.png">
<img width="569" alt="2018-06-28 4 28 49" src="https://user-images.githubusercontent.com/26762018/42621191-b8b260e8-85ef-11e8-9d10-e97a5918baa6.png">

## How was this patch tested?
UT

Closes #21664 from caneGuy/zhoukang/fix-noclassdeferror.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-20 08:26:25 -06:00
Marcelo Vanzin 4b3fe3a9cc [SPARK-25815][K8S] Support kerberos in client mode, keytab-based token renewal.
This change hooks up the k8s backed to the updated HadoopDelegationTokenManager,
so that delegation tokens are also available in client mode, and keytab-based token
renewal is enabled.

The change re-works the k8s feature steps related to kerberos so
that the driver does all the credential management and provides all
the needed information to executors - so nothing needs to be added
to executor pods. This also makes cluster mode behave a lot more
similarly to client mode, since no driver-related config steps are run
in the latter case.

The main two things that don't need to happen in executors anymore are:

- adding the Hadoop config to the executor pods: this is not needed
  since the Spark driver will serialize the Hadoop config and send
  it to executors when running tasks.

- mounting the kerberos config file in the executor pods: this is
  not needed once you remove the above. The Hadoop conf sent by
  the driver with the tasks is already resolved (i.e. has all the
  kerberos names properly defined), so executors do not need access
  to the kerberos realm information anymore.

The change also avoids creating delegation tokens unnecessarily.
This means that they'll only be created if a secret with tokens
was not provided, and if a keytab is not provided. In either of
those cases, the driver code will handle delegation tokens: in
cluster mode by creating a secret and stashing them, in client
mode by using existing mechanisms to send DTs to executors.

One last feature: the change also allows defining a keytab with
a "local:" URI. This is supported in client mode (although that's
the same as not saying "local:"), and in k8s cluster mode. This
allows the keytab to be mounted onto the image from a pre-existing
secret, for example.

Finally, the new code always sets SPARK_USER in the driver and
executor pods. This is in line with how other resource managers
behave: the submitting user reflects which user will access
Hadoop services in the app. (With kerberos, that's overridden
by the logged in user.) That user is unrelated to the OS user
the app is running as inside the containers.

Tested:
- client and cluster mode with kinit
- cluster mode with keytab
- cluster mode with local: keytab
- YARN cluster with keytab (to make sure it isn't broken)

Closes #22911 from vanzin/SPARK-25815.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-18 13:30:09 -08:00
Jackey Lee 428eb2ad0a [SPARK-26394][CORE] Fix annotation error for Utils.timeStringAsMs
## What changes were proposed in this pull request?

Change microseconds to milliseconds in annotation of Utils.timeStringAsMs.

Closes #23346 from stczwd/stczwd.

Authored-by: Jackey Lee <qcsd2011@163.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-18 12:15:36 -06:00
Wenchen Fan befca983d2
[SPARK-26382][CORE] prefix comparator should handle -0.0
## What changes were proposed in this pull request?

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

The `UnsafeProject` will normalize special float/double values(NaN and -0.0), so the sorter doesn't have to handle it.

However, for consistency and future-proof, this PR proposes to normalize `-0.0` in the prefix comparator, so that it's same with the normal ordering. Note that prefix comparator handles NaN as well.

This is not a bug fix, but a safe guard.

## How was this patch tested?

existing tests

Closes #23334 from cloud-fan/sort.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-18 10:09:56 -08:00
Hyukjin Kwon 9ccae0c9e7 [SPARK-26362][CORE] Remove 'spark.driver.allowMultipleContexts' to disallow multiple creation of SparkContexts
## What changes were proposed in this pull request?

Multiple SparkContexts are discouraged and it has been warning for last 4 years, see SPARK-4180. It could cause arbitrary and mysterious error cases, see SPARK-2243.

Honestly, I didn't even know Spark still allows it, which looks never officially supported, see SPARK-2243.

I believe It should be good timing now to remove this configuration.

## How was this patch tested?

Each doc was manually checked and manually tested:

```
$ ./bin/spark-shell --conf=spark.driver.allowMultipleContexts=true
...
scala> new SparkContext()
org.apache.spark.SparkException: Only one SparkContext should be running in this JVM (see SPARK-2243).The currently running SparkContext was created at:
org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:939)
...
org.apache.spark.SparkContext$.$anonfun$assertNoOtherContextIsRunning$2(SparkContext.scala:2435)
  at scala.Option.foreach(Option.scala:274)
  at org.apache.spark.SparkContext$.assertNoOtherContextIsRunning(SparkContext.scala:2432)
  at org.apache.spark.SparkContext$.markPartiallyConstructed(SparkContext.scala:2509)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:80)
  at org.apache.spark.SparkContext.<init>(SparkContext.scala:112)
  ... 49 elided
```

Closes #23311 from HyukjinKwon/SPARK-26362.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-15 13:55:24 +08:00
Liang-Chi Hsieh 1b604c1fd0 [SPARK-26265][CORE][FOLLOWUP] Put freePage into a finally block
## What changes were proposed in this pull request?

Based on the [comment](https://github.com/apache/spark/pull/23272#discussion_r240735509), it seems to be better to put `freePage` into a `finally` block. This patch as a follow-up to do so.

## How was this patch tested?

Existing tests.

Closes #23294 from viirya/SPARK-26265-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-15 13:52:07 +08:00
Gengliang Wang 524d1be6d2 [SPARK-26098][WEBUI] Show associated SQL query in Job page
## What changes were proposed in this pull request?

For jobs associated to SQL queries, it would be easier to understand the context to showing the SQL query in Job detail page.
Before code change, it is hard to tell what the job is about from the job page:

![image](https://user-images.githubusercontent.com/1097932/48659359-96baa180-ea8a-11e8-8419-a0a87c3f30fc.png)

After code change:
![image](https://user-images.githubusercontent.com/1097932/48659390-26f8e680-ea8b-11e8-8fdd-3b58909ea364.png)

After navigating to the associated SQL detail page, We can see the whole context :
![image](https://user-images.githubusercontent.com/1097932/48659463-9fac7280-ea8c-11e8-9dfe-244e849f72a5.png)

**For Jobs don't have associated SQL query, the text won't be shown.**

## How was this patch tested?

Manual test

Closes #23068 from gengliangwang/addSQLID.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-12-13 09:07:33 -08:00
n.fraison 29b3eb6fed [SPARK-26340][CORE] Ensure cores per executor is greater than cpu per task
Currently this check is only performed for dynamic allocation use case in
ExecutorAllocationManager.

## What changes were proposed in this pull request?

Checks that cpu per task is lower than number of cores per executor otherwise throw an exception

## How was this patch tested?

manual tests

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

Closes #23290 from ashangit/master.

Authored-by: n.fraison <n.fraison@criteo.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-13 08:34:47 -06:00
lichaoqun f69998ace6 [MINOR][DOC] update the condition description of BypassMergeSortShuffle
## What changes were proposed in this pull request?
These three condition descriptions should be updated, follow #23228  :
<li>no Ordering is specified,</li>
<li>no Aggregator is specified, and</li>
<li>the number of partitions is less than
<code>spark.shuffle.sort.bypassMergeThreshold</code>.
</li>
1、If the shuffle dependency specifies aggregation, but it only aggregates at the reduce-side, BypassMergeSortShuffle can still be used.
2、If the number of output partitions is spark.shuffle.sort.bypassMergeThreshold(eg.200), we can use BypassMergeSortShuffle.

## How was this patch tested?
N/A

Closes #23281 from lcqzte10192193/wid-lcq-1211.

Authored-by: lichaoqun <li.chaoqun@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-13 07:42:17 -06:00
Gabor Somogyi 6daa783094 [SPARK-26322][SS] Add spark.kafka.sasl.token.mechanism to ease delegation token configuration.
## What changes were proposed in this pull request?

When Kafka delegation token obtained, SCRAM `sasl.mechanism` has to be configured for authentication. This can be configured on the related source/sink which is inconvenient from user perspective. Such granularity is not required and this configuration can be implemented with one central parameter.

In this PR `spark.kafka.sasl.token.mechanism` added to configure this centrally (default: `SCRAM-SHA-512`).

## How was this patch tested?

Existing unit tests + on cluster.

Closes #23274 from gaborgsomogyi/SPARK-26322.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-12 16:45:50 -08:00
Luca Canali 2920438c43 [SPARK-25277][YARN] YARN applicationMaster metrics should not register static metrics
## What changes were proposed in this pull request?

YARN applicationMaster metrics registration introduced in SPARK-24594 causes further registration of static metrics (Codegenerator and HiveExternalCatalog) and of JVM metrics, which I believe do not belong in this context.
This looks like an unintended side effect of using the start method of [[MetricsSystem]].
A possible solution proposed here, is to introduce startNoRegisterSources to avoid these additional registrations of static sources and of JVM sources in the case of YARN applicationMaster metrics (this could be useful for other metrics that may be added in the future).

## How was this patch tested?

Manually tested on a YARN cluster,

Closes #22279 from LucaCanali/YarnMetricsRemoveExtraSourceRegistration.

Lead-authored-by: Luca Canali <luca.canali@cern.ch>
Co-authored-by: LucaCanali <luca.canali@cern.ch>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-12 16:18:22 -08:00
Yuanjian Li bd8da3799d [SPARK-26193][SQL][FOLLOW UP] Read metrics rename and display text changes
## What changes were proposed in this pull request?
Follow up pr for #23207, include following changes:

- Rename `SQLShuffleMetricsReporter` to `SQLShuffleReadMetricsReporter` to make it match with write side naming.
- Display text changes for read side for naming consistent.
- Rename function in `ShuffleWriteProcessor`.
- Delete `private[spark]` in execution package.

## How was this patch tested?

Existing tests.

Closes #23286 from xuanyuanking/SPARK-26193-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-12 10:03:50 +08:00
mcheah 57d6fbfa8c [SPARK-26239] File-based secret key loading for SASL.
This proposes an alternative way to load secret keys into a Spark application that is running on Kubernetes. Instead of automatically generating the secret, the secret key can reside in a file that is shared between both the driver and executor containers.

Unit tests.

Closes #23252 from mccheah/auth-secret-with-file.

Authored-by: mcheah <mcheah@palantir.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-11 13:50:16 -08:00
Liang-Chi Hsieh a3bbca98d7 [SPARK-26265][CORE] Fix deadlock in BytesToBytesMap.MapIterator when locking both BytesToBytesMap.MapIterator and TaskMemoryManager
## What changes were proposed in this pull request?

In `BytesToBytesMap.MapIterator.advanceToNextPage`, We will first lock this `MapIterator` and then `TaskMemoryManager` when going to free a memory page by calling `freePage`. At the same time, it is possibly that another memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it acquires memory and causes spilling on this `MapIterator`.

So it ends with the `MapIterator` object holds lock to the `MapIterator` object and waits for lock on `TaskMemoryManager`, and the other consumer holds lock to `TaskMemoryManager` and waits for lock on the `MapIterator` object.

To avoid deadlock here, this patch proposes to keep reference to the page to free and free it after releasing the lock of `MapIterator`.

## How was this patch tested?

Added test and manually test by running the test 100 times to make sure there is no deadlock.

Closes #23272 from viirya/SPARK-26265.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-11 21:08:39 +08:00
韩田田00222924 82c1ac48a3 [SPARK-25696] The storage memory displayed on spark Application UI is…
… incorrect.

## What changes were proposed in this pull request?
In the reported heartbeat information, the unit of the memory data is bytes, which is converted by the formatBytes() function in the utils.js file before being displayed in the interface. The cardinality of the unit conversion in the formatBytes function is 1000, which should be 1024.
Change the cardinality of the unit conversion in the formatBytes function to 1024.

## How was this patch tested?
 manual tests

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

Closes #22683 from httfighter/SPARK-25696.

Lead-authored-by: 韩田田00222924 <han.tiantian@zte.com.cn>
Co-authored-by: han.tiantian@zte.com.cn <han.tiantian@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-10 18:27:01 -06:00
Reza Safi 90c77ea313 [SPARK-24958][CORE] Add memory from procfs to executor metrics.
This adds the entire memory used by spark’s executor (as measured by procfs) to the executor metrics.  The memory usage is collected from the entire process tree under the executor.  The metrics are subdivided into memory used by java, by python, and by other processes, to aid users in diagnosing the source of high memory usage.
The additional metrics are sent to the driver in heartbeats, using the mechanism introduced by SPARK-23429.  This also slightly extends that approach to allow one ExecutorMetricType to collect multiple metrics.

Added unit tests and also tested on a live cluster.

Closes #22612 from rezasafi/ptreememory2.

Authored-by: Reza Safi <rezasafi@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-12-10 11:14:11 -06:00
liuxian 9794923272 [MINOR][DOC] Update the condition description of serialized shuffle
## What changes were proposed in this pull request?
`1. The shuffle dependency specifies no aggregation or output ordering.`
If the shuffle dependency specifies aggregation, but it only aggregates at the reduce-side, serialized shuffle can still be used.
`3. The shuffle produces fewer than 16777216 output partitions.`
If the number of output partitions is 16777216 , we can use serialized shuffle.

We can see this mothod: `canUseSerializedShuffle`
## How was this patch tested?
N/A

Closes #23228 from 10110346/SerializedShuffle_doc.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-10 22:37:17 +08:00
10087686 42e8c381b1 [SPARK-26286][TEST] Add MAXIMUM_PAGE_SIZE_BYTES exception bound unit test
## What changes were proposed in this pull request?
Add MAXIMUM_PAGE_SIZE_BYTES Exception test

## How was this patch tested?
Existing tests

(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 #23226 from wangjiaochun/BytesToBytesMapSuite.

Authored-by: 10087686 <wang.jiaochun@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-10 22:28:26 +08:00
10087686 403c8d5a60
[SPARK-26287][CORE] Don't need to create an empty spill file when memory has no records
## What changes were proposed in this pull request?
 If there are no records in memory, then we don't need to create an empty temp spill file.

## How was this patch tested?
Existing tests

(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 #23225 from wangjiaochun/ShufflSorter.

Authored-by: 10087686 <wang.jiaochun@zte.com.cn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-09 22:44:41 -08:00
Shahid ec506bd30c [SPARK-26283][CORE] Enable reading from open frames of zstd, when reading zstd compressed eventLog
## What changes were proposed in this pull request?
Root cause: Prior to Spark2.4, When we enable zst for eventLog compression, for inprogress application, It always throws exception in the Application UI, when we open from the history server. But after 2.4 it will display the UI information based on the completed frames in the zstd compressed eventLog. But doesn't read incomplete frames for inprogress application.
In this PR, we have added 'setContinous(true)' for reading input stream from eventLog, so that it can read from open frames also. (By default 'isContinous=false' for zstd inputStream and when we try to read an open frame, it throws truncated error)

## How was this patch tested?
Test steps:
1) Add the configurations in the spark-defaults.conf
   (i) spark.eventLog.compress true
   (ii) spark.io.compression.codec zstd
2) Restart history server
3) bin/spark-shell
4) sc.parallelize(1 to 1000, 1000).count
5) Open app UI from the history server UI

**Before fix**
![screenshot from 2018-12-06 00-01-38](https://user-images.githubusercontent.com/23054875/49537340-bfe28b00-f8ee-11e8-9fca-6d42fdc89e1a.png)

**After fix:**
![screenshot from 2018-12-06 00-34-39](https://user-images.githubusercontent.com/23054875/49537353-ca9d2000-f8ee-11e8-803d-645897b9153b.png)

Closes #23241 from shahidki31/zstdEventLog.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-09 11:44:16 -06:00
Yuanjian Li 877f82cb30 [SPARK-26193][SQL] Implement shuffle write metrics in SQL
## What changes were proposed in this pull request?

1. Implement `SQLShuffleWriteMetricsReporter` on the SQL side as the customized `ShuffleWriteMetricsReporter`.
2. Add shuffle write metrics to `ShuffleExchangeExec`, and use these metrics to create corresponding `SQLShuffleWriteMetricsReporter` in shuffle dependency.
3. Rework on `ShuffleMapTask` to add new class named `ShuffleWriteProcessor` which control shuffle write process, we use sql shuffle write metrics by customizing a ShuffleWriteProcessor on SQL side.

## How was this patch tested?
Add UT in SQLMetricsSuite.
Manually test locally, update screen shot to document attached in JIRA.

Closes #23207 from xuanyuanking/SPARK-26193.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-09 10:49:15 +08:00
Gabor Somogyi 9b1f6c8bab [SPARK-26304][SS] Add default value to spark.kafka.sasl.kerberos.service.name parameter
## What changes were proposed in this pull request?

spark.kafka.sasl.kerberos.service.name is an optional parameter but most of the time value `kafka` has to be set. As I've written in the jira the following reasoning is behind:
* Kafka's configuration guide suggest the same value: https://kafka.apache.org/documentation/#security_sasl_kerberos_brokerconfig
* It would be easier for spark users by providing less configuration
* Other streaming engines are doing the same

In this PR I've changed the parameter from optional to `WithDefault` and set `kafka` as default value.

## How was this patch tested?

Available unit tests + on cluster.

Closes #23254 from gaborgsomogyi/SPARK-26304.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-07 13:58:02 -08:00
Shahid 3b8ae23735 [SPARK-26196][SPARK-26281][WEBUI] Total tasks title in the stage page is incorrect when there are failed or killed tasks and update duration metrics
## What changes were proposed in this pull request?
This PR fixes 3 issues
1) Total tasks message in the tasks table is incorrect, when there are failed or killed tasks
2) Sorting of the "Duration" column is not correct
3) Duration in the aggregated tasks summary table and the tasks table and not matching.

Total tasks  = numCompleteTasks +  numActiveTasks + numKilledTasks + numFailedTasks;

Corrected the duration metrics in the tasks table as executorRunTime based on the PR https://github.com/apache/spark/pull/23081

## How was this patch tested?
test step:
1)
```
bin/spark-shell
scala > sc.parallelize(1 to 100, 10).map{ x => throw new RuntimeException("Bad executor")}.collect()
```
![screenshot from 2018-11-28 07-26-00](https://user-images.githubusercontent.com/23054875/49123523-e2691880-f2de-11e8-9c16-60d1865e6e77.png)

After patch:
![screenshot from 2018-11-28 07-24-31](https://user-images.githubusercontent.com/23054875/49123525-e432dc00-f2de-11e8-89ca-4a53e19c9c18.png)

2)  Duration metrics:
Before patch:
![screenshot from 2018-12-06 03-25-14](https://user-images.githubusercontent.com/23054875/49546591-9e8d9900-f906-11e8-8a0b-157742c47655.png)

After patch:
![screenshot from 2018-12-06 03-23-14](https://user-images.githubusercontent.com/23054875/49546589-9cc3d580-f906-11e8-827f-52ef8ffdeaec.png)

Closes #23160 from shahidki31/totalTasks.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-07 14:31:35 -06:00
10087686 9b7679a97e [SPARK-26294][CORE] Delete Unnecessary If statement
## What changes were proposed in this pull request?
Delete unnecessary If statement, because it Impossible execution when
records less than or equal to zero.it is only execution when records begin zero.
...................
if (inMemSorter == null || inMemSorter.numRecords() <= 0) {
       return 0L;
 }
....................
if (inMemSorter.numRecords() > 0) {
.....................
}
## How was this patch tested?
Existing tests

(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 #23247 from wangjiaochun/inMemSorter.

Authored-by: 10087686 <wang.jiaochun@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-07 14:11:25 -06:00
Sahil Takiar 543577a1e8 [SPARK-24243][CORE] Expose exceptions from InProcessAppHandle
Adds a new method to SparkAppHandle called getError which returns
the exception (if present) that caused the underlying Spark app to
fail.

New tests added to SparkLauncherSuite for the new method.

Closes #21849

Closes #23221 from vanzin/SPARK-24243.

Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-07 10:34:33 -08:00
caoxuewen bfc5569a53 [SPARK-26289][CORE] cleanup enablePerfMetrics parameter from BytesToBytesMap
## What changes were proposed in this pull request?

`enablePerfMetrics `was originally designed in `BytesToBytesMap `to control `getNumHashCollisions  getTimeSpentResizingNs  getAverageProbesPerLookup`.

However, as the Spark version gradual progress.  this parameter is only used for `getAverageProbesPerLookup ` and always given to true when using `BytesToBytesMap`.

 it is also dangerous to determine whether `getAverageProbesPerLookup `opens and throws an `IllegalStateException `exception.
So this pr will be remove `enablePerfMetrics `parameter from `BytesToBytesMap`. thanks.

## How was this patch tested?

the existed test cases.

Closes #23244 from heary-cao/enablePerfMetrics.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-07 09:57:35 +08:00
Marcelo Vanzin dbd90e5440 [SPARK-26194][K8S] Auto generate auth secret for k8s apps.
This change modifies the logic in the SecurityManager to do two
things:

- generate unique app secrets also when k8s is being used
- only store the secret in the user's UGI on YARN

The latter is needed so that k8s won't unnecessarily create
k8s secrets for the UGI credentials when only the auth token
is stored there.

On the k8s side, the secret is propagated to executors using
an environment variable instead. This ensures it works in both
client and cluster mode.

Security doc was updated to mention the feature and clarify that
proper access control in k8s should be enabled for it to be secure.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #23174 from vanzin/SPARK-26194.
2018-12-06 14:17:13 -08:00
Imran Rashid 180f969c97 [SPARK-26094][CORE][STREAMING] createNonEcFile creates parent dirs.
## What changes were proposed in this pull request?

We explicitly avoid files with hdfs erasure coding for the streaming WAL
and for event logs, as hdfs EC does not support all relevant apis.
However, the new builder api used has different semantics -- it does not
create parent dirs, and it does not resolve relative paths.  This
updates createNonEcFile to have similar semantics to the old api.

## How was this patch tested?

Ran tests with the WAL pointed at a non-existent dir, which failed before this change.  Manually tested the new function with a relative path as well.
Unit tests via jenkins.

Closes #23092 from squito/SPARK-26094.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-04 14:35:04 -08:00
Shahid 35f9163adf [SPARK-26119][CORE][WEBUI] Task summary table should contain only successful tasks' metrics
## What changes were proposed in this pull request?

Task summary table in the stage page currently displays the summary of all the tasks. However, we should display the task summary of only successful tasks, to follow the behavior of previous versions of spark.

## How was this patch tested?
Added UT. attached screenshot
Before patch:
![screenshot from 2018-11-20 00-36-18](https://user-images.githubusercontent.com/23054875/48729339-62e3a580-ec5d-11e8-81f0-0d191a234ffe.png)

![screenshot from 2018-11-20 01-18-37](https://user-images.githubusercontent.com/23054875/48731112-41d18380-ec62-11e8-8c31-1ffbfa04e746.png)

Closes #23088 from shahidki31/summaryMetrics.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-04 11:00:58 -08:00
Gengliang Wang 6e4e70fe7b [SPARK-26235][CORE] Change log level for ClassNotFoundException/NoClassDefFoundError in SparkSubmit to Error
## What changes were proposed in this pull request?

In my local setup, I set log4j root category as ERROR (https://stackoverflow.com/questions/27781187/how-to-stop-info-messages-displaying-on-spark-console , first item show up if we google search "set spark log level".) When I run such command
```
spark-submit --class foo bar.jar
```
Nothing shows up, and the script exits.

After quick investigation, I think the log level for ClassNotFoundException/NoClassDefFoundError in SparkSubmit should be ERROR instead of WARN. Since the whole process exit because of the exception/error.

Before https://github.com/apache/spark/pull/20925, the message is not controlled by `log4j.rootCategory`.

## How was this patch tested?

Manual check.

Closes #23189 from gengliangwang/changeLogLevel.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-03 10:02:15 -06:00
pgandhi eebb940edb [SPARK-26253][WEBUI] Task Summary Metrics Table on Stage Page shows empty table when no data is present
Task Summary Metrics Table on Stage Page shows empty table when no data is present instead of showing a message.

## What changes were proposed in this pull request?

Added a custom message to show on the task summary metrics table as well as executor summary table when no data is present.

## How was this patch tested?

**Before:**

![49335550-29277d00-f615-11e8-8e62-a953e76bcebf](https://user-images.githubusercontent.com/22228190/49361520-425a2780-f702-11e8-8df4-08862ab6ceb8.png)

**After:**

<img width="1413" alt="screen shot 2018-12-03 at 1 56 09 pm" src="https://user-images.githubusercontent.com/22228190/49362019-8699f780-f703-11e8-93e1-d02df6572923.png">

Closes #23205 from pgandhi999/SPARK-26253.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-03 07:53:21 -06:00
liuxian 60e4239a1e [MINOR][DOC] Correct some document description errors
## What changes were proposed in this pull request?

Correct some document description errors.

## How was this patch tested?
N/A

Closes #23162 from 10110346/docerror.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-01 07:11:31 -06:00
caoxuewen 327ac83f5c [SPARK-26180][CORE][TEST] Reuse withTempDir function to the SparkCore test case
## What changes were proposed in this pull request?

Currently, the common `withTempDir` function is used in Spark SQL test cases. To handle `val dir = Utils. createTempDir()` and `Utils. deleteRecursively (dir)`. Unfortunately, the `withTempDir` function cannot be used in the Spark Core test case. This PR Sharing `withTempDir` function in Spark Sql and SparkCore  to clean up SparkCore test cases. thanks.

## How was this patch tested?

N / A

Closes #23151 from heary-cao/withCreateTempDir.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-01 16:34:11 +08:00
Shahid 8856e9f6a3 [SPARK-26219][CORE] Executor summary should get updated for failure jobs in the history server UI
The root cause of the problem is, whenever the taskEnd event comes after stageCompleted event, execSummary is updating only for live UI. we need to update for history UI too.

To see the previous discussion, refer: PR for https://github.com/apache/spark/pull/23038, https://issues.apache.org/jira/browse/SPARK-26100.

Added UT. Manually verified

Test step to reproduce:

```
bin/spark-shell --master yarn --conf spark.executor.instances=3
sc.parallelize(1 to 10000, 10).map{ x => throw new RuntimeException("Bad executor")}.collect()
```

Open Executors page from the History UI

Before patch:
![screenshot from 2018-11-29 22-13-34](https://user-images.githubusercontent.com/23054875/49246338-a21ead00-f43a-11e8-8214-f1020420be52.png)

After patch:
![screenshot from 2018-11-30 00-54-49](https://user-images.githubusercontent.com/23054875/49246353-aa76e800-f43a-11e8-98ef-7faecaa7a50e.png)

Closes #23181 from shahidki31/executorUpdate.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-30 15:22:37 -08:00
schintap 9b23be2e95 [SPARK-26201] Fix python broadcast with encryption
## What changes were proposed in this pull request?
Python with rpc and disk encryption enabled along with a python broadcast variable and just read the value back on the driver side the job failed with:

Traceback (most recent call last): File "broadcast.py", line 37, in <module> words_new.value File "/pyspark.zip/pyspark/broadcast.py", line 137, in value File "pyspark.zip/pyspark/broadcast.py", line 122, in load_from_path File "pyspark.zip/pyspark/broadcast.py", line 128, in load EOFError: Ran out of input

To reproduce use configs: --conf spark.network.crypto.enabled=true --conf spark.io.encryption.enabled=true

Code:

words_new = sc.broadcast(["scala", "java", "hadoop", "spark", "akka"])
words_new.value
print(words_new.value)

## How was this patch tested?
words_new = sc.broadcast([“scala”, “java”, “hadoop”, “spark”, “akka”])
textFile = sc.textFile(“README.md”)
wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word + words_new.value[1], 1)).reduceByKey(lambda a, b: a+b)
 count = wordCounts.count()
 print(count)
 words_new.value
 print(words_new.value)

Closes #23166 from redsanket/SPARK-26201.

Authored-by: schintap <schintap@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-11-30 12:48:56 -06:00
Gabor Somogyi 0166c7373e [SPARK-25501][SS] Add kafka delegation token support.
## What changes were proposed in this pull request?

It adds kafka delegation token support for structured streaming. Please see the relevant [SPIP](https://docs.google.com/document/d/1ouRayzaJf_N5VQtGhVq9FURXVmRpXzEEWYHob0ne3NY/edit?usp=sharing)

What this PR contains:
* Configuration parameters for the feature
* Delegation token fetching from broker
* Usage of token through dynamic JAAS configuration
* Minor refactoring in the existing code

What this PR doesn't contain:
* Documentation changes because design can change

## How was this patch tested?

Existing tests + added small amount of additional unit tests.

Because it's an external service integration mainly tested on cluster.
* 4 node cluster
* Kafka broker version 1.1.0
* Topic with 4 partitions
* security.protocol = SASL_SSL
* sasl.mechanism = SCRAM-SHA-256

An example of obtaining a token:
```
18/10/01 01:07:49 INFO kafka010.TokenUtil: TOKENID         HMAC                           OWNER           RENEWERS                  ISSUEDATE       EXPIRYDATE      MAXDATE
18/10/01 01:07:49 INFO kafka010.TokenUtil: D1-v__Q5T_uHx55rW16Jwg [hidden] User:user    []                        2018-10-01T01:07 2018-10-02T01:07 2018-10-08T01:07
18/10/01 01:07:49 INFO security.KafkaDelegationTokenProvider: Get token from Kafka: Kind: KAFKA_DELEGATION_TOKEN, Service: kafka.server.delegation.token, Ident: 44 31 2d 76 5f 5f 51 35 54 5f 75 48 78 35 35 72 57 31 36 4a 77 67
```

An example token usage:
```
18/10/01 01:08:07 INFO kafka010.KafkaSecurityHelper: Scram JAAS params: org.apache.kafka.common.security.scram.ScramLoginModule required tokenauth=true serviceName="kafka" username="D1-v__Q5T_uHx55rW16Jwg" password="[hidden]";
18/10/01 01:08:07 INFO kafka010.KafkaSourceProvider: Delegation token detected, using it for login.
```

Closes #22598 from gaborgsomogyi/SPARK-25501.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-29 18:00:47 -08:00
Wing Yew Poon 59741887e2 [SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to a ChunkedByteBuffer
## What changes were proposed in this pull request?

In `BlockManager`, `getRemoteValues` gets a `ChunkedByteBuffer` (by calling `getRemoteBytes`) and creates an `InputStream` from it. `getRemoteBytes`, in turn, gets a `ManagedBuffer` and converts it to a `ChunkedByteBuffer`.
Instead, expose a `getRemoteManagedBuffer` method so `getRemoteValues` can just get this `ManagedBuffer` and use its `InputStream`.
When reading a remote cache block from disk, this reduces heap memory usage significantly.
Retain `getRemoteBytes` for other callers.

## How was this patch tested?

Imran Rashid wrote an application (https://github.com/squito/spark_2gb_test/blob/master/src/main/scala/com/cloudera/sparktest/LargeBlocks.scala), that among other things, tests reading remote cache blocks. I ran this application, using 2500MB blocks, to test reading a cache block on disk. Without this change, with `--executor-memory 5g`, the test fails with `java.lang.OutOfMemoryError: Java heap space`. With the change, the test passes with `--executor-memory 2g`.
I also ran the unit tests in core. In particular, `DistributedSuite` has a set of tests that exercise the `getRemoteValues` code path. `BlockManagerSuite` has several tests that call `getRemoteBytes`; I left these unchanged, so `getRemoteBytes` still gets exercised.

Closes #23058 from wypoon/SPARK-25905.

Authored-by: Wing Yew Poon <wypoon@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-11-29 14:56:34 -06:00
Shahid 24e78b7f16 [SPARK-26186][SPARK-26184][CORE] Last updated time is not getting updated for the Inprogress application
## What changes were proposed in this pull request?

When the 'spark.history.fs.inProgressOptimization.enabled' is true, inProgress application's last updated time is not getting updated in the History UI. Also, during the cleaning time, InProgress application is getting removed from the listing, even if the last updated time is within the cleaning threshold time.

In this PR, if the fastInprogressOptimization enabled, we update the `lastUpdateTime` of the application as last scan time. This will update the `lastUpdateTime` in the historyUI and also while cleaning, it won't remove if the updateTime is within the cleaning interval

## How was this patch tested?
Added UT, attached screen shot.
Before patch:
![screenshot from 2018-11-27 23-22-38](https://user-images.githubusercontent.com/23054875/49101600-9b5a3380-f29c-11e8-8efc-3fb594e4279a.png)
![screenshot from 2018-11-27 23-20-11](https://user-images.githubusercontent.com/23054875/49101601-9c8b6080-f29c-11e8-928e-643a8c8f4477.png)

After Patch:
![screenshot from 2018-11-27 23-37-10](https://user-images.githubusercontent.com/23054875/49101911-669aac00-f29d-11e8-8181-663e4a08ab0e.png)
![screenshot from 2018-11-27 23-39-04](https://user-images.githubusercontent.com/23054875/49102010-a5306680-f29d-11e8-947a-e8a2a09a785a.png)

Closes #23158 from shahidki31/HistoryLastUpdateTime.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-29 09:48:18 -08:00
Brandon Krieger 87bd9c75df [SPARK-25998][CORE] Change TorrentBroadcast to hold weak reference of broadcast object
## What changes were proposed in this pull request?

This PR changes the broadcast object in TorrentBroadcast from a strong reference to a weak reference. This allows it to be garbage collected even if the Dataset is held in memory. This is ok, because the broadcast object can always be re-read.

## How was this patch tested?

Tested in Spark shell by taking a heap dump, full repro steps listed in https://issues.apache.org/jira/browse/SPARK-25998.

Closes #22995 from bkrieger/bk/torrent-broadcast-weak.

Authored-by: Brandon Krieger <bkrieger@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-28 07:22:48 -08:00