Skip to content

Commit e9f3f62

Browse files
attilapirosMarcelo Vanzin
authored andcommitted
[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” <[email protected]> Signed-off-by: Marcelo Vanzin <[email protected]>
1 parent 1a68fc3 commit e9f3f62

32 files changed

+1392
-435
lines changed

common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -91,6 +91,9 @@ protected void channelRead0(
9191
try {
9292
streamManager.checkAuthorization(client, msg.streamChunkId.streamId);
9393
buf = streamManager.getChunk(msg.streamChunkId.streamId, msg.streamChunkId.chunkIndex);
94+
if (buf == null) {
95+
throw new IllegalStateException("Chunk was not found");
96+
}
9497
} catch (Exception e) {
9598
logger.error(String.format("Error opening block %s for request from %s",
9699
msg.streamChunkId, getRemoteAddress(channel)), e);

common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -125,7 +125,10 @@ public void connectionTerminated(Channel channel) {
125125

126126
// Release all remaining buffers.
127127
while (state.buffers.hasNext()) {
128-
state.buffers.next().release();
128+
ManagedBuffer buffer = state.buffers.next();
129+
if (buffer != null) {
130+
buffer.release();
131+
}
129132
}
130133
}
131134
}

common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java

Lines changed: 26 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323

2424
import io.netty.channel.Channel;
2525
import org.apache.spark.network.server.ChunkFetchRequestHandler;
26+
import org.junit.Assert;
2627
import org.junit.Test;
2728

2829
import static org.mockito.Mockito.*;
@@ -45,9 +46,8 @@ public void handleChunkFetchRequest() throws Exception {
4546
Channel channel = mock(Channel.class);
4647
ChannelHandlerContext context = mock(ChannelHandlerContext.class);
4748
when(context.channel())
48-
.thenAnswer(invocationOnMock0 -> {
49-
return channel;
50-
});
49+
.thenAnswer(invocationOnMock0 -> channel);
50+
5151
List<Pair<Object, ExtendedChannelPromise>> responseAndPromisePairs =
5252
new ArrayList<>();
5353
when(channel.writeAndFlush(any()))
@@ -62,6 +62,7 @@ public void handleChunkFetchRequest() throws Exception {
6262
List<ManagedBuffer> managedBuffers = new ArrayList<>();
6363
managedBuffers.add(new TestManagedBuffer(10));
6464
managedBuffers.add(new TestManagedBuffer(20));
65+
managedBuffers.add(null);
6566
managedBuffers.add(new TestManagedBuffer(30));
6667
managedBuffers.add(new TestManagedBuffer(40));
6768
long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel);
@@ -71,31 +72,40 @@ public void handleChunkFetchRequest() throws Exception {
7172

7273
RequestMessage request0 = new ChunkFetchRequest(new StreamChunkId(streamId, 0));
7374
requestHandler.channelRead(context, request0);
74-
assert responseAndPromisePairs.size() == 1;
75-
assert responseAndPromisePairs.get(0).getLeft() instanceof ChunkFetchSuccess;
76-
assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(0).getLeft())).body() ==
77-
managedBuffers.get(0);
75+
Assert.assertEquals(1, responseAndPromisePairs.size());
76+
Assert.assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof ChunkFetchSuccess);
77+
Assert.assertEquals(managedBuffers.get(0),
78+
((ChunkFetchSuccess) (responseAndPromisePairs.get(0).getLeft())).body());
7879

7980
RequestMessage request1 = new ChunkFetchRequest(new StreamChunkId(streamId, 1));
8081
requestHandler.channelRead(context, request1);
81-
assert responseAndPromisePairs.size() == 2;
82-
assert responseAndPromisePairs.get(1).getLeft() instanceof ChunkFetchSuccess;
83-
assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(1).getLeft())).body() ==
84-
managedBuffers.get(1);
82+
Assert.assertEquals(2, responseAndPromisePairs.size());
83+
Assert.assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof ChunkFetchSuccess);
84+
Assert.assertEquals(managedBuffers.get(1),
85+
((ChunkFetchSuccess) (responseAndPromisePairs.get(1).getLeft())).body());
8586

8687
// Finish flushing the response for request0.
8788
responseAndPromisePairs.get(0).getRight().finish(true);
8889

8990
RequestMessage request2 = new ChunkFetchRequest(new StreamChunkId(streamId, 2));
9091
requestHandler.channelRead(context, request2);
91-
assert responseAndPromisePairs.size() == 3;
92-
assert responseAndPromisePairs.get(2).getLeft() instanceof ChunkFetchSuccess;
93-
assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(2).getLeft())).body() ==
94-
managedBuffers.get(2);
92+
Assert.assertEquals(3, responseAndPromisePairs.size());
93+
Assert.assertTrue(responseAndPromisePairs.get(2).getLeft() instanceof ChunkFetchFailure);
94+
ChunkFetchFailure chunkFetchFailure =
95+
((ChunkFetchFailure) (responseAndPromisePairs.get(2).getLeft()));
96+
Assert.assertEquals("java.lang.IllegalStateException: Chunk was not found",
97+
chunkFetchFailure.errorString.split("\\r?\\n")[0]);
9598

