aboutsummaryrefslogtreecommitdiff
path: root/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
blob: a2509f5f34c880e9d79a949914dcac60b4a7578c (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
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
/*
 * 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.Arrays;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;

import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.mockito.stubbing.Stubber;

import static org.junit.Assert.*;
import static org.mockito.Mockito.*;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NioManagedBuffer;
import org.apache.spark.network.util.MapConfigProvider;
import org.apache.spark.network.util.TransportConf;
import static org.apache.spark.network.shuffle.RetryingBlockFetcher.BlockFetchStarter;

/**
 * Tests retry logic by throwing IOExceptions and ensuring that subsequent attempts are made to
 * fetch the lost blocks.
 */
public class RetryingBlockFetcherSuite {

  ManagedBuffer block0 = new NioManagedBuffer(ByteBuffer.wrap(new byte[13]));
  ManagedBuffer block1 = new NioManagedBuffer(ByteBuffer.wrap(new byte[7]));
  ManagedBuffer block2 = new NioManagedBuffer(ByteBuffer.wrap(new byte[19]));

  @Test
  public void testNoFailures() throws IOException {
    BlockFetchingListener listener = mock(BlockFetchingListener.class);

    List<? extends Map<String, Object>> interactions = Arrays.asList(
      // Immediately return both blocks successfully.
      ImmutableMap.<String, Object>builder()
        .put("b0", block0)
        .put("b1", block1)
        .build()
      );

    performInteractions(interactions, listener);

    verify(listener).onBlockFetchSuccess("b0", block0);
    verify(listener).onBlockFetchSuccess("b1", block1);
    verifyNoMoreInteractions(listener);
  }

  @Test
  public void testUnrecoverableFailure() throws IOException {
    BlockFetchingListener listener = mock(BlockFetchingListener.class);

    List<? extends Map<String, Object>> interactions = Arrays.asList(
      // b0 throws a non-IOException error, so it will be failed without retry.
      ImmutableMap.<String, Object>builder()
        .put("b0", new RuntimeException("Ouch!"))
        .put("b1", block1)
        .build()
    );

    performInteractions(interactions, listener);

    verify(listener).onBlockFetchFailure(eq("b0"), (Throwable) any());
    verify(listener).onBlockFetchSuccess("b1", block1);
    verifyNoMoreInteractions(listener);
  }

  @Test
  public void testSingleIOExceptionOnFirst() throws IOException {
    BlockFetchingListener listener = mock(BlockFetchingListener.class);

    List<? extends Map<String, Object>> interactions = Arrays.asList(
      // IOException will cause a retry. Since b0 fails, we will retry both.
      ImmutableMap.<String, Object>builder()
        .put("b0", new IOException("Connection failed or something"))
        .put("b1", block1)
        .build(),
      ImmutableMap.<String, Object>builder()
        .put("b0", block0)
        .put("b1", block1)
        .build()
    );

    performInteractions(interactions, listener);

    verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0);
    verify(listener, timeout(5000)).onBlockFetchSuccess("b1", block1);
    verifyNoMoreInteractions(listener);
  }

  @Test
  public void testSingleIOExceptionOnSecond() throws IOException {
    BlockFetchingListener listener = mock(BlockFetchingListener.class);

    List<? extends Map<String, Object>> interactions = Arrays.asList(
      // IOException will cause a retry. Since b1 fails, we will not retry b0.
      ImmutableMap.<String, Object>builder()
        .put("b0", block0)
        .put("b1", new IOException("Connection failed or something"))
        .build(),
      ImmutableMap.<String, Object>builder()
        .put("b1", block1)
        .build()
    );

    performInteractions(interactions, listener);

    verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0);
    verify(listener, timeout(5000)).onBlockFetchSuccess("b1", block1);
    verifyNoMoreInteractions(listener);
  }

  @Test
  public void testTwoIOExceptions() throws IOException {
    BlockFetchingListener listener = mock(BlockFetchingListener.class);

    List<? extends Map<String, Object>> interactions = Arrays.asList(
      // b0's IOException will trigger retry, b1's will be ignored.
      ImmutableMap.<String, Object>builder()
        .put("b0", new IOException())
        .put("b1", new IOException())
        .build(),
      // Next, b0 is successful and b1 errors again, so we just request that one.
      ImmutableMap.<String, Object>builder()
        .put("b0", block0)
        .put("b1", new IOException())
        .build(),
      // b1 returns successfully within 2 retries.
      ImmutableMap.<String, Object>builder()
        .put("b1", block1)
        .build()
    );

    performInteractions(interactions, listener);

    verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0);
    verify(listener, timeout(5000)).onBlockFetchSuccess("b1", block1);
    verifyNoMoreInteractions(listener);
  }

  @Test
  public void testThreeIOExceptions() throws IOException {
    BlockFetchingListener listener = mock(BlockFetchingListener.class);

    List<? extends Map<String, Object>> interactions = Arrays.asList(
      // b0's IOException will trigger retry, b1's will be ignored.
      ImmutableMap.<String, Object>builder()
        .put("b0", new IOException())
        .put("b1", new IOException())
        .build(),
      // Next, b0 is successful and b1 errors again, so we just request that one.
      ImmutableMap.<String, Object>builder()
        .put("b0", block0)
        .put("b1", new IOException())
        .build(),
      // b1 errors again, but this was the last retry
      ImmutableMap.<String, Object>builder()
        .put("b1", new IOException())
        .build(),
      // This is not reached -- b1 has failed.
      ImmutableMap.<String, Object>builder()
        .put("b1", block1)
        .build()
    );

    performInteractions(interactions, listener);

    verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0);
    verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), (Throwable) any());
    verifyNoMoreInteractions(listener);
  }

  @Test
  public void testRetryAndUnrecoverable() throws IOException {
    BlockFetchingListener listener = mock(BlockFetchingListener.class);

    List<? extends Map<String, Object>> interactions = Arrays.asList(
      // b0's IOException will trigger retry, subsequent messages will be ignored.
      ImmutableMap.<String, Object>builder()
        .put("b0", new IOException())
        .put("b1", new RuntimeException())
        .put("b2", block2)
        .build(),
      // Next, b0 is successful, b1 errors unrecoverably, and b2 triggers a retry.
      ImmutableMap.<String, Object>builder()
        .put("b0", block0)
        .put("b1", new RuntimeException())
        .put("b2", new IOException())
        .build(),
      // b2 succeeds in its last retry.
      ImmutableMap.<String, Object>builder()
        .put("b2", block2)
        .build()
    );

    performInteractions(interactions, listener);

    verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0);
    verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), (Throwable) any());
    verify(listener, timeout(5000)).onBlockFetchSuccess("b2", block2);
    verifyNoMoreInteractions(listener);
  }

  /**
   * Performs a set of interactions in response to block requests from a RetryingBlockFetcher.
   * Each interaction is a Map from BlockId to either ManagedBuffer or Exception. This interaction
   * means "respond to the next block fetch request with these Successful buffers and these Failure
   * exceptions". We verify that the expected block ids are exactly the ones requested.
   *
   * If multiple interactions are supplied, they will be used in order. This is useful for encoding
   * retries -- the first interaction may include an IOException, which causes a retry of some
   * subset of the original blocks in a second interaction.
   */
  @SuppressWarnings("unchecked")
  private static void performInteractions(List<? extends Map<String, Object>> interactions,
                                          BlockFetchingListener listener)
    throws IOException {

    MapConfigProvider provider = new MapConfigProvider(ImmutableMap.of(
      "spark.shuffle.io.maxRetries", "2",
      "spark.shuffle.io.retryWait", "0"));
    TransportConf conf = new TransportConf("shuffle", provider);
    BlockFetchStarter fetchStarter = mock(BlockFetchStarter.class);

    Stubber stub = null;

    // Contains all blockIds that are referenced across all interactions.
    final LinkedHashSet<String> blockIds = Sets.newLinkedHashSet();

    for (final Map<String, Object> interaction : interactions) {
      blockIds.addAll(interaction.keySet());

      Answer<Void> answer = new Answer<Void>() {
        @Override
        public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
          try {
            // Verify that the RetryingBlockFetcher requested the expected blocks.
            String[] requestedBlockIds = (String[]) invocationOnMock.getArguments()[0];
            String[] desiredBlockIds = interaction.keySet().toArray(new String[interaction.size()]);
            assertArrayEquals(desiredBlockIds, requestedBlockIds);

            // Now actually invoke the success/failure callbacks on each block.
            BlockFetchingListener retryListener =
              (BlockFetchingListener) invocationOnMock.getArguments()[1];
            for (Map.Entry<String, Object> block : interaction.entrySet()) {
              String blockId = block.getKey();
              Object blockValue = block.getValue();

              if (blockValue instanceof ManagedBuffer) {
                retryListener.onBlockFetchSuccess(blockId, (ManagedBuffer) blockValue);
              } else if (blockValue instanceof Exception) {
                retryListener.onBlockFetchFailure(blockId, (Exception) blockValue);
              } else {
                fail("Can only handle ManagedBuffers and Exceptions, got " + blockValue);
              }
            }
            return null;
          } catch (Throwable e) {
            e.printStackTrace();
            throw e;
          }
        }
      };

      // This is either the first stub, or should be chained behind the prior ones.
      if (stub == null) {
        stub = doAnswer(answer);
      } else {
        stub.doAnswer(answer);
      }
    }

    assertNotNull(stub);
    stub.when(fetchStarter).createAndStart((String[]) any(), (BlockFetchingListener) anyObject());
    String[] blockIdArray = blockIds.toArray(new String[blockIds.size()]);
    new RetryingBlockFetcher(conf, fetchStarter, blockIdArray, listener).start();
  }
}