aboutsummaryrefslogtreecommitdiff
path: root/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
blob: 9248ef3c467dfd38704a23ce797fd637b3e962d7 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
/*
 * 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.network.shuffle;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;

import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;

import static org.junit.Assert.*;

import org.apache.spark.network.TestUtils;
import org.apache.spark.network.TransportContext;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NioManagedBuffer;
import org.apache.spark.network.server.TransportServer;
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
import org.apache.spark.network.util.MapConfigProvider;
import org.apache.spark.network.util.TransportConf;

public class ExternalShuffleIntegrationSuite {

  private static final String APP_ID = "app-id";
  private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager";

  // Executor 0 is sort-based
  static TestShuffleDataContext dataContext0;

  static ExternalShuffleBlockHandler handler;
  static TransportServer server;
  static TransportConf conf;

  static byte[][] exec0Blocks = new byte[][] {
    new byte[123],
    new byte[12345],
    new byte[1234567],
  };

  static byte[][] exec1Blocks = new byte[][] {
    new byte[321],
    new byte[54321],
  };

  @BeforeClass
  public static void beforeAll() throws IOException {
    Random rand = new Random();

    for (byte[] block : exec0Blocks) {
      rand.nextBytes(block);
    }
    for (byte[] block: exec1Blocks) {
      rand.nextBytes(block);
    }

    dataContext0 = new TestShuffleDataContext(2, 5);
    dataContext0.create();
    dataContext0.insertSortShuffleData(0, 0, exec0Blocks);

    conf = new TransportConf("shuffle", MapConfigProvider.EMPTY);
    handler = new ExternalShuffleBlockHandler(conf, null);
    TransportContext transportContext = new TransportContext(conf, handler);
    server = transportContext.createServer();
  }

  @AfterClass
  public static void afterAll() {
    dataContext0.cleanup();
    server.close();
  }

  @After
  public void afterEach() {
    handler.applicationRemoved(APP_ID, false /* cleanupLocalDirs */);
  }

  static class FetchResult {
    public Set<String> successBlocks;
    public Set<String> failedBlocks;
    public List<ManagedBuffer> buffers;

    public void releaseBuffers() {
      for (ManagedBuffer buffer : buffers) {
        buffer.release();
      }
    }
  }

  // Fetch a set of blocks from a pre-registered executor.
  private FetchResult fetchBlocks(String execId, String[] blockIds) throws Exception {
    return fetchBlocks(execId, blockIds, conf, server.getPort());
  }

  // Fetch a set of blocks from a pre-registered executor. Connects to the server on the given port,
  // to allow connecting to invalid servers.
  private FetchResult fetchBlocks(
      String execId,
      String[] blockIds,
      TransportConf clientConf,
      int port) throws Exception {
    final FetchResult res = new FetchResult();
    res.successBlocks = Collections.synchronizedSet(new HashSet<String>());
    res.failedBlocks = Collections.synchronizedSet(new HashSet<String>());
    res.buffers = Collections.synchronizedList(new LinkedList<ManagedBuffer>());

    final Semaphore requestsRemaining = new Semaphore(0);

    ExternalShuffleClient client = new ExternalShuffleClient(clientConf, null, false);
    client.init(APP_ID);
    client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds,
      new BlockFetchingListener() {
        @Override
        public void onBlockFetchSuccess(String blockId, ManagedBuffer data) {
          synchronized (this) {
            if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) {
              data.retain();
              res.successBlocks.add(blockId);
              res.buffers.add(data);
              requestsRemaining.release();
            }
          }
        }

        @Override
        public void onBlockFetchFailure(String blockId, Throwable exception) {
          synchronized (this) {
            if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) {
              res.failedBlocks.add(blockId);
              requestsRemaining.release();
            }
          }
        }
      });

    if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) {
      fail("Timeout getting response from the server");
    }
    client.close();
    return res;
  }

  @Test
  public void testFetchOneSort() throws Exception {
    registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
    FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" });
    assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks);
    assertTrue(exec0Fetch.failedBlocks.isEmpty());
    assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks[0]));
    exec0Fetch.releaseBuffers();
  }

  @Test
  public void testFetchThreeSort() throws Exception {
    registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
    FetchResult exec0Fetch = fetchBlocks("exec-0",
      new String[] { "shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2" });
    assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"),
      exec0Fetch.successBlocks);
    assertTrue(exec0Fetch.failedBlocks.isEmpty());
    assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks));
    exec0Fetch.releaseBuffers();
  }

  @Test (expected = RuntimeException.class)
  public void testRegisterInvalidExecutor() throws Exception {
    registerExecutor("exec-1", dataContext0.createExecutorInfo("unknown sort manager"));
  }

  @Test
  public void testFetchWrongBlockId() throws Exception {
    registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER));
    FetchResult execFetch = fetchBlocks("exec-1", new String[] { "rdd_1_0_0" });
    assertTrue(execFetch.successBlocks.isEmpty());
    assertEquals(Sets.newHashSet("rdd_1_0_0"), execFetch.failedBlocks);
  }

  @Test
  public void testFetchNonexistent() throws Exception {
    registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
    FetchResult execFetch = fetchBlocks("exec-0",
      new String[] { "shuffle_2_0_0" });
    assertTrue(execFetch.successBlocks.isEmpty());
    assertEquals(Sets.newHashSet("shuffle_2_0_0"), execFetch.failedBlocks);
  }

  @Test
  public void testFetchWrongExecutor() throws Exception {
    registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
    FetchResult execFetch = fetchBlocks("exec-0",
      new String[] { "shuffle_0_0_0" /* right */, "shuffle_1_0_0" /* wrong */ });
    // Both still fail, as we start by checking for all block.
    assertTrue(execFetch.successBlocks.isEmpty());
    assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks);
  }

  @Test
  public void testFetchUnregisteredExecutor() throws Exception {
    registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
    FetchResult execFetch = fetchBlocks("exec-2",
      new String[] { "shuffle_0_0_0", "shuffle_1_0_0" });
    assertTrue(execFetch.successBlocks.isEmpty());
    assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks);
  }

  @Test
  public void testFetchNoServer() throws Exception {
    TransportConf clientConf = new TransportConf("shuffle",
      new MapConfigProvider(ImmutableMap.of("spark.shuffle.io.maxRetries", "0")));
    registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
    FetchResult execFetch = fetchBlocks("exec-0",
      new String[]{"shuffle_1_0_0", "shuffle_1_0_1"}, clientConf, 1 /* port */);
    assertTrue(execFetch.successBlocks.isEmpty());
    assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks);
  }

  private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo)
      throws IOException {
    ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false);
    client.init(APP_ID);
    client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(),
      executorId, executorInfo);
  }

  private void assertBufferListsEqual(List<ManagedBuffer> list0, List<byte[]> list1)
    throws Exception {
    assertEquals(list0.size(), list1.size());
    for (int i = 0; i < list0.size(); i ++) {
      assertBuffersEqual(list0.get(i), new NioManagedBuffer(ByteBuffer.wrap(list1.get(i))));
    }
  }

  private void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) throws Exception {
    ByteBuffer nio0 = buffer0.nioByteBuffer();
    ByteBuffer nio1 = buffer1.nioByteBuffer();

    int len = nio0.remaining();
    assertEquals(nio0.remaining(), nio1.remaining());
    for (int i = 0; i < len; i ++) {
      assertEquals(nio0.get(), nio1.get());
    }
  }
}