-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-26288][CORE] Restore RegisteredExecutors information for External shuffle service in Standalone/Kubernetes backend when the service is restarted #23393
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 12 commits
105a321
b9ff5aa
628ffa5
d1b54fc
2ba3695
8424852
d53c52a
c681bfb
29804eb
7e87f64
7b8b31c
cb78728
ed9a842
1f94b86
475d278
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,7 +30,7 @@ private[spark] object Worker { | |
|
|
||
| val WORKER_CLEANUP_ENABLED = ConfigBuilder("spark.worker.cleanup.enabled") | ||
| .booleanConf | ||
| .createWithDefault(false) | ||
| .createWithDefault(true) | ||
|
||
|
|
||
| val WORKER_CLEANUP_INTERVAL = ConfigBuilder("spark.worker.cleanup.interval") | ||
| .longConf | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,140 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.deploy | ||
|
|
||
| import java.io._ | ||
| import java.nio.charset.StandardCharsets | ||
|
|
||
| import com.google.common.io.CharStreams | ||
|
|
||
| import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} | ||
| import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleBlockResolver} | ||
| import org.apache.spark.network.shuffle.TestShuffleDataContext | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| /** | ||
| * This suite gets BlockData when the ExternalShuffleService is restarted | ||
| * with #spark.shuffle.service.db.enabled = true or false | ||
| * Note that failures in this suite may arise when#spark.shuffle.service.db.enabled = false | ||
| */ | ||
| class ExternalShuffleServiceDbSuite extends SparkFunSuite { | ||
| val sortBlock0 = "Hello!" | ||
| val sortBlock1 = "World!" | ||
| val SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager" | ||
|
|
||
| var sparkConf: SparkConf = _ | ||
| var dataContext: TestShuffleDataContext = _ | ||
|
|
||
| var securityManager: SecurityManager = _ | ||
| var externalShuffleService: ExternalShuffleService = _ | ||
| var blockHandler: ExternalShuffleBlockHandler = _ | ||
| var blockResolver: ExternalShuffleBlockResolver = _ | ||
|
|
||
| override def beforeAll() { | ||
| super.beforeAll() | ||
| sparkConf = new SparkConf() | ||
| sparkConf.set("spark.shuffle.service.enabled", "true") | ||
| sparkConf.set("spark.local.dir", System.getProperty("java.io.tmpdir")) | ||
| Utils.loadDefaultSparkProperties(sparkConf, null) | ||
| securityManager = new SecurityManager(sparkConf) | ||
|
|
||
| dataContext = new TestShuffleDataContext(2, 5) | ||
| dataContext.create() | ||
| // Write some sort data. | ||
| dataContext.insertSortShuffleData(0, 0, | ||
| Array[Array[Byte]](sortBlock0.getBytes(StandardCharsets.UTF_8), | ||
| sortBlock1.getBytes(StandardCharsets.UTF_8))) | ||
| registerExecutor() | ||
| } | ||
|
|
||
| override def afterAll() { | ||
| try { | ||
| dataContext.cleanup() | ||
| } finally { | ||
| super.afterAll() | ||
| } | ||
| } | ||
|
|
||
| def registerExecutor(): Unit = { | ||
| try { | ||
| sparkConf.set("spark.shuffle.service.db.enabled", "true") | ||
| externalShuffleService = new ExternalShuffleService(sparkConf, securityManager) | ||
|
|
||
| // external Shuffle Service start | ||
| externalShuffleService.start() | ||
| blockHandler = externalShuffleService.getBlockHandler | ||
| blockResolver = blockHandler.getBlockResolver | ||
| blockResolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)) | ||
| } finally { | ||
| blockHandler.close() | ||
| // external Shuffle Service stop | ||
| externalShuffleService.stop() | ||
| } | ||
| } | ||
|
|
||
| // The beforeAll ensures the shuffle data was already written, and then | ||
| // the shuffle service was stopped. Here we restart the shuffle service | ||
| // and make we can read the shuffle data | ||
| test("Recover shuffle data with spark.shuffle.service.db.enabled=true after " + | ||
| "shuffle service restart") { | ||
| try { | ||
| sparkConf.set("spark.shuffle.service.db.enabled", "true") | ||
| externalShuffleService = new ExternalShuffleService(sparkConf, securityManager) | ||
| // externalShuffleService restart | ||
| externalShuffleService.start() | ||
| blockHandler = externalShuffleService.getBlockHandler | ||
| blockResolver = blockHandler.getBlockResolver | ||
|
|
||
| val block0Stream = blockResolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream | ||
| val block0 = CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)) | ||
| block0Stream.close() | ||
| assert(sortBlock0 == block0) | ||
| // pass | ||
| } finally { | ||
| blockHandler.close() | ||
| // externalShuffleService stop | ||
| externalShuffleService.stop() | ||
| } | ||
|
|
||
| } | ||
|
|
||
| // The beforeAll ensures the shuffle data was already written, and then | ||
| // the shuffle service was stopped. Here we restart the shuffle service , | ||
| // but we can't read the shuffle data | ||
| test("Can't recover shuffle data with spark.shuffle.service.db.enabled=false after" + | ||
| " shuffle service restart") { | ||
| try { | ||
| sparkConf.set("spark.shuffle.service.db.enabled", "false") | ||
| externalShuffleService = new ExternalShuffleService(sparkConf, securityManager) | ||
| // externalShuffleService restart | ||
| externalShuffleService.start() | ||
| blockHandler = externalShuffleService.getBlockHandler | ||
| blockResolver = blockHandler.getBlockResolver | ||
|
|
||
| val error = intercept[RuntimeException] { | ||
| blockResolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream | ||
| }.getMessage | ||
|
|
||
| assert(error.contains("not registered")) | ||
| } finally { | ||
| blockHandler.close() | ||
| // externalShuffleService stop | ||
| externalShuffleService.stop() | ||
| } | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change | ||||||
|---|---|---|---|---|---|---|---|---|
|
|
@@ -236,10 +236,11 @@ SPARK_WORKER_OPTS supports the following system properties: | |||||||
| <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr> | ||||||||
| <tr> | ||||||||
| <td><code>spark.worker.cleanup.enabled</code></td> | ||||||||
| <td>false</td> | ||||||||
| <td>true</td> | ||||||||
|
||||||||
| val WORKER_CLEANUP_ENABLED = ConfigBuilder("spark.worker.cleanup.enabled") | |
| .booleanConf | |
| .createWithDefault(false) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sorry,i missed it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I believe attila meant to revert your change to the docs, not to change the default
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Some minor rewordings:
Store External Shuffle service state on local disk so that when the external shuffle service is restarted, it will
automatically reload info on current executors. This only affects standalone mode (yarn always has this behavior
enabled). You should also enable <code>spark.worker.cleanup.enabled</code>, to ensure that the state
eventually gets cleaned up. This config may be removed in the future.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Very clear description, thank you.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is unused now, right? You can undo this change?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes,your are right,i have fixxed it.