9699
RequestMessage request3 = new ChunkFetchRequest(new StreamChunkId(streamId, 3));
97100
requestHandler.channelRead(context, request3);
101+
Assert.assertEquals(4, responseAndPromisePairs.size());
102+
Assert.assertTrue(responseAndPromisePairs.get(3).getLeft() instanceof ChunkFetchSuccess);
103+
Assert.assertEquals(managedBuffers.get(3),
104+
((ChunkFetchSuccess) (responseAndPromisePairs.get(3).getLeft())).body());
105+
106+
RequestMessage request4 = new ChunkFetchRequest(new StreamChunkId(streamId, 4));
107+
requestHandler.channelRead(context, request4);
98108
verify(channel, times(1)).close();
99-
assert responseAndPromisePairs.size() == 3;
109+
Assert.assertEquals(4, responseAndPromisePairs.size());
100110
}
101111
}

common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java

Lines changed: 28 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import java.util.List;
2222

2323
import io.netty.channel.Channel;
24+
import org.junit.Assert;
2425
import org.junit.Test;
2526

2627
import static org.mockito.Mockito.*;
@@ -38,7 +39,7 @@
3839
public class TransportRequestHandlerSuite {
3940

4041
@Test
41-
public void handleStreamRequest() throws Exception {
42+
public void handleStreamRequest() {
4243
RpcHandler rpcHandler = new NoOpRpcHandler();
4344
OneForOneStreamManager streamManager = (OneForOneStreamManager) (rpcHandler.getStreamManager());
4445
Channel channel = mock(Channel.class);
@@ -56,48 +57,56 @@ public void handleStreamRequest() throws Exception {
5657
List<ManagedBuffer> managedBuffers = new ArrayList<>();
5758
managedBuffers.add(new TestManagedBuffer(10));
5859
managedBuffers.add(new TestManagedBuffer(20));
60+
managedBuffers.add(null);
5961
managedBuffers.add(new TestManagedBuffer(30));
6062
managedBuffers.add(new TestManagedBuffer(40));
6163
long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel);
6264

63-
assert streamManager.numStreamStates() == 1;
65+
Assert.assertEquals(1, streamManager.numStreamStates());
6466

6567
TransportClient reverseClient = mock(TransportClient.class);
6668
TransportRequestHandler requestHandler = new TransportRequestHandler(channel, reverseClient,
6769
rpcHandler, 2L);
6870

6971
RequestMessage request0 = new StreamRequest(String.format("%d_%d", streamId, 0));
7072
requestHandler.handle(request0);
71-
assert responseAndPromisePairs.size() == 1;
72-
assert responseAndPromisePairs.get(0).getLeft() instanceof StreamResponse;
73-
assert ((StreamResponse) (responseAndPromisePairs.get(0).getLeft())).body() ==
74-
managedBuffers.get(0);
73+
Assert.assertEquals(1, responseAndPromisePairs.size());
74+
Assert.assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof StreamResponse);
75+
Assert.assertEquals(managedBuffers.get(0),
76+
((StreamResponse) (responseAndPromisePairs.get(0).getLeft())).body());
7577

7678
RequestMessage request1 = new StreamRequest(String.format("%d_%d", streamId, 1));
7779
requestHandler.handle(request1);
78-
assert responseAndPromisePairs.size() == 2;
79-
assert responseAndPromisePairs.get(1).getLeft() instanceof StreamResponse;
80-
assert ((StreamResponse) (responseAndPromisePairs.get(1).getLeft())).body() ==
81-
managedBuffers.get(1);
80+
Assert.assertEquals(2, responseAndPromisePairs.size());
81+
Assert.assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof StreamResponse);
82+
Assert.assertEquals(managedBuffers.get(1),
83+
((StreamResponse) (responseAndPromisePairs.get(1).getLeft())).body());
8284

8385
// Finish flushing the response for request0.
8486
responseAndPromisePairs.get(0).getRight().finish(true);
8587

86-
RequestMessage request2 = new StreamRequest(String.format("%d_%d", streamId, 2));
88+
StreamRequest request2 = new StreamRequest(String.format("%d_%d", streamId, 2));
8789
requestHandler.handle(request2);
88-
assert responseAndPromisePairs.size() == 3;
89-
assert responseAndPromisePairs.get(2).getLeft() instanceof StreamResponse;
90-
assert ((StreamResponse) (responseAndPromisePairs.get(2).getLeft())).body() ==
91-
managedBuffers.get(2);
90+
Assert.assertEquals(3, responseAndPromisePairs.size());
91+
Assert.assertTrue(responseAndPromisePairs.get(2).getLeft() instanceof StreamFailure);
92+
Assert.assertEquals(String.format("Stream '%s' was not found.", request2.streamId),
93+
((StreamFailure) (responseAndPromisePairs.get(2).getLeft())).error);
9294

93-
// Request3 will trigger the close of channel, because the number of max chunks being
94-
// transferred is 2;
9595
RequestMessage request3 = new StreamRequest(String.format("%d_%d", streamId, 3));
9696
requestHandler.handle(request3);
97+
Assert.assertEquals(4, responseAndPromisePairs.size());
98+
Assert.assertTrue(responseAndPromisePairs.get(3).getLeft() instanceof StreamResponse);
99+
Assert.assertEquals(managedBuffers.get(3),
100+
((StreamResponse) (responseAndPromisePairs.get(3).getLeft())).body());
101+
102+
// Request4 will trigger the close of channel, because the number of max chunks being
103+
// transferred is 2;
104+
RequestMessage request4 = new StreamRequest(String.format("%d_%d", streamId, 4));
105+
requestHandler.handle(request4);
97106
verify(channel, times(1)).close();
98-
assert responseAndPromisePairs.size() == 3;
107+
Assert.assertEquals(4, responseAndPromisePairs.size());
99108

100109
streamManager.connectionTerminated(channel);
101-
assert streamManager.numStreamStates() == 0;
110+
Assert.assertEquals(0, streamManager.numStreamStates());
102111
}
103112
}

