-
Notifications
You must be signed in to change notification settings - Fork 64
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[NEMO-54] Handle remote data fetch failures due to executor removal (#67
) JIRA: [NEMO-54: Handle remote data fetch failures due to executor removal](https://issues.apache.org/jira/projects/NEMO/issues/NEMO-54) **Major changes:** - Catch all Throwables in ParentTaskDataFetcher, wrap it with an IOException, and throw the IOException over to TaskExecutor - Handle CompleteableFuture failures by passing the throwable to TaskExecutor via iteratorQueue **Minor changes to note:** - Introduce ClosableBlockingQueue#closeExceptionally, with the hope to propagate errors all the way up to ParentTaskDataFetcher **Tests for the changes:** - ParentTaskDataFetcherTest **Other comments:** - I'll add fault-injected integration tests after https://issues.apache.org/jira/browse/NEMO-55 resolves [NEMO-54](https://issues.apache.org/jira/projects/NEMO/issues/NEMO-54)
- Loading branch information
Showing
5 changed files
with
201 additions
and
25 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
136 changes: 136 additions & 0 deletions
136
.../executor/src/test/java/edu/snu/nemo/runtime/executor/task/ParentTaskDataFetcherTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,136 @@ | ||
/* | ||
* Copyright (C) 2018 Seoul National University | ||
* | ||
* Licensed 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 edu.snu.nemo.runtime.executor.task; | ||
|
||
import edu.snu.nemo.common.ir.vertex.IRVertex; | ||
import edu.snu.nemo.runtime.executor.data.DataUtil; | ||
import edu.snu.nemo.runtime.executor.datatransfer.InputReader; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.powermock.core.classloader.annotations.PrepareForTest; | ||
import org.powermock.modules.junit4.PowerMockRunner; | ||
|
||
import java.io.IOException; | ||
import java.util.*; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.Executors; | ||
|
||
import static org.junit.Assert.assertEquals; | ||
import static org.junit.Assert.assertTrue; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.when; | ||
|
||
/** | ||
* Tests {@link ParentTaskDataFetcher}. | ||
*/ | ||
@RunWith(PowerMockRunner.class) | ||
@PrepareForTest({InputReader.class, VertexHarness.class}) | ||
public final class ParentTaskDataFetcherTest { | ||
|
||
@Test(timeout=5000) | ||
public void testEmpty() throws Exception { | ||
// InputReader | ||
final List<String> dataElements = new ArrayList<>(0); // empty data | ||
final InputReader inputReader = generateInputReader(generateCompletableFuture(dataElements.iterator())); | ||
|
||
// Fetcher | ||
final ParentTaskDataFetcher fetcher = createFetcher(inputReader); | ||
|
||
// Should return Void.TYPE | ||
assertEquals(Void.TYPE, fetcher.fetchDataElement()); | ||
} | ||
|
||
@Test(timeout=5000) | ||
public void testNonEmpty() throws Exception { | ||
// InputReader | ||
final String singleData = "Single"; | ||
final List<String> dataElements = new ArrayList<>(1); | ||
dataElements.add(singleData); // Single element | ||
final InputReader inputReader = generateInputReader(generateCompletableFuture(dataElements.iterator())); | ||
|
||
// Fetcher | ||
final ParentTaskDataFetcher fetcher = createFetcher(inputReader); | ||
|
||
// Should return only a single element | ||
assertEquals(singleData, fetcher.fetchDataElement()); | ||
assertEquals(null, fetcher.fetchDataElement()); | ||
} | ||
|
||
@Test(timeout=5000, expected = IOException.class) | ||
public void testErrorWhenRPC() throws Exception { | ||
// Failing future | ||
final CompletableFuture failingFuture = CompletableFuture.runAsync(() -> { | ||
try { | ||
Thread.sleep(2 * 1000); // Block the fetcher for 2 seconds | ||
throw new RuntimeException(); // Fail this future | ||
} catch (InterruptedException e) { | ||
// This shouldn't happen. | ||
// We don't throw anything here, so that IOException does not occur and the test fails | ||
} | ||
}, Executors.newSingleThreadExecutor()); | ||
final InputReader inputReader = generateInputReader(failingFuture); | ||
|
||
// Fetcher | ||
final ParentTaskDataFetcher fetcher = createFetcher(inputReader); | ||
|
||
// Should throw an IOException | ||
fetcher.fetchDataElement(); // checked by 'expected = IOException.class' | ||
assertTrue(failingFuture.isCompletedExceptionally()); | ||
} | ||
|
||
@Test(timeout=5000, expected = IOException.class) | ||
public void testErrorWhenReadingData() throws Exception { | ||
// Failed iterator | ||
final InputReader inputReader = generateInputReader(generateCompletableFuture(new FailedIterator())); | ||
|
||
// Fetcher | ||
final ParentTaskDataFetcher fetcher = createFetcher(inputReader); | ||
|
||
// Should throw an IOException | ||
fetcher.fetchDataElement(); // checked by 'expected = IOException.class' | ||
} | ||
|
||
private ParentTaskDataFetcher createFetcher(final InputReader readerForParentTask) { | ||
return new ParentTaskDataFetcher( | ||
mock(IRVertex.class), | ||
readerForParentTask, // This is the only argument that affects the behavior of ParentTaskDataFetcher | ||
mock(VertexHarness.class), | ||
new HashMap<>(0), | ||
false); | ||
} | ||
|
||
private InputReader generateInputReader(final CompletableFuture completableFuture) { | ||
final InputReader inputReader = mock(InputReader.class); | ||
when(inputReader.read()).thenReturn(Arrays.asList(completableFuture)); | ||
return inputReader; | ||
} | ||
|
||
private CompletableFuture generateCompletableFuture(final Iterator iterator) { | ||
return CompletableFuture.completedFuture(DataUtil.IteratorWithNumBytes.of(iterator)); | ||
} | ||
|
||
private class FailedIterator implements Iterator { | ||
@Override | ||
public boolean hasNext() { | ||
throw new RuntimeException("Fail"); | ||
} | ||
|
||
@Override | ||
public Object next() { | ||
throw new RuntimeException("Fail"); | ||
} | ||
} | ||
} |