Skip to content

Conversation

@jsoltren
Copy link

@jsoltren jsoltren commented Dec 20, 2016

Builds on top of work in SPARK-8425 to update Application Level Blacklisting in the scheduler.

What changes were proposed in this pull request?

Adds a UI to these patches by:

  • defining new listener events for blacklisting and unblacklisting, nodes and executors;
  • sending said events at the relevant points in BlacklistTracker;
  • adding JSON (de)serialization code for these events;
  • augmenting the Executors UI page to show which, and how many, executors are blacklisted;
  • adding a unit test to make sure events are being fired;
  • adding HistoryServerSuite coverage to verify that the SHS reads these events correctly.
  • updates the Executor UI to show Blacklisted/Active/Dead as a tri-state in Executors Status

Updates .rat-excludes to pass tests.

@username squito

How was this patch tested?

./dev/run-tests
testOnly org.apache.spark.util.JsonProtocolSuite
testOnly org.apache.spark.scheduler.BlacklistTrackerSuite
testOnly org.apache.spark.deploy.history.HistoryServerSuite
https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh
blacklist-20161219

Builds on top of work in SPARK-8425 to update
Application Level Blacklisting in the scheduler.

Adds a UI to these patches by:
- defining new listener events for blacklisting and unblacklisting,
  nodes and executors;
- sending said events at the relevant points in BlacklistTracker;
- adding JSON (de)serialization code for these events;
- augmenting the Executors UI page to show which, and how many,
  executors are blacklisted;
- adding a unit test to make sure events are being fired;
- adding HistoryServerSuite coverage to verify that the SHS reads
  these events correctly.
- updates the Executor UI to show Blacklisted/Active/Dead
  as a tri-state in Executors Status

Updates .rat-excludes to pass tests.

@username squito
@vanzin
Copy link
Contributor

vanzin commented Dec 20, 2016

ok to test

@SparkQA
Copy link

SparkQA commented Dec 20, 2016

Test build #70393 has finished for PR 16346 at commit 20ff7dd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class SparkListenerExecutorBlacklisted(
  • case class SparkListenerExecutorUnblacklisted(time: Long, executorId: String)
  • case class SparkListenerNodeBlacklisted(
  • case class SparkListenerNodeUnblacklisted(time: Long, nodeId: String)

s"has timed out")
nodeIdToBlacklistExpiryTime --= nodesToUnblacklist
nodesToUnblacklist.foreach { node =>
nodeIdToBlacklistExpiryTime.remove(node)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is already done two lines above?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, that's an error. The foreach block is correct so that we can send an event on the following line.

val node = failuresInTaskSet.node
executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTimeForNewBlacklists))
listenerBus.post(
SparkListenerExecutorBlacklisted(now, exec, newTotal))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: fits in previous line

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

