-
Notifications
You must be signed in to change notification settings - Fork 51
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
[SPARK-25299] Driver lifecycle api #533
Changes from all commits
bebeea4
f6e84de
6f41113
0d0f368
6cb592a
8d362b9
d8194a4
4ee7954
b370172
111d5ad
484cf3f
0a94436
2dd5519
29b9a6c
89d0a6c
121c112
6dc0a24
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 |
---|---|---|
@@ -0,0 +1,33 @@ | ||
/* | ||
* 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.api.shuffle; | ||
|
||
import java.io.IOException; | ||
import java.util.Map; | ||
|
||
public interface ShuffleDriverComponents { | ||
|
||
/** | ||
* @return additional SparkConf values necessary for the executors. | ||
*/ | ||
Map<String, String> initializeApplication(); | ||
|
||
void cleanupApplication() throws IOException; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Any reason for this to throw There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm i guess i was expecting this to include something that would make a call somewhere or hit the file system? I can remove it for now |
||
|
||
void removeShuffleData(int shuffleId, boolean blocking) throws IOException; | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,54 @@ | ||
/* | ||
* 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.shuffle.sort.lifecycle; | ||
|
||
import com.google.common.collect.ImmutableMap; | ||
import org.apache.spark.SparkEnv; | ||
import org.apache.spark.api.shuffle.ShuffleDriverComponents; | ||
import org.apache.spark.storage.BlockManagerMaster; | ||
|
||
import java.io.IOException; | ||
import java.util.Map; | ||
|
||
public class DefaultShuffleDriverComponents implements ShuffleDriverComponents { | ||
|
||
private BlockManagerMaster blockManagerMaster; | ||
|
||
@Override | ||
public Map<String, String> initializeApplication() { | ||
blockManagerMaster = SparkEnv.get().blockManager().master(); | ||
return ImmutableMap.of(); | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I guess the idea is that for a truly external shuffle service, here the driver would register with that service (including any authentication etc.) and setup heartbeats? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, that would be for non-default implementations, like the external implementation I experimented with in my old PR |
||
|
||
@Override | ||
public void cleanupApplication() { | ||
// do nothing | ||
} | ||
|
||
@Override | ||
public void removeShuffleData(int shuffleId, boolean blocking) throws IOException { | ||
checkInitialized(); | ||
blockManagerMaster.removeShuffle(shuffleId, blocking); | ||
} | ||
|
||
private void checkInitialized() { | ||
if (blockManagerMaster == null) { | ||
throw new IllegalStateException("Driver components must be initialized before using"); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
/* | ||
* 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.shuffle | ||
|
||
import java.util | ||
|
||
import com.google.common.collect.ImmutableMap | ||
|
||
import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite} | ||
import org.apache.spark.api.shuffle.{ShuffleDataIO, ShuffleDriverComponents, ShuffleExecutorComponents, ShuffleWriteSupport} | ||
import org.apache.spark.internal.config.SHUFFLE_IO_PLUGIN_CLASS | ||
import org.apache.spark.shuffle.sort.io.DefaultShuffleWriteSupport | ||
|
||
class ShuffleDriverComponentsSuite extends SparkFunSuite with LocalSparkContext { | ||
test(s"test serialization of shuffle initialization conf to executors") { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. string interpolation |
||
val testConf = new SparkConf() | ||
.setAppName("testing") | ||
.setMaster("local-cluster[2,1,1024]") | ||
.set(SHUFFLE_IO_PLUGIN_CLASS, "org.apache.spark.shuffle.TestShuffleDataIO") | ||
|
||
sc = new SparkContext(testConf) | ||
|
||
sc.parallelize(Seq((1, "one"), (2, "two"), (3, "three")), 3) | ||
.groupByKey() | ||
.collect() | ||
} | ||
} | ||
|
||
class TestShuffleDriverComponents extends ShuffleDriverComponents { | ||
override def initializeApplication(): util.Map[String, String] = | ||
ImmutableMap.of("test-key", "test-value") | ||
|
||
override def cleanupApplication(): Unit = {} | ||
|
||
override def removeShuffleData(shuffleId: Int, blocking: Boolean): Unit = {} | ||
} | ||
|
||
class TestShuffleDataIO(sparkConf: SparkConf) extends ShuffleDataIO { | ||
override def driver(): ShuffleDriverComponents = new TestShuffleDriverComponents() | ||
|
||
override def executor(): ShuffleExecutorComponents = | ||
new TestShuffleExecutorComponents(sparkConf) | ||
} | ||
|
||
class TestShuffleExecutorComponents(sparkConf: SparkConf) extends ShuffleExecutorComponents { | ||
override def initializeExecutor(appId: String, execId: String, | ||
extraConfigs: util.Map[String, String]): Unit = { | ||
assert(extraConfigs.get("test-key") == "test-value") | ||
} | ||
|
||
override def writes(): ShuffleWriteSupport = { | ||
val blockManager = SparkEnv.get.blockManager | ||
val blockResolver = new IndexShuffleBlockResolver(sparkConf, blockManager) | ||
new DefaultShuffleWriteSupport(sparkConf, blockResolver) | ||
} | ||
} |
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.
Let's move
driver
aboveexecutor
. I don't have a great reason for this except that this is how I tend to think about Spark in general though -> driver comes before executors...