common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java

Lines changed: 52 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,8 @@
2121
import java.util.List;
2222

2323
import io.netty.channel.Channel;
24+
import org.junit.After;
25+
import org.junit.Assert;
2426
import org.junit.Test;
2527
import org.mockito.Mockito;
2628

@@ -29,23 +31,69 @@
2931

3032
public class OneForOneStreamManagerSuite {
3133

34+
List<ManagedBuffer> managedBuffersToRelease = new ArrayList<>();
35+
36+
@After
37+
public void tearDown() {
38+
managedBuffersToRelease.forEach(managedBuffer -> managedBuffer.release());
39+
managedBuffersToRelease.clear();
40+
}
41+
42+
private ManagedBuffer getChunk(OneForOneStreamManager manager, long streamId, int chunkIndex) {
43+
ManagedBuffer chunk = manager.getChunk(streamId, chunkIndex);
44+
if (chunk != null) {
45+
managedBuffersToRelease.add(chunk);
46+
}
47+
return chunk;
48+
}
49+
3250
@Test
33-
public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception {
51+
public void testMissingChunk() {
3452
OneForOneStreamManager manager = new OneForOneStreamManager();
3553
List<ManagedBuffer> buffers = new ArrayList<>();
3654
TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10));
3755
TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20));
56+
TestManagedBuffer buffer3 = Mockito.spy(new TestManagedBuffer(20));
57+
3858
buffers.add(buffer1);
59+
// the nulls here are to simulate a file which goes missing before being read,
60+
// just as a defensive measure
61+
buffers.add(null);
3962
buffers.add(buffer2);
63+
buffers.add(null);
64+
buffers.add(buffer3);
4065

4166
Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS);
42-
manager.registerStream("appId", buffers.iterator(), dummyChannel);
43-
assert manager.numStreamStates() == 1;
67+
long streamId = manager.registerStream("appId", buffers.iterator(), dummyChannel);
68+
Assert.assertEquals(1, manager.numStreamStates());
69+
Assert.assertNotNull(getChunk(manager, streamId, 0));
70+
Assert.assertNull(getChunk(manager, streamId, 1));
71+
Assert.assertNotNull(getChunk(manager, streamId, 2));
72+
manager.connectionTerminated(dummyChannel);
73+
74+
// loaded buffers are not released yet as in production a MangedBuffer returned by getChunk()
75+
// would only be released by Netty after it is written to the network
76+
Mockito.verify(buffer1, Mockito.never()).release();
77+
Mockito.verify(buffer2, Mockito.never()).release();
78+
Mockito.verify(buffer3, Mockito.times(1)).release();
79+
}
4480

81+
@Test
82+
public void managedBuffersAreFreedWhenConnectionIsClosed() {
83+
OneForOneStreamManager manager = new OneForOneStreamManager();
84+
List<ManagedBuffer> buffers = new ArrayList<>();
85+
TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10));
86+
TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20));
87+
buffers.add(buffer1);
88+
buffers.add(buffer2);
89+
90+
Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS);
91+
manager.registerStream("appId", buffers.iterator(), dummyChannel);
92+
Assert.assertEquals(1, manager.numStreamStates());
4593
manager.connectionTerminated(dummyChannel);
4694

4795
Mockito.verify(buffer1, Mockito.times(1)).release();
4896
Mockito.verify(buffer2, Mockito.times(1)).release();
49-
assert manager.numStreamStates() == 0;
97+
Assert.assertEquals(0, manager.numStreamStates());
5098
}
5199
}
Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,24 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.network.shuffle;
19+
20+
public class Constants {
21+
22+
public static final String SHUFFLE_SERVICE_FETCH_RDD_ENABLED =
23+
"spark.shuffle.service.fetch.rdd.enabled";
24+
}

0 commit comments

Comments
 (0)