clock: Clock = new SystemClock()) extends Logging {

def this(sc: SparkContext) = {
this(sc.listenerBus, sc.getConf)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably ok to use sc.conf here to avoid another copy of the conf instance.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.


@DeveloperApi
case class SparkListenerNodeBlacklisted(
time: Long,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm... I don't remember nodeId being a thing in Spark. Do you mean a hostname here, or ip address, or something else? Should probably use a better name, or at least document what it is.

(I see BlacklistTracker uses that, but as a user, I don't know what "nodeId" is.)

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We've been using "node" and "host" interchangeably in this change set. To be more clear, I'll write this as hostId here and in SparkListenerNodeUnblacklisted.

/*
Implicitly blacklist every executor associated with this node, and show this in the UI.
*/
activeStorageStatusList.foreach{status =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: foreach { status =>

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

they may have been blacklisted initially (either explicitly through executor blacklisting
or implicitly through node blacklisting). Show this in the UI.
*/
activeStorageStatusList.foreach{status =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: foreach { status =>

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.


clock.setTime(0)
blacklist = new BlacklistTracker(conf, clock)
blacklist = new BlacklistTracker(null, conf, clock)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this tracker is ever used, won't you get NPEs because the listener bus is null?

This basically means you're forcing every test to call configureBlacklistAndScheduler() manually. So either remove this initialization, or do this properly, and just call configureBlacklistAndScheduler() for tests that need to override the default config.

Since I don't see any test actually passing any custom config parameters, might as well do the initialization properly here and get rid of configureBlacklistAndScheduler() altogether.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Correct. For now, I've left configureBlacklistAndScheduler() in place, since the two tests at the end do not call it.

// We don't directly use the application blacklist, but its presence triggers blacklisting
// within the taskset.
val blacklistTrackerOpt = Some(new BlacklistTracker(conf, clock))
val blacklistTrackerOpt = Some(new BlacklistTracker(null, conf, clock))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same comment regarding the null listener bus. I'd be more confident if you provided something here (even if it's a mocked bus you never actually check).

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure.


override def onNodeUnblacklisted(nodeUnblacklisted: SparkListenerNodeUnblacklisted)
: Unit = synchronized {
/*
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: the preference is to use line comments for these.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

@SparkQA
Copy link

SparkQA commented Jan 11, 2017

Test build #71226 has finished for PR 16346 at commit d3d3583.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class SparkListenerNodeUnblacklisted(time: Long, hostId: String)

Copy link
Contributor

@squito squito left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry I am only just thinking of this now, but I think it would also be nice for that executor summary table to also list the blacklisted executors. Otherwise, just some minor changes.

(nevermind, I didn't notice the "Blacklisted" column)

}

override def onExecutorBlacklisted(executorBlacklisted: SparkListenerExecutorBlacklisted)
: Unit = synchronized {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: move param to second line, and double-indent. (and the other added methods in this file)

override def onExecutorBlacklisted(
    executorBlacklisted: SparkListenerExecutorBlacklisted): Unit = synchronized {

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know the Spark project has some style standards. It would be great if these could be expressed programmatically, and if failing to meet these standards caused a compile time error.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you can run dev/scalastyle to check but it still misses some things

new TaskSetBlacklist(conf, stageId, clock)
}

def configureBlacklistAndScheduler(confs: (String, String)*): Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

since you're never passing in confs, couldn't you get rid of this method, and just change the initialization in beforeEach to

listenerBusMock = mock[LiveListenerBus]
blacklist = new BlacklistTracker(listenerBusMock, conf, clock)

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. The code is cleaner this way.

}

test("Blacklisting individual tasks and checking for SparkListenerEvents") {
configureBlacklistAndScheduler()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the rest of the test code here has changed a lot since you first started working on this, but I'd avoid putting in a whole test case just for this. Instead, I'd look at the existing test cases, find all occurrences of blacklist.isNodeBlacklisted and blacklist.isExecutorBlacklisted, and add appropriate verify(listenerBusMock) calls around all of those.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Certainly this doesn't need a dedicated test. But there will be a large number - dozens - of listenerBusMock checks for blacklisting and unblacklisting. I'm working on this now, but, there will be a substantial diff here.

A freebie is that by checking for the listener bus events we're also asserting that executors and nodes are being blacklisted with the correct number of failures, and we're making that number explicit.

@squito
Copy link
Contributor

squito commented Jan 12, 2017

cc @ajbozarth as you were interested in this earlier

@ajbozarth
Copy link
Member

I haven't been able to take a detailed look at the code, but all of my UI concerns seemed to have been addressed in the previous pr. I am wondering what you mean by having the summary table list the blacklisted executors though @squito

@squito
Copy link
Contributor

squito commented Jan 12, 2017

@ajbozarth whoops, I hadn't noticed the extra "Blacklisted" column in the summary table at the top -- I was thinking we'd add another row for blacklisted executors. But I actually think the current version is better. So ignore that comment :) (mentioned the same to @jsoltren directly)

@SparkQA
Copy link

SparkQA commented Jan 18, 2017

Test build #71541 has finished for PR 16346 at commit 6e2c59f.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@squito squito left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

a couple of really minor things left, otherwise lgtm!

}

private def updateExecutorBlacklist(
eid: String, isBlacklisted: Boolean): Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: when multi-line, each arg goes on its own line

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed.

: Unit = synchronized {
/*
Implicitly blacklist every executor associated with this node, and show this in the UI.
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: single line comment //

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed.

new TaskSetBlacklist(conf, stageId, clock)
}

def configureBlacklistAndScheduler: Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

now that you're doing the equivalent thing in beforeEach, this entire method is unnecessary. beforeEach gets called automatically for each test so you don't need to call this explicitly.

sorry I was unclear about this earlier.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense, fixed.

@squito
Copy link
Contributor

squito commented Jan 18, 2017

the test failures look real and from the changes you've made, can you take a look?

Strikes nodeId and replaces with hostId in resources for JSON tests.
@jsoltren
Copy link
Author

The test failures are due to s/nodeId/hostId/ in the JSON code but not the tests. This is passing locally now with the changes I'm about to push to this PR. Thanks.

@SparkQA
Copy link

SparkQA commented Jan 18, 2017

Test build #71609 has finished for PR 16346 at commit 701ebb7.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@asfgit asfgit closed this in 640f942 Jan 19, 2017
@squito
Copy link
Contributor

squito commented Jan 19, 2017

merged to master, thanks @jsoltren!

uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
Builds on top of work in SPARK-8425 to update Application Level Blacklisting in the scheduler.

## What changes were proposed in this pull request?

Adds a UI to these patches by:
- defining new listener events for blacklisting and unblacklisting, nodes and executors;
- sending said events at the relevant points in BlacklistTracker;
- adding JSON (de)serialization code for these events;
- augmenting the Executors UI page to show which, and how many, executors are blacklisted;
- adding a unit test to make sure events are being fired;
- adding HistoryServerSuite coverage to verify that the SHS reads these events correctly.
- updates the Executor UI to show Blacklisted/Active/Dead as a tri-state in Executors Status

Updates .rat-excludes to pass tests.

username squito

## How was this patch tested?

./dev/run-tests
testOnly org.apache.spark.util.JsonProtocolSuite
testOnly org.apache.spark.scheduler.BlacklistTrackerSuite
testOnly org.apache.spark.deploy.history.HistoryServerSuite
https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh
![blacklist-20161219](https://cloud.githubusercontent.com/assets/1208477/21335321/9eda320a-c623-11e6-8b8c-9c912a73c276.jpg)

Author: José Hiram Soltren <[email protected]>

Closes apache#16346 from jsoltren/SPARK-16654-submit.
cmonkey pushed a commit to cmonkey/spark that referenced this pull request Feb 15, 2017
Builds on top of work in SPARK-8425 to update Application Level Blacklisting in the scheduler.

## What changes were proposed in this pull request?

Adds a UI to these patches by:
- defining new listener events for blacklisting and unblacklisting, nodes and executors;
- sending said events at the relevant points in BlacklistTracker;
- adding JSON (de)serialization code for these events;
- augmenting the Executors UI page to show which, and how many, executors are blacklisted;
- adding a unit test to make sure events are being fired;
- adding HistoryServerSuite coverage to verify that the SHS reads these events correctly.
- updates the Executor UI to show Blacklisted/Active/Dead as a tri-state in Executors Status

Updates .rat-excludes to pass tests.

username squito

## How was this patch tested?

./dev/run-tests
testOnly org.apache.spark.util.JsonProtocolSuite
testOnly org.apache.spark.scheduler.BlacklistTrackerSuite
testOnly org.apache.spark.deploy.history.HistoryServerSuite
https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh
![blacklist-20161219](https://cloud.githubusercontent.com/assets/1208477/21335321/9eda320a-c623-11e6-8b8c-9c912a73c276.jpg)

Author: José Hiram Soltren <[email protected]>

Closes apache#16346 from jsoltren/SPARK-16654-submit.
yoonlee95 pushed a commit to yoonlee95/spark that referenced this pull request Aug 17, 2017
Builds on top of work in SPARK-8425 to update Application Level Blacklisting in the scheduler.

## What changes were proposed in this pull request?

Adds a UI to these patches by:
- defining new listener events for blacklisting and unblacklisting, nodes and executors;
- sending said events at the relevant points in BlacklistTracker;
- adding JSON (de)serialization code for these events;
- augmenting the Executors UI page to show which, and how many, executors are blacklisted;
- adding a unit test to make sure events are being fired;
- adding HistoryServerSuite coverage to verify that the SHS reads these events correctly.
- updates the Executor UI to show Blacklisted/Active/Dead as a tri-state in Executors Status

Updates .rat-excludes to pass tests.

username squito

## How was this patch tested?

./dev/run-tests
testOnly org.apache.spark.util.JsonProtocolSuite
testOnly org.apache.spark.scheduler.BlacklistTrackerSuite
testOnly org.apache.spark.deploy.history.HistoryServerSuite
https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh
![blacklist-20161219](https://cloud.githubusercontent.com/assets/1208477/21335321/9eda320a-c623-11e6-8b8c-9c912a73c276.jpg)

Author: José Hiram Soltren <[email protected]>

Closes apache#16346 from jsoltren/SPARK-16654-submit.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants