Skip to content

Commit

Permalink
[hotfix] HsMemoryDataSpiller's spilling thread will trigger fatal err…
Browse files Browse the repository at this point in the history
…or when an exception is encountered.
  • Loading branch information
reswqa authored and xintongsong committed Jul 26, 2022
1 parent 50dc511 commit 0610e16
Show file tree
Hide file tree
Showing 2 changed files with 14 additions and 24 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil;
import org.apache.flink.runtime.io.network.partition.hybrid.HsFileDataIndex.SpilledBuffer;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FatalExitExceptionHandler;

import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.ThreadFactoryBuilder;

Expand All @@ -41,7 +43,15 @@ public class HsMemoryDataSpiller implements AutoCloseable {
/** One thread to perform spill operation. */
private final ExecutorService ioExecutor =
Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder().setNameFormat("hybrid spiller thread").build());
new ThreadFactoryBuilder()
.setNameFormat("hybrid spiller thread")
// It is more appropriate to use task fail over than exit JVM here,
// but the task thread will bring some extra overhead to check the
// exception information set by other thread. As the spiller thread will
// not encounter exceptions in most cases, we temporarily choose the
// form of fatal error to deal except thrown by spiller thread.
.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE)
.build());

/** File channel to write data. */
private final FileChannel dataFileChannel;
Expand Down Expand Up @@ -81,8 +91,9 @@ private void spill(
// which controls data's life cycle.
// TODO update file data index and handle buffers release in future ticket.
spilledFuture.complete(spilledBuffers);
} catch (Throwable t) {
spilledFuture.completeExceptionally(t);
} catch (IOException exception) {
// if spilling is failed, throw exception directly to uncaughtExceptionHandler.
ExceptionUtils.rethrow(exception);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,10 +33,8 @@
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.api.io.TempDir;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.Path;
Expand All @@ -46,7 +44,6 @@
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;

import static org.assertj.core.api.Assertions.assertThat;
Expand Down Expand Up @@ -74,24 +71,6 @@ void before(@TempDir Path tempDir) throws Exception {
this.memoryDataSpiller = new HsMemoryDataSpiller(dataFileChannel);
}

@Test
void testSpillExceptionally() throws IOException {
int targetChannel = 0;
List<BufferWithIdentity> bufferWithIdentityList =
createBufferWithIdentityList(
targetChannel,
Arrays.asList(Tuple2.of(0, 0), Tuple2.of(1, 1), Tuple2.of(2, 2)));
// close data file channel to trigger exception when spill data into disk.
dataFileChannel.close();

CompletableFuture<List<SpilledBuffer>> future =
memoryDataSpiller.spillAsync(bufferWithIdentityList);
assertThat(future)
.failsWithin(60, TimeUnit.SECONDS)
.withThrowableOfType(ExecutionException.class)
.withCauseInstanceOf(ClosedChannelException.class);
}

@Test
void testSpillSuccessfully() throws Exception {
List<BufferWithIdentity> bufferWithIdentityList = new ArrayList<>();
Expand Down

0 comments on commit 0610e16

Please sign in to comment.