From a612d99ccaa8661731106e2248d37e7e1153d101 Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Mon, 24 Jan 2022 20:03:22 +0800 Subject: [PATCH 01/49] [FLINK-25774][network] Restrict the maximum number of buffers can be used per result partition for sort-shuffle Currently, for sort-shuffle, the maximum number of buffers can be used per result partition is Integer.MAX_VALUE. However, if too many buffers are taken by one result partition, other result partitions and input gates may spend too much time waiting for buffers which can influence performance. This patch restricts the maximum number of buffers can be used per result partition to Math.max(min, 4 * numSubpartitions). Note that the selected value is an empirical one based on the TPC-DS benchmark results. This closes #18470. --- .../flink/runtime/shuffle/NettyShuffleUtils.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java index a25a68ce7846a..663212ed62163 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java @@ -64,15 +64,15 @@ public static Pair getMinMaxNetworkBuffersPerResultPartition( final int sortShuffleMinBuffers, final int numSubpartitions, final ResultPartitionType type) { - int min = - type.isBlocking() && numSubpartitions >= sortShuffleMinParallelism - ? sortShuffleMinBuffers - : numSubpartitions + 1; + boolean isSortShuffle = type.isBlocking() && numSubpartitions >= sortShuffleMinParallelism; + int min = isSortShuffle ? sortShuffleMinBuffers : numSubpartitions + 1; int max = type.isBounded() ? numSubpartitions * configuredNetworkBuffersPerChannel + numFloatingBuffersPerGate - : NetworkBufferPool.UNBOUNDED_POOL_SIZE; + : (isSortShuffle + ? Math.max(min, 4 * numSubpartitions) + : NetworkBufferPool.UNBOUNDED_POOL_SIZE); // for each upstream hash-based blocking/pipelined subpartition, at least one buffer is // needed even the configured network buffers per channel is 0 and this behavior is for // performance. If it's not guaranteed that each subpartition can get at least one buffer, From dfa672f48d365ec938e21d0e15816e8d8de6eca4 Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Mon, 24 Jan 2022 20:23:13 +0800 Subject: [PATCH 02/49] [FLINK-25780][network] Reduce the maximum size of data output buffer per result partition for sort-shuffle The data output buffer of sort-shuffle is for better disk IO performance and currently, the total data output buffer size is 16M which is pretty big. However, blocking request too many buffers may influence performance. This patch reduces the maximum size of data output buffer from 16M to 8M to reduce the buffer request time. Note that the selected value is an empirical one based on the TPC-DS benchmark results. This closes #18471. --- .../io/network/partition/SortMergeResultPartition.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java index 36fde496789f4..9456762349be8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java @@ -68,9 +68,9 @@ public class SortMergeResultPartition extends ResultPartition { /** * Number of expected buffer size to allocate for data writing. Currently, it is an empirical - * value (16M) which can not be configured. + * value (8M) which can not be configured. */ - private static final int NUM_WRITE_BUFFER_BYTES = 16 * 1024 * 1024; + private static final int NUM_WRITE_BUFFER_BYTES = 8 * 1024 * 1024; private final Object lock = new Object(); From 34d0972655deb33b68768f641b7ca97dec30a768 Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Mon, 24 Jan 2022 20:49:06 +0800 Subject: [PATCH 03/49] [FLINK-25781][network] Adjust the maximum number of buffers can be used per result partition for data read of sort-shuffle Currently, for sort-shuffle, the maximum number of buffers can be used per result partition for shuffle data read is 32M. However, for large parallelism jobs, 32M is not enough and for small parallelism jobs, 32M may waste buffers. This patch tries to solve the problem by adjusting the maximum number of buffers can be used per result partition for shuffle data read from 32M to Math.max(16M, numSubpartitions). This closes #18473. --- .../partition/SortMergeResultPartition.java | 3 ++- .../SortMergeResultPartitionReadScheduler.java | 16 +++++++++++----- ...ortMergeResultPartitionReadSchedulerTest.java | 8 +++++--- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java index 9456762349be8..728c08d0d94fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java @@ -141,7 +141,8 @@ public SortMergeResultPartition( // input balance of the downstream tasks this.subpartitionOrder = getRandomSubpartitionOrder(numSubpartitions); this.readScheduler = - new SortMergeResultPartitionReadScheduler(readBufferPool, readIOExecutor, lock); + new SortMergeResultPartitionReadScheduler( + numSubpartitions, readBufferPool, readIOExecutor, lock); PartitionedFileWriter fileWriter = null; try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java index 539a43bf7cef7..8dd33af8a3c20 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java @@ -125,11 +125,15 @@ class SortMergeResultPartitionReadScheduler implements Runnable, BufferRecycler private volatile boolean isReleased; SortMergeResultPartitionReadScheduler( - BatchShuffleReadBufferPool bufferPool, Executor ioExecutor, Object lock) { - this(bufferPool, ioExecutor, lock, DEFAULT_BUFFER_REQUEST_TIMEOUT); + int numSubpartitions, + BatchShuffleReadBufferPool bufferPool, + Executor ioExecutor, + Object lock) { + this(numSubpartitions, bufferPool, ioExecutor, lock, DEFAULT_BUFFER_REQUEST_TIMEOUT); } SortMergeResultPartitionReadScheduler( + int numSubpartitions, BatchShuffleReadBufferPool bufferPool, Executor ioExecutor, Object lock, @@ -138,9 +142,11 @@ class SortMergeResultPartitionReadScheduler implements Runnable, BufferRecycler this.lock = checkNotNull(lock); this.bufferPool = checkNotNull(bufferPool); this.ioExecutor = checkNotNull(ioExecutor); - // one partition reader can consume at most 32M (the expected buffers per request is 8M) - // buffers for data read. Currently, it is only an empirical value can not be configured - this.maxRequestedBuffers = Math.max(1, 4 * bufferPool.getNumBuffersPerRequest()); + // one partition reader can consume at most Math.max(16M, numSubpartitions) (the expected + // buffers per request is 8M) buffers for data read, which means larger parallelism, more + // buffers. Currently, it is only an empirical strategy which can not be configured. + this.maxRequestedBuffers = + Math.max(2 * bufferPool.getNumBuffersPerRequest(), numSubpartitions); this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout); // initialize the buffer pool eagerly to avoid reporting errors like OOM too late diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java index 46506307c92c0..090a4323ad0d8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java @@ -92,7 +92,9 @@ public void before() throws Exception { dataBytes); bufferPool = new BatchShuffleReadBufferPool(totalBytes, bufferSize); executor = Executors.newFixedThreadPool(numThreads); - readScheduler = new SortMergeResultPartitionReadScheduler(bufferPool, executor, this); + readScheduler = + new SortMergeResultPartitionReadScheduler( + numSubpartitions, bufferPool, executor, this); } @After @@ -209,7 +211,7 @@ public void testRequestBufferTimeoutAndFailed() throws Exception { List buffers = bufferPool.requestBuffers(); SortMergeResultPartitionReadScheduler readScheduler = new SortMergeResultPartitionReadScheduler( - bufferPool, executor, this, bufferRequestTimeout); + numSubpartitions, bufferPool, executor, this, bufferRequestTimeout); SortMergeSubpartitionReader subpartitionReader = readScheduler.createSubpartitionReader( @@ -237,7 +239,7 @@ public void testRequestTimeoutIsRefreshedAndSuccess() throws Exception { new FakeBatchShuffleReadBufferPool(bufferSize * 3, bufferSize); SortMergeResultPartitionReadScheduler readScheduler = new SortMergeResultPartitionReadScheduler( - bufferPool, executor, this, bufferRequestTimeout); + numSubpartitions, bufferPool, executor, this, bufferRequestTimeout); FileChannel dataFileChannel = openFileChannel(partitionedFile.getDataFilePath()); FileChannel indexFileChannel = openFileChannel(partitionedFile.getIndexFilePath()); From 8bd62ec0819d73b7b42d74835b3077f7f25ab48d Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Mon, 24 Jan 2022 21:16:07 +0800 Subject: [PATCH 04/49] [FLINK-25786][network] Adjust the generation of subpartition data storage order for sort-shuffle from random shuffle to random shift Currently, for sort-shuffle, the generation of subpartition data storage order is random shuffle. However, if there is no enough resources to run the downstream consumer tasks in parallel, the performance can be influenced because of the random disk IO caused by the random subpartition data storage order. This patch tries improve this scenario by adjusting the generation of subpartition data storage order for sort-shuffle from random shuffle to random shift. This closes #18474. --- .../partition/SortMergeResultPartition.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java index 728c08d0d94fd..b94191b209b08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java @@ -43,13 +43,11 @@ import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Queue; +import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType; import static org.apache.flink.util.Preconditions.checkElementIndex; @@ -493,10 +491,13 @@ public int getNumberOfQueuedBuffers(int targetSubpartition) { } private int[] getRandomSubpartitionOrder(int numSubpartitions) { - List list = - IntStream.range(0, numSubpartitions).boxed().collect(Collectors.toList()); - Collections.shuffle(list); - return list.stream().mapToInt(Integer::intValue).toArray(); + int[] order = new int[numSubpartitions]; + Random random = new Random(); + int shift = random.nextInt(numSubpartitions); + for (int channel = 0; channel < numSubpartitions; ++channel) { + order[(channel + shift) % numSubpartitions] = channel; + } + return order; } @VisibleForTesting From f4fd4f8bb66867821cb0f171218d85f8386b0897 Mon Sep 17 00:00:00 2001 From: Lijie Wang Date: Wed, 26 Jan 2022 13:12:32 +0800 Subject: [PATCH 05/49] [FLINK-25035][runtime] Move consumedSubpartitionIndex from SingleInputGate to InputChannel --- .../partition/consumer/InputChannel.java | 23 +++-- .../partition/consumer/LocalInputChannel.java | 12 +-- .../consumer/LocalRecoveredInputChannel.java | 3 + .../consumer/RecoveredInputChannel.java | 4 +- .../consumer/RemoteInputChannel.java | 14 ++-- .../consumer/RemoteRecoveredInputChannel.java | 3 + .../partition/consumer/SingleInputGate.java | 22 ++--- .../consumer/SingleInputGateFactory.java | 18 +++- .../consumer/UnknownInputChannel.java | 15 +++- ...asedPartitionRequestClientHandlerTest.java | 11 +-- ...NettyMessageClientDecoderDelegateTest.java | 2 +- ...ttyMessageClientSideSerializationTest.java | 2 +- .../NettyPartitionRequestClientTest.java | 8 +- .../partition/InputGateFairnessTest.java | 4 +- .../consumer/InputBuffersMetricsTest.java | 2 +- .../consumer/InputChannelBuilder.java | 11 +++ .../partition/consumer/InputChannelTest.java | 3 +- .../consumer/LocalInputChannelTest.java | 44 +++++----- .../consumer/RecoveredInputChannelTest.java | 3 +- .../consumer/RemoteInputChannelTest.java | 83 ++++++++++--------- .../consumer/SingleInputGateBuilder.java | 8 -- .../partition/consumer/TestInputChannel.java | 3 +- .../SingleInputGateBenchmarkFactory.java | 16 ++-- .../CheckpointedInputGateTest.java | 6 +- 24 files changed, 184 insertions(+), 136 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java index 7b70d35886377..5991ca81f7e97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java @@ -43,7 +43,7 @@ *

For each channel, the consumption life cycle is as follows: * *

    - *
  1. {@link #requestSubpartition(int)} + *
  2. {@link #requestSubpartition()} *
  3. {@link #getNextBuffer()} *
  4. {@link #releaseAllResources()} *
@@ -52,8 +52,12 @@ public abstract class InputChannel { /** The info of the input channel to identify it globally within a task. */ protected final InputChannelInfo channelInfo; + /** The parent partition of the subpartition consumed by this channel. */ protected final ResultPartitionID partitionId; + /** The index of the subpartition consumed by this channel. */ + protected final int consumedSubpartitionIndex; + protected final SingleInputGate inputGate; // - Asynchronous error notification -------------------------------------- @@ -79,6 +83,7 @@ protected InputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, int initialBackoff, int maxBackoff, Counter numBytesIn, @@ -95,6 +100,9 @@ protected InputChannel( this.channelInfo = new InputChannelInfo(inputGate.getGateIndex(), channelIndex); this.partitionId = checkNotNull(partitionId); + checkArgument(consumedSubpartitionIndex >= 0); + this.consumedSubpartitionIndex = consumedSubpartitionIndex; + this.initialBackoff = initial; this.maxBackoff = max; this.currentBackoff = initial == 0 ? -1 : 0; @@ -124,6 +132,10 @@ public ResultPartitionID getPartitionId() { return partitionId; } + public int getConsumedSubpartitionIndex() { + return consumedSubpartitionIndex; + } + /** * After sending a {@link org.apache.flink.runtime.io.network.api.CheckpointBarrier} of * exactly-once mode, the upstream will be blocked and become unavailable. This method tries to @@ -163,13 +175,10 @@ protected void notifyBufferAvailable(int numAvailableBuffers) throws IOException // ------------------------------------------------------------------------ /** - * Requests the queue with the specified index of the source intermediate result partition. - * - *

The queue index to request depends on which sub task the channel belongs to and is - * specified by the consumer of this channel. + * Requests the subpartition specified by {@link #partitionId} and {@link + * #consumedSubpartitionIndex}. */ - abstract void requestSubpartition(int subpartitionIndex) - throws IOException, InterruptedException; + abstract void requestSubpartition() throws IOException, InterruptedException; /** * Returns the next buffer from the consumed subpartition or {@code Optional.empty()} if there diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java index 13c6538a8fc06..47297234f8da2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java @@ -76,6 +76,7 @@ public LocalInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ResultPartitionManager partitionManager, TaskEventPublisher taskEventPublisher, int initialBackoff, @@ -88,6 +89,7 @@ public LocalInputChannel( inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackoff, maxBackoff, numBytesIn, @@ -111,7 +113,7 @@ public void checkpointStopped(long checkpointId) { } @Override - protected void requestSubpartition(int subpartitionIndex) throws IOException { + protected void requestSubpartition() throws IOException { boolean retriggerRequest = false; boolean notifyDataAvailable = false; @@ -124,14 +126,14 @@ protected void requestSubpartition(int subpartitionIndex) throws IOException { LOG.debug( "{}: Requesting LOCAL subpartition {} of partition {}. {}", this, - subpartitionIndex, + consumedSubpartitionIndex, partitionId, channelStatePersister); try { ResultSubpartitionView subpartitionView = partitionManager.createSubpartitionView( - partitionId, subpartitionIndex, this); + partitionId, consumedSubpartitionIndex, this); if (subpartitionView == null) { throw new IOException("Error requesting subpartition."); @@ -170,7 +172,7 @@ protected void requestSubpartition(int subpartitionIndex) throws IOException { } /** Retriggers a subpartition request. */ - void retriggerSubpartitionRequest(Timer timer, final int subpartitionIndex) { + void retriggerSubpartitionRequest(Timer timer) { synchronized (requestLock) { checkState(subpartitionView == null, "already requested partition"); @@ -179,7 +181,7 @@ void retriggerSubpartitionRequest(Timer timer, final int subpartitionIndex) { @Override public void run() { try { - requestSubpartition(subpartitionIndex); + requestSubpartition(); } catch (Throwable t) { setError(t); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalRecoveredInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalRecoveredInputChannel.java index 1b790a9e91e5c..fa106188b03eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalRecoveredInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalRecoveredInputChannel.java @@ -37,6 +37,7 @@ public class LocalRecoveredInputChannel extends RecoveredInputChannel { SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ResultPartitionManager partitionManager, TaskEventPublisher taskEventPublisher, int initialBackOff, @@ -47,6 +48,7 @@ public class LocalRecoveredInputChannel extends RecoveredInputChannel { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackOff, maxBackoff, metrics.getNumBytesInLocalCounter(), @@ -63,6 +65,7 @@ protected InputChannel toInputChannelInternal() { inputGate, getChannelIndex(), partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannel.java index 7f31116d2d763..469b05e96a19d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannel.java @@ -76,6 +76,7 @@ public abstract class RecoveredInputChannel extends InputChannel implements Chan SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, int initialBackoff, int maxBackoff, Counter numBytesIn, @@ -85,6 +86,7 @@ public abstract class RecoveredInputChannel extends InputChannel implements Chan inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackoff, maxBackoff, numBytesIn, @@ -223,7 +225,7 @@ public void acknowledgeAllRecordsProcessed() throws IOException { } @Override - final void requestSubpartition(int subpartitionIndex) { + final void requestSubpartition() { throw new UnsupportedOperationException( "RecoveredInputChannel should never request partition."); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index a9bce9935b7ab..c0a83f9edee0b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -119,6 +119,7 @@ public RemoteInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ConnectionID connectionId, ConnectionManager connectionManager, int initialBackOff, @@ -132,6 +133,7 @@ public RemoteInputChannel( inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackOff, maxBackoff, numBytesIn, @@ -170,13 +172,12 @@ void setup() throws IOException { /** Requests a remote subpartition. */ @VisibleForTesting @Override - public void requestSubpartition(int subpartitionIndex) - throws IOException, InterruptedException { + public void requestSubpartition() throws IOException, InterruptedException { if (partitionRequestClient == null) { LOG.debug( "{}: Requesting REMOTE subpartition {} of partition {}. {}", this, - subpartitionIndex, + consumedSubpartitionIndex, partitionId, channelStatePersister); // Create a client and request the partition @@ -189,17 +190,18 @@ public void requestSubpartition(int subpartitionIndex) throw new PartitionConnectionException(partitionId, e); } - partitionRequestClient.requestSubpartition(partitionId, subpartitionIndex, this, 0); + partitionRequestClient.requestSubpartition( + partitionId, consumedSubpartitionIndex, this, 0); } } /** Retriggers a remote subpartition request. */ - void retriggerSubpartitionRequest(int subpartitionIndex) throws IOException { + void retriggerSubpartitionRequest() throws IOException { checkPartitionRequestQueueInitialized(); if (increaseBackoff()) { partitionRequestClient.requestSubpartition( - partitionId, subpartitionIndex, this, getCurrentBackoff()); + partitionId, consumedSubpartitionIndex, this, getCurrentBackoff()); } else { failPartitionRequest(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java index 18ac6bf42e376..2c0efa01592ab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java @@ -39,6 +39,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel { SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ConnectionID connectionId, ConnectionManager connectionManager, int initialBackOff, @@ -49,6 +50,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackOff, maxBackoff, metrics.getNumBytesInRemoteCounter(), @@ -66,6 +68,7 @@ protected InputChannel toInputChannelInternal() throws IOException { inputGate, getChannelIndex(), partitionId, + consumedSubpartitionIndex, connectionId, connectionManager, initialBackoff, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index 00e05ea745324..9ba3325aeb1b0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -40,7 +40,6 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; -import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; @@ -133,12 +132,6 @@ public class SingleInputGate extends IndexedInputGate { /** The type of the partition the input gate is consuming. */ private final ResultPartitionType consumedPartitionType; - /** - * The index of the consumed subpartition of each consumed partition. This index depends on the - * {@link DistributionPattern} and the subtask indices of the producing and consuming task. - */ - private final int consumedSubpartitionIndex; - /** The number of input channels (equivalent to the number of consumed partitions). */ private final int numberOfInputChannels; @@ -216,7 +209,6 @@ public SingleInputGate( int gateIndex, IntermediateDataSetID consumedResultId, final ResultPartitionType consumedPartitionType, - int consumedSubpartitionIndex, int numberOfInputChannels, PartitionProducerStateProvider partitionProducerStateProvider, SupplierWithException bufferPoolFactory, @@ -234,9 +226,6 @@ public SingleInputGate( this.consumedPartitionType = checkNotNull(consumedPartitionType); this.bufferPoolFactory = checkNotNull(bufferPoolFactory); - checkArgument(consumedSubpartitionIndex >= 0); - this.consumedSubpartitionIndex = consumedSubpartitionIndex; - checkArgument(numberOfInputChannels > 0); this.numberOfInputChannels = numberOfInputChannels; @@ -339,7 +328,7 @@ public void convertRecoveredInputChannels() { private void internalRequestPartitions() { for (InputChannel inputChannel : inputChannels.values()) { try { - inputChannel.requestSubpartition(consumedSubpartitionIndex); + inputChannel.requestSubpartition(); } catch (Throwable t) { inputChannel.setError(t); return; @@ -581,7 +570,7 @@ public void updateInputChannel( channels[current.getChannelIndex()] = newChannel; if (requestedPartitionsFlag) { - newChannel.requestSubpartition(consumedSubpartitionIndex); + newChannel.requestSubpartition(); } for (TaskEvent event : pendingEvents) { @@ -608,11 +597,11 @@ public void retriggerPartitionRequest(IntermediateResultPartitionID partitionId) "{}: Retriggering partition request {}:{}.", owningTaskName, ch.partitionId, - consumedSubpartitionIndex); + ch.getConsumedSubpartitionIndex()); if (ch.getClass() == RemoteInputChannel.class) { final RemoteInputChannel rch = (RemoteInputChannel) ch; - rch.retriggerSubpartitionRequest(consumedSubpartitionIndex); + rch.retriggerSubpartitionRequest(); } else if (ch.getClass() == LocalInputChannel.class) { final LocalInputChannel ich = (LocalInputChannel) ch; @@ -620,8 +609,7 @@ public void retriggerPartitionRequest(IntermediateResultPartitionID partitionId) retriggerLocalRequestTimer = new Timer(true); } - ich.retriggerSubpartitionRequest( - retriggerLocalRequestTimer, consumedSubpartitionIndex); + ich.retriggerSubpartitionRequest(retriggerLocalRequestTimer); } else { throw new IllegalStateException( "Unexpected type of channel to retrigger partition: " + ch.getClass()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index 17a9c1a8fd7b9..5cc43898bc44a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -132,7 +132,6 @@ public SingleInputGate create( gateIndex, igdd.getConsumedResultId(), igdd.getConsumedPartitionType(), - igdd.getConsumedSubpartitionIndex(), igdd.getShuffleDescriptors().length, partitionProducerStateProvider, bufferPoolFactory, @@ -145,7 +144,8 @@ public SingleInputGate create( InputChannelMetrics metrics = new InputChannelMetrics(networkInputGroup, owner.getParentGroup()); - createInputChannels(owningTaskName, igdd, inputGate, metrics); + createInputChannels( + owningTaskName, igdd, inputGate, igdd.getConsumedSubpartitionIndex(), metrics); return inputGate; } @@ -173,6 +173,7 @@ private void createInputChannels( String owningTaskName, InputGateDeploymentDescriptor inputGateDeploymentDescriptor, SingleInputGate inputGate, + int consumedSubpartitionIndex, InputChannelMetrics metrics) { ShuffleDescriptor[] shuffleDescriptors = inputGateDeploymentDescriptor.getShuffleDescriptors(); @@ -185,7 +186,12 @@ private void createInputChannels( for (int i = 0; i < inputChannels.length; i++) { inputChannels[i] = createInputChannel( - inputGate, i, shuffleDescriptors[i], channelStatistics, metrics); + inputGate, + i, + shuffleDescriptors[i], + consumedSubpartitionIndex, + channelStatistics, + metrics); } inputGate.setInputChannels(inputChannels); @@ -200,6 +206,7 @@ private InputChannel createInputChannel( SingleInputGate inputGate, int index, ShuffleDescriptor shuffleDescriptor, + int consumedSubpartitionIndex, ChannelStatistics channelStatistics, InputChannelMetrics metrics) { return applyWithShuffleTypeCheck( @@ -211,6 +218,7 @@ private InputChannel createInputChannel( inputGate, index, unknownShuffleDescriptor.getResultPartitionID(), + consumedSubpartitionIndex, partitionManager, taskEventPublisher, connectionManager, @@ -224,6 +232,7 @@ private InputChannel createInputChannel( inputGate, index, nettyShuffleDescriptor, + consumedSubpartitionIndex, channelStatistics, metrics)); } @@ -233,6 +242,7 @@ protected InputChannel createKnownInputChannel( SingleInputGate inputGate, int index, NettyShuffleDescriptor inputChannelDescriptor, + int consumedSubpartitionIndex, ChannelStatistics channelStatistics, InputChannelMetrics metrics) { ResultPartitionID partitionId = inputChannelDescriptor.getResultPartitionID(); @@ -243,6 +253,7 @@ protected InputChannel createKnownInputChannel( inputGate, index, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, partitionRequestInitialBackoff, @@ -256,6 +267,7 @@ protected InputChannel createKnownInputChannel( inputGate, index, partitionId, + consumedSubpartitionIndex, inputChannelDescriptor.getConnectionId(), connectionManager, partitionRequestInitialBackoff, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java index 88d91c7c3916f..18696072cd712 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java @@ -66,6 +66,7 @@ public UnknownInputChannel( SingleInputGate gate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ResultPartitionManager partitionManager, TaskEventPublisher taskEventPublisher, ConnectionManager connectionManager, @@ -74,7 +75,15 @@ public UnknownInputChannel( int networkBuffersPerChannel, InputChannelMetrics metrics) { - super(gate, channelIndex, partitionId, initialBackoff, maxBackoff, null, null); + super( + gate, + channelIndex, + partitionId, + consumedSubpartitionIndex, + initialBackoff, + maxBackoff, + null, + null); this.partitionManager = checkNotNull(partitionManager); this.taskEventPublisher = checkNotNull(taskEventPublisher); @@ -97,7 +106,7 @@ public void acknowledgeAllRecordsProcessed() throws IOException { } @Override - public void requestSubpartition(int subpartitionIndex) throws IOException { + public void requestSubpartition() throws IOException { // Nothing to do here } @@ -154,6 +163,7 @@ public RemoteInputChannel toRemoteInputChannel(ConnectionID producerAddress) { inputGate, getChannelIndex(), partitionId, + consumedSubpartitionIndex, checkNotNull(producerAddress), connectionManager, initialBackoff, @@ -169,6 +179,7 @@ public LocalInputChannel toLocalInputChannel() { inputGate, getChannelIndex(), partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java index c53717edb514a..f3074f9682b94 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java @@ -405,8 +405,8 @@ public void testNotifyCreditAvailable() throws Exception { inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannels[0].requestSubpartition(0); - inputChannels[1].requestSubpartition(0); + inputChannels[0].requestSubpartition(); + inputChannels[1].requestSubpartition(); // The two input channels should send partition requests assertTrue(channel.isWritable()); @@ -532,7 +532,7 @@ public void testNotifyCreditAvailableAfterReleased() throws Exception { inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // This should send the partition request Object readFromOutbound = channel.readOutbound(); @@ -699,8 +699,8 @@ public void testAnnounceBufferSize() throws Exception { inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannels[0].requestSubpartition(0); - inputChannels[1].requestSubpartition(0); + inputChannels[0].requestSubpartition(); + inputChannels[1].requestSubpartition(); channel.readOutbound(); channel.readOutbound(); @@ -827,6 +827,7 @@ private static class TestRemoteInputChannelForError extends RemoteInputChannel { inputGate, 0, new ResultPartitionID(), + 0, InputChannelBuilder.STUB_CONNECTION_ID, new TestingConnectionManager(), 0, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java index 4931a2f934fe2..801e478b3404c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java @@ -85,7 +85,7 @@ public void setup() throws IOException, InterruptedException { inputGate, new TestingPartitionRequestClient(), NUMBER_OF_BUFFER_RESPONSES); inputGate.setInputChannels(inputChannel); inputGate.setup(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); handler.addInputChannel(inputChannel); inputChannelId = inputChannel.getInputChannelId(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java index cba3be2575feb..ee42d8dd748a2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java @@ -84,7 +84,7 @@ public void setup() throws IOException, InterruptedException { inputGate = createSingleInputGate(1, networkBufferPool); RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate, new TestingPartitionRequestClient()); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); inputGate.setInputChannels(inputChannel); inputGate.setup(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java index 3d8e8d5658bce..f53d2d798ff62 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java @@ -78,7 +78,7 @@ public void testRetriggerPartitionRequest() throws Exception { inputGate.setupChannels(); // first subpartition request - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); assertTrue(channel.isWritable()); Object readFromOutbound = channel.readOutbound(); @@ -137,7 +137,7 @@ public void testDoublePartitionRequest() throws Exception { final BufferPool bufferPool = networkBufferPool.createBufferPool(6, 6); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // The input channel should only send one partition request assertTrue(channel.isWritable()); @@ -173,7 +173,7 @@ public void testResumeConsumption() throws Exception { final BufferPool bufferPool = networkBufferPool.createBufferPool(6, 6); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); inputChannel.resumeConsumption(); channel.runPendingTasks(); @@ -211,7 +211,7 @@ public void testAcknowledgeAllRecordsProcessed() throws Exception { BufferPool bufferPool = networkBufferPool.createBufferPool(6, 6); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); inputChannel.acknowledgeAllRecordsProcessed(); channel.runPendingTasks(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java index 3fdab4419fe28..b5e77f9d8a37c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java @@ -304,7 +304,7 @@ public void testFairConsumptionRemoteChannels() throws Exception { private SingleInputGate createFairnessVerifyingInputGate(int numberOfChannels) { return new FairnessVerifyingInputGate( - "Test Task Name", new IntermediateDataSetID(), 0, numberOfChannels); + "Test Task Name", new IntermediateDataSetID(), numberOfChannels); } private void fillRandom( @@ -362,7 +362,6 @@ private static class FairnessVerifyingInputGate extends SingleInputGate { public FairnessVerifyingInputGate( String owningTaskName, IntermediateDataSetID consumedResultId, - int consumedSubpartitionIndex, int numberOfInputChannels) { super( @@ -370,7 +369,6 @@ public FairnessVerifyingInputGate( 0, consumedResultId, ResultPartitionType.PIPELINED, - consumedSubpartitionIndex, numberOfInputChannels, SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER, STUB_BUFFER_POOL_FACTORY, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java index 3da8e21d18762..7d4b434995c0d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java @@ -215,7 +215,7 @@ public void testFloatingBuffersUsage() throws Exception { extraNetworkBuffersPerGate * inputGates.length + buffersPerChannel * totalNumberOfRemoteChannels; - remoteInputChannel1.requestSubpartition(0); + remoteInputChannel1.requestSubpartition(); int backlog = 3; int totalRequestedBuffers = buffersPerChannel + backlog; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java index cb6e975abf578..bb5d24a736b85 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java @@ -42,6 +42,7 @@ public class InputChannelBuilder { private int channelIndex = 0; private ResultPartitionID partitionId = new ResultPartitionID(); + private int consumedSubpartitionIndex = 0; private ConnectionID connectionID = STUB_CONNECTION_ID; private ResultPartitionManager partitionManager = new TestingResultPartitionManager(new NoOpResultSubpartitionView()); @@ -68,6 +69,11 @@ public InputChannelBuilder setPartitionId(ResultPartitionID partitionId) { return this; } + public InputChannelBuilder setConsumedSubpartitionIndex(int consumedSubpartitionIndex) { + this.consumedSubpartitionIndex = consumedSubpartitionIndex; + return this; + } + public InputChannelBuilder setPartitionManager(ResultPartitionManager partitionManager) { this.partitionManager = partitionManager; return this; @@ -123,6 +129,7 @@ UnknownInputChannel buildUnknownChannel(SingleInputGate inputGate) { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, connectionManager, @@ -139,6 +146,7 @@ public LocalInputChannel buildLocalChannel(SingleInputGate inputGate) { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, @@ -153,6 +161,7 @@ public RemoteInputChannel buildRemoteChannel(SingleInputGate inputGate) { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, connectionID, connectionManager, initialBackoff, @@ -169,6 +178,7 @@ public LocalRecoveredInputChannel buildLocalRecoveredChannel(SingleInputGate inp inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, @@ -185,6 +195,7 @@ public RemoteRecoveredInputChannel buildRemoteRecoveredChannel(SingleInputGate i inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, connectionID, connectionManager, initialBackoff, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java index d6de0ea72f035..d8757331742ac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java @@ -126,6 +126,7 @@ private MockInputChannel( inputGate, channelIndex, partitionId, + 0, initialBackoff, maxBackoff, new SimpleCounter(), @@ -139,7 +140,7 @@ public void resumeConsumption() {} public void acknowledgeAllRecordsProcessed() throws IOException {} @Override - void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException {} + void requestSubpartition() throws IOException, InterruptedException {} @Override Optional getNextBuffer() throws IOException, InterruptedException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index 30f355ab8ab57..1df9fc556a27b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -116,7 +116,7 @@ public void testNoDataPersistedAfterReceivingAlignedBarrier() throws Exception { barrierHolder, data))) .setStateWriter(stateWriter) .buildLocalChannel(new SingleInputGateBuilder().build()); - channel.requestSubpartition(0); + channel.requestSubpartition(); // pull AC barrier channel.getNextBuffer(); @@ -261,21 +261,21 @@ public void testPartitionRequestExponentialBackoff() throws Exception { .schedule(any(TimerTask.class), anyLong()); // Initial request - ch.requestSubpartition(0); + ch.requestSubpartition(); verify(partitionManager) .createSubpartitionView( eq(ch.partitionId), eq(0), any(BufferAvailabilityListener.class)); // Request subpartition and verify that the actual requests are delayed. for (long expected : expectedDelays) { - ch.retriggerSubpartitionRequest(timer, 0); + ch.retriggerSubpartitionRequest(timer); verify(timer).schedule(any(TimerTask.class), eq(expected)); } // Exception after backoff is greater than the maximum backoff. try { - ch.retriggerSubpartitionRequest(timer, 0); + ch.retriggerSubpartitionRequest(timer); ch.getNextBuffer(); fail("Did not throw expected exception."); } catch (Exception expected) { @@ -301,14 +301,14 @@ public void testProducerFailedException() throws Exception { LocalInputChannel ch = createLocalInputChannel(inputGate, partitionManager); - ch.requestSubpartition(0); + ch.requestSubpartition(); // Should throw an instance of CancelTaskException. ch.getNextBuffer(); } /** - * Tests that {@link LocalInputChannel#requestSubpartition(int)} throws {@link + * Tests that {@link LocalInputChannel#requestSubpartition()} throws {@link * PartitionNotFoundException} if the result partition was not registered in {@link * ResultPartitionManager} and no backoff. */ @@ -319,7 +319,7 @@ public void testPartitionNotFoundExceptionWhileRequestingPartition() throws Exce createLocalInputChannel(inputGate, new ResultPartitionManager()); try { - localChannel.requestSubpartition(0); + localChannel.requestSubpartition(); fail("Should throw a PartitionNotFoundException."); } catch (PartitionNotFoundException notFound) { @@ -329,7 +329,7 @@ public void testPartitionNotFoundExceptionWhileRequestingPartition() throws Exce /** * Tests that {@link SingleInputGate#retriggerPartitionRequest(IntermediateResultPartitionID)} - * is triggered after {@link LocalInputChannel#requestSubpartition(int)} throws {@link + * is triggered after {@link LocalInputChannel#requestSubpartition()} throws {@link * PartitionNotFoundException} within backoff. */ @Test @@ -339,15 +339,15 @@ public void testRetriggerPartitionRequestWhilePartitionNotFound() throws Excepti createLocalInputChannel(inputGate, new ResultPartitionManager(), 1, 1); inputGate.setInputChannels(localChannel); - localChannel.requestSubpartition(0); + localChannel.requestSubpartition(); // The timer should be initialized at the first time of retriggering partition request. assertNotNull(inputGate.getRetriggerLocalRequestTimer()); } /** - * Tests that {@link LocalInputChannel#retriggerSubpartitionRequest(Timer, int)} would throw - * {@link PartitionNotFoundException} which is set onto the input channel then. + * Tests that {@link LocalInputChannel#retriggerSubpartitionRequest(Timer)} would throw {@link + * PartitionNotFoundException} which is set onto the input channel then. */ @Test public void testChannelErrorWhileRetriggeringRequest() { @@ -376,7 +376,7 @@ public void schedule(TimerTask task, long delay) { }; try { - localChannel.retriggerSubpartitionRequest(timer, 0); + localChannel.retriggerSubpartitionRequest(timer); } finally { timer.cancel(); } @@ -437,7 +437,7 @@ public void testConcurrentReleaseAndRetriggerPartitionRequest() throws Exception new Thread( () -> { try { - channel.requestSubpartition(0); + channel.requestSubpartition(); } catch (IOException ignored) { } }); @@ -462,7 +462,7 @@ public void testGetNextAfterPartitionReleased() throws Exception { LocalInputChannel channel = createLocalInputChannel(new SingleInputGateBuilder().build(), partitionManager); - channel.requestSubpartition(0); + channel.requestSubpartition(); assertFalse(channel.getNextBuffer().isPresent()); // release the subpartition view @@ -489,7 +489,7 @@ public void testGetBufferFromLocalChannelWhenCompressionEnabled() throws Excepti createLocalInputChannel(new SingleInputGateBuilder().build(), partitionManager); // request partition and get next buffer - channel.requestSubpartition(0); + channel.requestSubpartition(); Optional bufferAndAvailability = channel.getNextBuffer(); assertTrue(bufferAndAvailability.isPresent()); @@ -515,7 +515,7 @@ public void testAnnounceBufferSize() throws Exception { InputChannelTestUtils.createResultSubpartitionView(true)); SingleInputGate inputGate = createSingleInputGate(1); LocalInputChannel localChannel = createLocalInputChannel(inputGate, partitionManager); - localChannel.requestSubpartition(0); + localChannel.requestSubpartition(); localChannel.announceBufferSize(10); @@ -539,7 +539,7 @@ public void testEnqueueAvailableChannelWhenResuming() throws IOException, Interr new TestingResultPartitionManager(subpartitionView); LocalInputChannel channel = createLocalInputChannel(new SingleInputGateBuilder().build(), partitionManager); - channel.requestSubpartition(0); + channel.requestSubpartition(); // Block the subpartition subpartition.add( @@ -581,7 +581,7 @@ public void testCheckpointingInflightData() throws Exception { createLocalInputChannel( inputGate, partitionManager, 0, 0, b -> b.setStateWriter(stateWriter)); inputGate.setInputChannels(channel); - channel.requestSubpartition(0); + channel.requestSubpartition(); final CheckpointStorageLocationReference location = getDefault(); CheckpointOptions options = @@ -631,8 +631,8 @@ public void testAnnounceNewBufferSize() throws IOException, InterruptedException new SingleInputGateBuilder().build(), new TestingResultPartitionManager(subpartition1.createReadView(() -> {}))); - channel0.requestSubpartition(0); - channel1.requestSubpartition(0); + channel0.requestSubpartition(); + channel1.requestSubpartition(); // and: Preferable buffer size is default value. assertEquals(Integer.MAX_VALUE, subpartition0.add(createFilledFinishedBufferConsumer(16))); @@ -666,7 +666,7 @@ public void testReceivingBuffersInUseBeforeSubpartitionViewInitialization() thro assertEquals(0, localChannel.getBuffersInUseCount()); // when: The subpartition view is initialized. - localChannel.requestSubpartition(0); + localChannel.requestSubpartition(); // then: Buffers in use should show correct value. assertEquals(3, localChannel.getBuffersInUseCount()); @@ -737,7 +737,6 @@ public TestLocalInputChannelConsumer( this.inputGate = new SingleInputGateBuilder() - .setConsumedSubpartitionIndex(subpartitionIndex) .setNumberOfChannels(numberOfInputChannels) .setBufferPoolFactory(bufferPool) .build(); @@ -748,6 +747,7 @@ public TestLocalInputChannelConsumer( inputChannels[i] = InputChannelBuilder.newBuilder() .setChannelIndex(i) + .setConsumedSubpartitionIndex(subpartitionIndex) .setPartitionManager(partitionManager) .setPartitionId(consumedPartitionIds[i]) .setTaskEventPublisher(taskEventDispatcher) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java index 99a562b50ef08..eb3fb01180a15 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java @@ -41,7 +41,7 @@ public void testConversionOnlyPossibleAfterConsumed() throws IOException { @Test(expected = UnsupportedOperationException.class) public void testRequestPartitionsImpossible() { - buildChannel().requestSubpartition(0); + buildChannel().requestSubpartition(); } @Test(expected = CheckpointException.class) @@ -60,6 +60,7 @@ private RecoveredInputChannel buildChannel() { new ResultPartitionID(), 0, 0, + 0, new SimpleCounter(), new SimpleCounter(), 10) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index 942d91caa6642..71d1e5f2a1ce6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -139,7 +139,7 @@ public void testGateNotifiedOnBarrierConversion() throws IOException, Interrupte new TestVerifyConnectionManager( new TestVerifyPartitionRequestClient())) .buildRemoteChannel(inputGate); - channel.requestSubpartition(0); + channel.requestSubpartition(); channel.onBuffer( toBuffer( @@ -342,9 +342,9 @@ private void testConcurrentReleaseAndSomething( public void testRetriggerWithoutPartitionRequest() throws Exception { SingleInputGate inputGate = createSingleInputGate(1); - RemoteInputChannel ch = createRemoteInputChannel(inputGate, 500, 3000); + RemoteInputChannel ch = createRemoteInputChannel(inputGate, 0, 500, 3000); - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); } @Test @@ -361,19 +361,19 @@ public void testPartitionRequestExponentialBackoff() throws Exception { createRemoteInputChannel(inputGate, connectionManager, partitionId, 500, 3000); // Initial request - ch.requestSubpartition(0); + ch.requestSubpartition(); client.verifyResult(partitionId, 0, 0); // Request subpartition and verify that the actual requests are delayed. for (int expected : expectedDelays) { - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); client.verifyResult(partitionId, 0, expected); } // Exception after backoff is greater than the maximum backoff. try { - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); ch.getNextBuffer(); fail("Did not throw expected exception."); } catch (Exception expected) { @@ -391,16 +391,16 @@ public void testPartitionRequestSingleBackoff() throws Exception { createRemoteInputChannel(inputGate, connectionManager, partitionId, 500, 500); // No delay for first request - ch.requestSubpartition(0); + ch.requestSubpartition(); client.verifyResult(partitionId, 0, 0); // Initial delay for second request - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); client.verifyResult(partitionId, 0, 500); // Exception after backoff is greater than the maximum backoff. try { - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); ch.getNextBuffer(); fail("Did not throw expected exception."); } catch (Exception expected) { @@ -418,12 +418,12 @@ public void testPartitionRequestNoBackoff() throws Exception { createRemoteInputChannel(inputGate, connectionManager, partitionId, 0, 0); // No delay for first request - ch.requestSubpartition(0); + ch.requestSubpartition(); client.verifyResult(partitionId, 0, 0); // Exception, because backoff is disabled. try { - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); ch.getNextBuffer(); fail("Did not throw expected exception."); } catch (Exception expected) { @@ -460,7 +460,7 @@ public void testProducerFailedException() throws Exception { ch.onError(new ProducerFailedException(new RuntimeException("Expected test exception."))); - ch.requestSubpartition(0); + ch.requestSubpartition(); // Should throw an instance of CancelTaskException. ch.getNextBuffer(); @@ -504,7 +504,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers)); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); @@ -692,7 +692,7 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers)); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); @@ -788,7 +788,7 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers)); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); @@ -911,7 +911,7 @@ public void testFairDistributionFloatingBuffers() throws Exception { inputGate.setupChannels(); inputGate.requestPartitions(); for (RemoteInputChannel inputChannel : inputChannels) { - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); } // Exhaust all the floating buffers @@ -972,7 +972,7 @@ public void testFailureInNotifyBufferAvailable() throws Exception { final SingleInputGate inputGate = createSingleInputGate(1); final RemoteInputChannel successfulRemoteIC = createRemoteInputChannel(inputGate); - successfulRemoteIC.requestSubpartition(0); + successfulRemoteIC.requestSubpartition(); // late creation -> no exclusive buffers, also no requested subpartition in // successfulRemoteIC @@ -1041,7 +1041,7 @@ public void testConcurrentOnSenderBacklogAndRelease() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); final Callable requestBufferTask = new Callable() { @@ -1113,7 +1113,7 @@ public void testConcurrentOnSenderBacklogAndRecycle() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); final Callable requestBufferTask = new Callable() { @@ -1177,7 +1177,7 @@ public void testConcurrentRecycleAndRelease() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); final Callable releaseTask = new Callable() { @@ -1244,7 +1244,7 @@ public void testConcurrentRecycleAndRelease2() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); final Callable bufferPoolInteractionsTask = () -> { @@ -1326,7 +1326,7 @@ public void testConcurrentGetNextBufferAndRelease() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); for (int i = 0; i < numTotalBuffers; i++) { Buffer buffer = inputChannel.requestBuffer(); @@ -1367,8 +1367,8 @@ public void testConcurrentGetNextBufferAndRelease() throws Exception { } /** - * Tests that {@link RemoteInputChannel#retriggerSubpartitionRequest(int)} would throw the - * {@link PartitionNotFoundException} if backoff is 0. + * Tests that {@link RemoteInputChannel#retriggerSubpartitionRequest()} would throw the {@link + * PartitionNotFoundException} if backoff is 0. */ @Test public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exception { @@ -1378,9 +1378,9 @@ public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exce // Request partition to initialize client to avoid illegal state after retriggering // partition - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // The default backoff is 0 then it would set PartitionNotFoundException on this channel - inputChannel.retriggerSubpartitionRequest(0); + inputChannel.retriggerSubpartitionRequest(); try { inputChannel.checkError(); @@ -1393,7 +1393,7 @@ public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exce /** * Tests that any exceptions thrown by {@link * ConnectionManager#createPartitionRequestClient(ConnectionID)} would be wrapped into {@link - * PartitionConnectionException} during {@link RemoteInputChannel#requestSubpartition(int)}. + * PartitionConnectionException} during {@link RemoteInputChannel#requestSubpartition()}. */ @Test public void testPartitionConnectionExceptionWhileRequestingPartition() throws Exception { @@ -1401,7 +1401,7 @@ public void testPartitionConnectionExceptionWhileRequestingPartition() throws Ex InputChannelTestUtils.createRemoteInputChannel( createSingleInputGate(1), 0, new TestingExceptionConnectionManager()); try { - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); fail("Expected PartitionConnectionException."); } catch (PartitionConnectionException ex) { assertThat(inputChannel.getPartitionId(), is(ex.getPartitionId())); @@ -1431,11 +1431,11 @@ public void testOnUpstreamBlockedAndResumed() throws Exception { BufferPool bufferPool = new TestBufferPool(); SingleInputGate inputGate = createSingleInputGate(bufferPool); - RemoteInputChannel remoteChannel1 = createRemoteInputChannel(inputGate, 2); - RemoteInputChannel remoteChannel2 = createRemoteInputChannel(inputGate, 0); + RemoteInputChannel remoteChannel1 = createRemoteInputChannel(inputGate, 0, 2); + RemoteInputChannel remoteChannel2 = createRemoteInputChannel(inputGate, 1, 0); inputGate.setup(); - remoteChannel1.requestSubpartition(0); - remoteChannel2.requestSubpartition(1); + remoteChannel1.requestSubpartition(); + remoteChannel2.requestSubpartition(); remoteChannel1.onSenderBacklog(2); remoteChannel2.onSenderBacklog(2); @@ -1480,11 +1480,11 @@ public void testRequestBuffer() throws Exception { BufferPool bufferPool = new TestBufferPool(); SingleInputGate inputGate = createSingleInputGate(bufferPool); - RemoteInputChannel remoteChannel1 = createRemoteInputChannel(inputGate, 2); - RemoteInputChannel remoteChannel2 = createRemoteInputChannel(inputGate, 0); + RemoteInputChannel remoteChannel1 = createRemoteInputChannel(inputGate, 0, 2); + RemoteInputChannel remoteChannel2 = createRemoteInputChannel(inputGate, 1, 0); inputGate.setup(); - remoteChannel1.requestSubpartition(0); - remoteChannel2.requestSubpartition(1); + remoteChannel1.requestSubpartition(); + remoteChannel2.requestSubpartition(); remoteChannel1.onSenderBacklog(2); remoteChannel2.onSenderBacklog(2); @@ -1722,19 +1722,24 @@ private void assertGetNextBufferSequenceNumbers( // --------------------------------------------------------------------------------------------- private RemoteInputChannel createRemoteInputChannel(SingleInputGate inputGate) { - return createRemoteInputChannel(inputGate, 0, 0); + return createRemoteInputChannel(inputGate, 0, 0, 0); } private RemoteInputChannel createRemoteInputChannel( - SingleInputGate inputGate, int initialCredits) { + SingleInputGate inputGate, int consumedSubpartitionIndex, int initialCredits) { return InputChannelBuilder.newBuilder() + .setConsumedSubpartitionIndex(consumedSubpartitionIndex) .setNetworkBuffersPerChannel(initialCredits) .buildRemoteChannel(inputGate); } private RemoteInputChannel createRemoteInputChannel( - SingleInputGate inputGate, int initialBackoff, int maxBackoff) { + SingleInputGate inputGate, + int consumedSubpartitionIndex, + int initialBackoff, + int maxBackoff) { return InputChannelBuilder.newBuilder() + .setConsumedSubpartitionIndex(consumedSubpartitionIndex) .setInitialBackoff(initialBackoff) .setMaxBackoff(maxBackoff) .buildRemoteChannel(inputGate); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java index 3ee0a748aabdd..524cf807c5fca 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java @@ -55,8 +55,6 @@ public class SingleInputGateBuilder { private ResultPartitionType partitionType = ResultPartitionType.PIPELINED; - private int consumedSubpartitionIndex = 0; - private int gateIndex = 0; private int numberOfChannels = 1; @@ -91,11 +89,6 @@ public SingleInputGateBuilder setResultPartitionType(ResultPartitionType partiti return this; } - public SingleInputGateBuilder setConsumedSubpartitionIndex(int consumedSubpartitionIndex) { - this.consumedSubpartitionIndex = consumedSubpartitionIndex; - return this; - } - public SingleInputGateBuilder setSingleInputGateIndex(int gateIndex) { this.gateIndex = gateIndex; return this; @@ -161,7 +154,6 @@ public SingleInputGate build() { gateIndex, intermediateDataSetID, partitionType, - consumedSubpartitionIndex, numberOfChannels, partitionProducerStateProvider, bufferPoolFactory, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java index c891a730a2079..006161fee2063 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java @@ -80,6 +80,7 @@ public TestInputChannel( new ResultPartitionID(), 0, 0, + 0, new SimpleCounter(), new SimpleCounter()); this.reuseLastReturnBuffer = reuseLastReturnBuffer; @@ -166,7 +167,7 @@ static TestInputChannel[] createInputChannels( } @Override - void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException {} + void requestSubpartition() throws IOException, InterruptedException {} @Override Optional getNextBuffer() throws IOException, InterruptedException { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java index b1276b2786d85..7dda72c3f4963 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java @@ -64,6 +64,7 @@ protected InputChannel createKnownInputChannel( SingleInputGate inputGate, int index, NettyShuffleDescriptor inputChannelDescriptor, + int consumedSubpartitionIndex, SingleInputGateFactory.ChannelStatistics channelStatistics, InputChannelMetrics metrics) { ResultPartitionID partitionId = inputChannelDescriptor.getResultPartitionID(); @@ -72,6 +73,7 @@ protected InputChannel createKnownInputChannel( inputGate, index, partitionId, + index, partitionManager, taskEventPublisher, partitionRequestInitialBackoff, @@ -82,6 +84,7 @@ protected InputChannel createKnownInputChannel( inputGate, index, partitionId, + index, inputChannelDescriptor.getConnectionId(), connectionManager, partitionRequestInitialBackoff, @@ -103,6 +106,7 @@ public TestLocalInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ResultPartitionManager partitionManager, TaskEventPublisher taskEventPublisher, int initialBackoff, @@ -112,6 +116,7 @@ public TestLocalInputChannel( inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, @@ -122,8 +127,8 @@ public TestLocalInputChannel( } @Override - public void requestSubpartition(int subpartitionIndex) throws IOException { - super.requestSubpartition(getChannelIndex()); + public void requestSubpartition() throws IOException { + super.requestSubpartition(); } @Override @@ -152,6 +157,7 @@ public TestRemoteInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ConnectionID connectionId, ConnectionManager connectionManager, int initialBackOff, @@ -162,6 +168,7 @@ public TestRemoteInputChannel( inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, connectionId, connectionManager, initialBackOff, @@ -173,9 +180,8 @@ public TestRemoteInputChannel( } @Override - public void requestSubpartition(int subpartitionIndex) - throws IOException, InterruptedException { - super.requestSubpartition(getChannelIndex()); + public void requestSubpartition() throws IOException, InterruptedException { + super.requestSubpartition(); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/CheckpointedInputGateTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/CheckpointedInputGateTest.java index ad3857f36b593..b49c47d292759 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/CheckpointedInputGateTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/CheckpointedInputGateTest.java @@ -218,7 +218,7 @@ public void testPriorityBeforeClose() throws IOException, InterruptedException { .setChannelFactory(InputChannelBuilder::buildRemoteChannel) .build(); singleInputGate.setup(); - ((RemoteInputChannel) singleInputGate.getChannel(0)).requestSubpartition(0); + ((RemoteInputChannel) singleInputGate.getChannel(0)).requestSubpartition(); final TaskMailboxImpl mailbox = new TaskMailboxImpl(); MailboxExecutorImpl mailboxExecutor = @@ -355,7 +355,7 @@ public void invoke() {} mailboxExecutor, UpstreamRecoveryTracker.forInputGate(singleInputGate)); for (int i = 0; i < numberOfChannels; i++) { - ((RemoteInputChannel) checkpointedInputGate.getChannel(i)).requestSubpartition(0); + ((RemoteInputChannel) checkpointedInputGate.getChannel(i)).requestSubpartition(); } return checkpointedInputGate; } @@ -395,7 +395,7 @@ private CheckpointedInputGate setupInputGateWithAlternatingController( mailboxExecutor, UpstreamRecoveryTracker.forInputGate(singleInputGate)); for (int i = 0; i < numberOfChannels; i++) { - ((RemoteInputChannel) checkpointedInputGate.getChannel(i)).requestSubpartition(0); + ((RemoteInputChannel) checkpointedInputGate.getChannel(i)).requestSubpartition(); } return checkpointedInputGate; } From 5c2ec72982128fc1a30014677916e0dd21f9c1d2 Mon Sep 17 00:00:00 2001 From: Lijie Wang Date: Wed, 26 Jan 2022 13:18:59 +0800 Subject: [PATCH 06/49] [FLINK-25035][runtime] SingleInputGate supports consuming subpartition range This closes #18130. --- .../deployment/SubpartitionIndexRange.java | 2 +- .../partition/consumer/LocalInputChannel.java | 3 +- .../consumer/RemoteInputChannel.java | 2 +- .../partition/consumer/SingleInputGate.java | 129 ++++++--- .../consumer/SingleInputGateFactory.java | 50 ++-- .../NettyPartitionRequestClientTest.java | 8 +- .../partition/InputGateFairnessTest.java | 8 +- .../consumer/SingleInputGateBuilder.java | 10 + .../consumer/SingleInputGateTest.java | 254 ++++++++++++++---- 9 files changed, 358 insertions(+), 108 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/SubpartitionIndexRange.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/SubpartitionIndexRange.java index 19484a6cb3b67..5682f04ca92d8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/SubpartitionIndexRange.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/SubpartitionIndexRange.java @@ -27,7 +27,7 @@ public class SubpartitionIndexRange implements Serializable { private final int startIndex; private final int endIndex; - SubpartitionIndexRange(int startIndex, int endIndex) { + public SubpartitionIndexRange(int startIndex, int endIndex) { checkArgument(startIndex >= 0); checkArgument(endIndex >= startIndex); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java index 47297234f8da2..f409a68524f97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java @@ -167,7 +167,8 @@ protected void requestSubpartition() throws IOException { // deadlock with a concurrent release of the channel via the // input gate. if (retriggerRequest) { - inputGate.retriggerPartitionRequest(partitionId.getPartitionId()); + inputGate.retriggerPartitionRequest( + partitionId.getPartitionId(), consumedSubpartitionIndex); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index c0a83f9edee0b..5b476b5db0a70 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -799,7 +799,7 @@ public void onEmptyBuffer(int sequenceNumber, int backlog) throws IOException { } public void onFailedPartitionRequest() { - inputGate.triggerPartitionStateCheck(partitionId); + inputGate.triggerPartitionStateCheck(partitionId, consumedSubpartitionIndex); } public void onError(Throwable cause) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index 9ba3325aeb1b0..09f8144337ae8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -24,6 +24,7 @@ import org.apache.flink.core.memory.MemorySegmentProvider; import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.execution.CancelTaskException; @@ -40,6 +41,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; +import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; @@ -132,14 +134,21 @@ public class SingleInputGate extends IndexedInputGate { /** The type of the partition the input gate is consuming. */ private final ResultPartitionType consumedPartitionType; + /** + * Range of the index of the consumed subpartition of each consumed partition. This index + * depends on the {@link DistributionPattern} and the subtask indices of the producing and + * consuming task. The range is inclusive. + */ + private final SubpartitionIndexRange subpartitionIndexRange; + /** The number of input channels (equivalent to the number of consumed partitions). */ private final int numberOfInputChannels; /** - * Input channels. There is a one input channel for each consumed intermediate result partition. - * We store this in a map for runtime updates of single channels. + * Input channels. There is one input channel for each consumed subpartition. We store this in a + * map for runtime updates of single channels. */ - private final Map inputChannels; + private final Map inputChannels; @GuardedBy("requestLock") private final InputChannel[] channels; @@ -209,6 +218,7 @@ public SingleInputGate( int gateIndex, IntermediateDataSetID consumedResultId, final ResultPartitionType consumedPartitionType, + SubpartitionIndexRange subpartitionIndexRange, int numberOfInputChannels, PartitionProducerStateProvider partitionProducerStateProvider, SupplierWithException bufferPoolFactory, @@ -226,6 +236,8 @@ public SingleInputGate( this.consumedPartitionType = checkNotNull(consumedPartitionType); this.bufferPoolFactory = checkNotNull(bufferPoolFactory); + this.subpartitionIndexRange = checkNotNull(subpartitionIndexRange); + checkArgument(numberOfInputChannels > 0); this.numberOfInputChannels = numberOfInputChannels; @@ -307,8 +319,7 @@ public void requestPartitions() { @VisibleForTesting public void convertRecoveredInputChannels() { LOG.debug("Converting recovered input channels ({} channels)", getNumberOfInputChannels()); - for (Map.Entry entry : - inputChannels.entrySet()) { + for (Map.Entry entry : inputChannels.entrySet()) { InputChannel inputChannel = entry.getValue(); if (inputChannel instanceof RecoveredInputChannel) { try { @@ -528,7 +539,11 @@ public void setInputChannels(InputChannel... channels) { for (InputChannel inputChannel : channels) { IntermediateResultPartitionID partitionId = inputChannel.getPartitionId().getPartitionId(); - if (inputChannels.put(partitionId, inputChannel) == null + int subpartitionIndex = inputChannel.getConsumedSubpartitionIndex(); + if (inputChannels.put( + new SubpartitionInfo(partitionId, subpartitionIndex), + inputChannel) + == null && inputChannel instanceof UnknownInputChannel) { numberOfUninitializedChannels++; @@ -549,47 +564,55 @@ public void updateInputChannel( IntermediateResultPartitionID partitionId = shuffleDescriptor.getResultPartitionID().getPartitionId(); - InputChannel current = inputChannels.get(partitionId); - - if (current instanceof UnknownInputChannel) { - UnknownInputChannel unknownChannel = (UnknownInputChannel) current; - boolean isLocal = shuffleDescriptor.isLocalTo(localLocation); - InputChannel newChannel; - if (isLocal) { - newChannel = unknownChannel.toLocalInputChannel(); - } else { - RemoteInputChannel remoteInputChannel = - unknownChannel.toRemoteInputChannel( - shuffleDescriptor.getConnectionId()); - remoteInputChannel.setup(); - newChannel = remoteInputChannel; - } - LOG.debug("{}: Updated unknown input channel to {}.", owningTaskName, newChannel); + for (int subpartitionIndex = subpartitionIndexRange.getStartIndex(); + subpartitionIndex <= subpartitionIndexRange.getEndIndex(); + ++subpartitionIndex) { + SubpartitionInfo subpartitionInfo = + new SubpartitionInfo(partitionId, subpartitionIndex); + InputChannel current = inputChannels.get(subpartitionInfo); + + if (current instanceof UnknownInputChannel) { + UnknownInputChannel unknownChannel = (UnknownInputChannel) current; + boolean isLocal = shuffleDescriptor.isLocalTo(localLocation); + InputChannel newChannel; + if (isLocal) { + newChannel = unknownChannel.toLocalInputChannel(); + } else { + RemoteInputChannel remoteInputChannel = + unknownChannel.toRemoteInputChannel( + shuffleDescriptor.getConnectionId()); + remoteInputChannel.setup(); + newChannel = remoteInputChannel; + } + LOG.debug( + "{}: Updated unknown input channel to {}.", owningTaskName, newChannel); - inputChannels.put(partitionId, newChannel); - channels[current.getChannelIndex()] = newChannel; + inputChannels.put(subpartitionInfo, newChannel); + channels[current.getChannelIndex()] = newChannel; - if (requestedPartitionsFlag) { - newChannel.requestSubpartition(); - } + if (requestedPartitionsFlag) { + newChannel.requestSubpartition(); + } - for (TaskEvent event : pendingEvents) { - newChannel.sendTaskEvent(event); - } + for (TaskEvent event : pendingEvents) { + newChannel.sendTaskEvent(event); + } - if (--numberOfUninitializedChannels == 0) { - pendingEvents.clear(); + if (--numberOfUninitializedChannels == 0) { + pendingEvents.clear(); + } } } } } /** Retriggers a partition request. */ - public void retriggerPartitionRequest(IntermediateResultPartitionID partitionId) - throws IOException { + public void retriggerPartitionRequest( + IntermediateResultPartitionID partitionId, int subpartitionIndex) throws IOException { synchronized (requestLock) { if (!closeFuture.isDone()) { - final InputChannel ch = inputChannels.get(partitionId); + final InputChannel ch = + inputChannels.get(new SubpartitionInfo(partitionId, subpartitionIndex)); checkNotNull(ch, "Unknown input channel with ID " + partitionId); @@ -950,7 +973,7 @@ void notifyPriorityEventForce(InputChannel inputChannel) { queueChannel(checkNotNull(inputChannel), null, true); } - void triggerPartitionStateCheck(ResultPartitionID partitionId) { + void triggerPartitionStateCheck(ResultPartitionID partitionId, int subpartitionIndex) { partitionProducerStateProvider.requestPartitionProducerState( consumedResultId, partitionId, @@ -960,7 +983,8 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) { .isProducerReadyOrAbortConsumption(responseHandle); if (isProducingState) { try { - retriggerPartitionRequest(partitionId.getPartitionId()); + retriggerPartitionRequest( + partitionId.getPartitionId(), subpartitionIndex); } catch (IOException t) { responseHandle.failConsumption(t); } @@ -1061,7 +1085,36 @@ private Optional getChannel(boolean blocking) throws InterruptedEx // ------------------------------------------------------------------------ - public Map getInputChannels() { + public Map getInputChannels() { return inputChannels; } + + static class SubpartitionInfo { + private final IntermediateResultPartitionID partitionID; + private final int subpartitionIndex; + + SubpartitionInfo(IntermediateResultPartitionID partitionID, int subpartitionIndex) { + this.partitionID = checkNotNull(partitionID); + checkArgument(subpartitionIndex >= 0); + this.subpartitionIndex = subpartitionIndex; + } + + @Override + public int hashCode() { + return partitionID.hashCode() ^ subpartitionIndex; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } else if (obj != null && obj.getClass() == getClass()) { + SubpartitionInfo that = (SubpartitionInfo) obj; + return that.partitionID.equals(this.partitionID) + && that.subpartitionIndex == this.subpartitionIndex; + } else { + return false; + } + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index 5cc43898bc44a..d862af37be066 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventPublisher; @@ -42,6 +43,7 @@ import org.apache.flink.runtime.throughput.BufferDebloatConfiguration; import org.apache.flink.runtime.throughput.BufferDebloater; import org.apache.flink.runtime.throughput.ThroughputCalculator; +import org.apache.flink.util.MathUtils; import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.function.SupplierWithException; @@ -126,13 +128,17 @@ public SingleInputGate create( final String owningTaskName = owner.getOwnerName(); final MetricGroup networkInputGroup = owner.getInputGroup(); + + SubpartitionIndexRange subpartitionIndexRange = igdd.getConsumedSubpartitionIndexRange(); SingleInputGate inputGate = new SingleInputGate( owningTaskName, gateIndex, igdd.getConsumedResultId(), igdd.getConsumedPartitionType(), - igdd.getShuffleDescriptors().length, + subpartitionIndexRange, + calculateNumChannels( + igdd.getShuffleDescriptors().length, subpartitionIndexRange), partitionProducerStateProvider, bufferPoolFactory, bufferDecompressor, @@ -144,8 +150,7 @@ public SingleInputGate create( InputChannelMetrics metrics = new InputChannelMetrics(networkInputGroup, owner.getParentGroup()); - createInputChannels( - owningTaskName, igdd, inputGate, igdd.getConsumedSubpartitionIndex(), metrics); + createInputChannels(owningTaskName, igdd, inputGate, subpartitionIndexRange, metrics); return inputGate; } @@ -173,26 +178,35 @@ private void createInputChannels( String owningTaskName, InputGateDeploymentDescriptor inputGateDeploymentDescriptor, SingleInputGate inputGate, - int consumedSubpartitionIndex, + SubpartitionIndexRange subpartitionIndexRange, InputChannelMetrics metrics) { ShuffleDescriptor[] shuffleDescriptors = inputGateDeploymentDescriptor.getShuffleDescriptors(); - // Create the input channels. There is one input channel for each consumed partition. - InputChannel[] inputChannels = new InputChannel[shuffleDescriptors.length]; + // Create the input channels. There is one input channel for each consumed subpartition. + InputChannel[] inputChannels = + new InputChannel + [calculateNumChannels(shuffleDescriptors.length, subpartitionIndexRange)]; ChannelStatistics channelStatistics = new ChannelStatistics(); - for (int i = 0; i < inputChannels.length; i++) { - inputChannels[i] = - createInputChannel( - inputGate, - i, - shuffleDescriptors[i], - consumedSubpartitionIndex, - channelStatistics, - metrics); + int channelIdx = 0; + for (int i = 0; i < shuffleDescriptors.length; ++i) { + for (int subpartitionIndex = subpartitionIndexRange.getStartIndex(); + subpartitionIndex <= subpartitionIndexRange.getEndIndex(); + ++subpartitionIndex) { + inputChannels[channelIdx] = + createInputChannel( + inputGate, + channelIdx, + shuffleDescriptors[i], + subpartitionIndex, + channelStatistics, + metrics); + channelIdx++; + } } + inputGate.setInputChannels(inputChannels); LOG.debug( @@ -237,6 +251,12 @@ private InputChannel createInputChannel( metrics)); } + private static int calculateNumChannels( + int numShuffleDescriptors, SubpartitionIndexRange subpartitionIndexRange) { + return MathUtils.checkedDownCast( + ((long) numShuffleDescriptors) * subpartitionIndexRange.size()); + } + @VisibleForTesting protected InputChannel createKnownInputChannel( SingleInputGate inputGate, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java index f53d2d798ff62..276a2e2828419 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java @@ -89,7 +89,9 @@ public void testRetriggerPartitionRequest() throws Exception { assertEquals(numExclusiveBuffers, ((PartitionRequest) readFromOutbound).credit); // retrigger subpartition request, e.g. due to failures - inputGate.retriggerPartitionRequest(inputChannel.getPartitionId().getPartitionId()); + inputGate.retriggerPartitionRequest( + inputChannel.getPartitionId().getPartitionId(), + inputChannel.getConsumedSubpartitionIndex()); runAllScheduledPendingTasks(channel, deadline); readFromOutbound = channel.readOutbound(); @@ -100,7 +102,9 @@ public void testRetriggerPartitionRequest() throws Exception { assertEquals(numExclusiveBuffers, ((PartitionRequest) readFromOutbound).credit); // retrigger subpartition request once again, e.g. due to failures - inputGate.retriggerPartitionRequest(inputChannel.getPartitionId().getPartitionId()); + inputGate.retriggerPartitionRequest( + inputChannel.getPartitionId().getPartitionId(), + inputChannel.getConsumedSubpartitionIndex()); runAllScheduledPendingTasks(channel, deadline); readFromOutbound = channel.readOutbound(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java index b5e77f9d8a37c..8af69be7e64c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; @@ -304,7 +305,10 @@ public void testFairConsumptionRemoteChannels() throws Exception { private SingleInputGate createFairnessVerifyingInputGate(int numberOfChannels) { return new FairnessVerifyingInputGate( - "Test Task Name", new IntermediateDataSetID(), numberOfChannels); + "Test Task Name", + new IntermediateDataSetID(), + new SubpartitionIndexRange(0, 0), + numberOfChannels); } private void fillRandom( @@ -362,6 +366,7 @@ private static class FairnessVerifyingInputGate extends SingleInputGate { public FairnessVerifyingInputGate( String owningTaskName, IntermediateDataSetID consumedResultId, + SubpartitionIndexRange subpartitionIndexRange, int numberOfInputChannels) { super( @@ -369,6 +374,7 @@ public FairnessVerifyingInputGate( 0, consumedResultId, ResultPartitionType.PIPELINED, + subpartitionIndexRange, numberOfInputChannels, SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER, STUB_BUFFER_POOL_FACTORY, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java index 524cf807c5fca..c7e1faa82c37f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemorySegmentProvider; import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; @@ -55,6 +56,8 @@ public class SingleInputGateBuilder { private ResultPartitionType partitionType = ResultPartitionType.PIPELINED; + private SubpartitionIndexRange subpartitionIndexRange = new SubpartitionIndexRange(0, 0); + private int gateIndex = 0; private int numberOfChannels = 1; @@ -89,6 +92,12 @@ public SingleInputGateBuilder setResultPartitionType(ResultPartitionType partiti return this; } + public SingleInputGateBuilder setSubpartitionIndexRange( + SubpartitionIndexRange subpartitionIndexRange) { + this.subpartitionIndexRange = subpartitionIndexRange; + return this; + } + public SingleInputGateBuilder setSingleInputGateIndex(int gateIndex) { this.gateIndex = gateIndex; return this; @@ -154,6 +163,7 @@ public SingleInputGate build() { gateIndex, intermediateDataSetID, partitionType, + subpartitionIndexRange, numberOfChannels, partitionProducerStateProvider, bufferPoolFactory, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index abf1d58abdd58..1dd28ece36ea1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -26,9 +26,12 @@ import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.PullingAsyncDataInput; +import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; @@ -46,6 +49,8 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; import org.apache.flink.runtime.io.network.partition.BufferWritingResultPartition; +import org.apache.flink.runtime.io.network.partition.ChannelStateHolder; +import org.apache.flink.runtime.io.network.partition.InputChannelTestUtils; import org.apache.flink.runtime.io.network.partition.NoOpResultSubpartitionView; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.ResultPartition; @@ -54,6 +59,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; +import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.SubpartitionInfo; import org.apache.flink.runtime.io.network.util.TestTaskEvent; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -62,6 +68,7 @@ import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; +import org.apache.flink.util.CompressedSerializedValue; import org.apache.flink.shaded.guava30.com.google.common.io.Closer; @@ -73,6 +80,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; @@ -83,7 +91,6 @@ import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createLocalInputChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createRemoteInputChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createResultSubpartitionView; -import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; import static org.apache.flink.runtime.io.network.partition.InputGateFairnessTest.setupInputGate; import static org.apache.flink.runtime.io.network.util.TestBufferFactory.createBuffer; import static org.apache.flink.runtime.state.CheckpointStorageLocationReference.getDefault; @@ -574,25 +581,6 @@ public void testRequestBackoffConfiguration() throws Exception { new IntermediateResultPartitionID() }; - ResourceID localLocation = ResourceID.generate(); - ShuffleDescriptor[] channelDescs = - new ShuffleDescriptor[] { - // Local - createRemoteWithIdAndLocation(partitionIds[0], localLocation), - // Remote - createRemoteWithIdAndLocation(partitionIds[1], ResourceID.generate()), - // Unknown - new UnknownShuffleDescriptor( - new ResultPartitionID(partitionIds[2], new ExecutionAttemptID())) - }; - - InputGateDeploymentDescriptor gateDesc = - new InputGateDeploymentDescriptor( - new IntermediateDataSetID(), - ResultPartitionType.PIPELINED, - 0, - channelDescs); - int initialBackoff = 137; int maxBackoff = 1001; @@ -602,22 +590,8 @@ public void testRequestBackoffConfiguration() throws Exception { .setPartitionRequestMaxBackoff(maxBackoff) .build(); - final TaskMetricGroup taskMetricGroup = - UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(); SingleInputGate gate = - new SingleInputGateFactory( - localLocation, - netEnv.getConfiguration(), - netEnv.getConnectionManager(), - netEnv.getResultPartitionManager(), - new TaskEventDispatcher(), - netEnv.getNetworkBufferPool()) - .create( - netEnv.createShuffleIOOwnerContext( - "TestTask", taskMetricGroup.executionId(), taskMetricGroup), - 0, - gateDesc, - SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER); + createSingleInputGate(partitionIds, ResultPartitionType.PIPELINED, netEnv); gate.setChannelStateWriter(ChannelStateWriter.NO_OP); gate.finishReadRecoveredState(); @@ -630,9 +604,9 @@ public void testRequestBackoffConfiguration() throws Exception { closer.register(netEnv::close); closer.register(gate::close); - assertEquals(gateDesc.getConsumedPartitionType(), gate.getConsumedPartitionType()); + assertEquals(ResultPartitionType.PIPELINED, gate.getConsumedPartitionType()); - Map channelMap = gate.getInputChannels(); + Map channelMap = gate.getInputChannels(); assertEquals(3, channelMap.size()); channelMap @@ -645,13 +619,13 @@ public void testRequestBackoffConfiguration() throws Exception { throw new RuntimeException(e); } }); - InputChannel localChannel = channelMap.get(partitionIds[0]); + InputChannel localChannel = channelMap.get(createSubpartitionInfo(partitionIds[0])); assertEquals(LocalInputChannel.class, localChannel.getClass()); - InputChannel remoteChannel = channelMap.get(partitionIds[1]); + InputChannel remoteChannel = channelMap.get(createSubpartitionInfo(partitionIds[1])); assertEquals(RemoteInputChannel.class, remoteChannel.getClass()); - InputChannel unknownChannel = channelMap.get(partitionIds[2]); + InputChannel unknownChannel = channelMap.get(createSubpartitionInfo(partitionIds[2])); assertEquals(UnknownInputChannel.class, unknownChannel.getClass()); InputChannel[] channels = @@ -749,7 +723,11 @@ public void testRequestBuffersWithUnknownInputChannel() throws Exception { RemoteInputChannel remote = (RemoteInputChannel) - inputGate.getInputChannels().get(resultPartitionId.getPartitionId()); + inputGate + .getInputChannels() + .get( + createSubpartitionInfo( + resultPartitionId.getPartitionId())); // only the exclusive buffers should be assigned/available now assertEquals(buffersPerChannel, remote.getNumberOfAvailableBuffers()); @@ -806,10 +784,14 @@ public void testUpdateUnknownInputChannel() throws Exception { inputGate.setup(); assertThat( - inputGate.getInputChannels().get(remoteResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(remoteResultPartitionId.getPartitionId())), is(instanceOf((UnknownInputChannel.class)))); assertThat( - inputGate.getInputChannels().get(localResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(localResultPartitionId.getPartitionId())), is(instanceOf((UnknownInputChannel.class)))); ResourceID localLocation = ResourceID.generate(); @@ -821,10 +803,14 @@ public void testUpdateUnknownInputChannel() throws Exception { remoteResultPartitionId.getPartitionId(), ResourceID.generate())); assertThat( - inputGate.getInputChannels().get(remoteResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(remoteResultPartitionId.getPartitionId())), is(instanceOf((RemoteInputChannel.class)))); assertThat( - inputGate.getInputChannels().get(localResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(localResultPartitionId.getPartitionId())), is(instanceOf((UnknownInputChannel.class)))); // Trigger updates to local input channel from unknown input channel @@ -834,14 +820,109 @@ public void testUpdateUnknownInputChannel() throws Exception { localResultPartitionId.getPartitionId(), localLocation)); assertThat( - inputGate.getInputChannels().get(remoteResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(remoteResultPartitionId.getPartitionId())), is(instanceOf((RemoteInputChannel.class)))); assertThat( - inputGate.getInputChannels().get(localResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(localResultPartitionId.getPartitionId())), is(instanceOf((LocalInputChannel.class)))); } } + @Test + public void testSingleInputGateWithSubpartitionIndexRange() + throws IOException, InterruptedException { + + IntermediateResultPartitionID[] partitionIds = + new IntermediateResultPartitionID[] { + new IntermediateResultPartitionID(), + new IntermediateResultPartitionID(), + new IntermediateResultPartitionID() + }; + + SubpartitionIndexRange subpartitionIndexRange = new SubpartitionIndexRange(0, 1); + final NettyShuffleEnvironment netEnv = new NettyShuffleEnvironmentBuilder().build(); + + ResourceID localLocation = ResourceID.generate(); + + SingleInputGate gate = + createSingleInputGate( + partitionIds, + ResultPartitionType.BLOCKING, + subpartitionIndexRange, + netEnv, + localLocation, + new TestingConnectionManager(), + new TestingResultPartitionManager(new NoOpResultSubpartitionView())); + + for (InputChannel channel : gate.getInputChannels().values()) { + if (channel instanceof ChannelStateHolder) { + ((ChannelStateHolder) channel).setChannelStateWriter(ChannelStateWriter.NO_OP); + } + } + + SubpartitionInfo info1 = createSubpartitionInfo(partitionIds[0], 0); + SubpartitionInfo info2 = createSubpartitionInfo(partitionIds[0], 1); + SubpartitionInfo info3 = createSubpartitionInfo(partitionIds[1], 0); + SubpartitionInfo info4 = createSubpartitionInfo(partitionIds[1], 1); + SubpartitionInfo info5 = createSubpartitionInfo(partitionIds[2], 0); + SubpartitionInfo info6 = createSubpartitionInfo(partitionIds[2], 1); + + assertThat(gate.getInputChannels().size(), is(6)); + assertThat(gate.getInputChannels().get(info1).getConsumedSubpartitionIndex(), is(0)); + assertThat(gate.getInputChannels().get(info2).getConsumedSubpartitionIndex(), is(1)); + assertThat(gate.getInputChannels().get(info3).getConsumedSubpartitionIndex(), is(0)); + assertThat(gate.getInputChannels().get(info4).getConsumedSubpartitionIndex(), is(1)); + assertThat(gate.getInputChannels().get(info5).getConsumedSubpartitionIndex(), is(0)); + assertThat(gate.getInputChannels().get(info6).getConsumedSubpartitionIndex(), is(1)); + + assertChannelsType(gate, LocalRecoveredInputChannel.class, Arrays.asList(info1, info2)); + assertChannelsType(gate, RemoteRecoveredInputChannel.class, Arrays.asList(info3, info4)); + assertChannelsType(gate, UnknownInputChannel.class, Arrays.asList(info5, info6)); + + // test setup + gate.setup(); + assertNotNull(gate.getBufferPool()); + assertEquals(1, gate.getBufferPool().getNumberOfRequiredMemorySegments()); + + gate.finishReadRecoveredState(); + while (!gate.getStateConsumedFuture().isDone()) { + gate.pollNext(); + } + + // test request partitions + gate.requestPartitions(); + gate.pollNext(); + assertChannelsType(gate, LocalInputChannel.class, Arrays.asList(info1, info2)); + assertChannelsType(gate, RemoteInputChannel.class, Arrays.asList(info3, info4)); + assertChannelsType(gate, UnknownInputChannel.class, Arrays.asList(info5, info6)); + for (InputChannel inputChannel : gate.getInputChannels().values()) { + if (inputChannel instanceof RemoteInputChannel) { + assertNotNull(((RemoteInputChannel) inputChannel).getPartitionRequestClient()); + assertEquals(2, ((RemoteInputChannel) inputChannel).getInitialCredit()); + } else if (inputChannel instanceof LocalInputChannel) { + assertNotNull(((LocalInputChannel) inputChannel).getSubpartitionView()); + } + } + + // test update channels + gate.updateInputChannel( + localLocation, createRemoteWithIdAndLocation(partitionIds[2], localLocation)); + assertChannelsType(gate, LocalInputChannel.class, Arrays.asList(info1, info2)); + assertChannelsType(gate, RemoteInputChannel.class, Arrays.asList(info3, info4)); + assertChannelsType(gate, LocalInputChannel.class, Arrays.asList(info5, info6)); + } + + private void assertChannelsType( + SingleInputGate gate, Class clazz, List infos) { + for (SubpartitionInfo subpartitionInfo : infos) { + assertThat(gate.getInputChannels().get(subpartitionInfo), instanceOf(clazz)); + } + } + @Test public void testQueuedBuffers() throws Exception { final NettyShuffleEnvironment network = createNettyShuffleEnvironment(); @@ -898,7 +979,7 @@ public void testQueuedBuffers() throws Exception { */ @Test public void testPartitionNotFoundExceptionWhileGetNextBuffer() throws Exception { - final SingleInputGate inputGate = createSingleInputGate(1); + final SingleInputGate inputGate = InputChannelTestUtils.createSingleInputGate(1); final LocalInputChannel localChannel = createLocalInputChannel(inputGate, new ResultPartitionManager()); final ResultPartitionID partitionId = localChannel.getPartitionId(); @@ -916,7 +997,7 @@ public void testPartitionNotFoundExceptionWhileGetNextBuffer() throws Exception @Test public void testAnnounceBufferSize() throws Exception { - final SingleInputGate inputGate = createSingleInputGate(2); + final SingleInputGate inputGate = InputChannelTestUtils.createSingleInputGate(2); final LocalInputChannel localChannel = createLocalInputChannel( inputGate, @@ -1058,6 +1139,81 @@ public void testBufferInUseCount() throws Exception { // --------------------------------------------------------------------------------------------- + private static SubpartitionInfo createSubpartitionInfo( + IntermediateResultPartitionID partitionId) { + return createSubpartitionInfo(partitionId, 0); + } + + private static SubpartitionInfo createSubpartitionInfo( + IntermediateResultPartitionID partitionId, int subpartitionIndex) { + return new SubpartitionInfo(partitionId, subpartitionIndex); + } + + static SingleInputGate createSingleInputGate( + IntermediateResultPartitionID[] partitionIds, + ResultPartitionType resultPartitionType, + NettyShuffleEnvironment netEnv) + throws IOException { + return createSingleInputGate( + partitionIds, + resultPartitionType, + new SubpartitionIndexRange(0, 0), + netEnv, + ResourceID.generate(), + null, + null); + } + + static SingleInputGate createSingleInputGate( + IntermediateResultPartitionID[] partitionIds, + ResultPartitionType resultPartitionType, + SubpartitionIndexRange subpartitionIndexRange, + NettyShuffleEnvironment netEnv, + ResourceID localLocation, + ConnectionManager connectionManager, + ResultPartitionManager resultPartitionManager) + throws IOException { + + ShuffleDescriptor[] channelDescs = + new ShuffleDescriptor[] { + // Local + createRemoteWithIdAndLocation(partitionIds[0], localLocation), + // Remote + createRemoteWithIdAndLocation(partitionIds[1], ResourceID.generate()), + // Unknown + new UnknownShuffleDescriptor( + new ResultPartitionID(partitionIds[2], new ExecutionAttemptID())) + }; + + InputGateDeploymentDescriptor gateDesc = + new InputGateDeploymentDescriptor( + new IntermediateDataSetID(), + resultPartitionType, + subpartitionIndexRange, + new TaskDeploymentDescriptor.NonOffloaded<>( + CompressedSerializedValue.fromObject(channelDescs))); + + final TaskMetricGroup taskMetricGroup = + UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(); + return new SingleInputGateFactory( + localLocation, + netEnv.getConfiguration(), + connectionManager != null + ? connectionManager + : netEnv.getConnectionManager(), + resultPartitionManager != null + ? resultPartitionManager + : netEnv.getResultPartitionManager(), + new TaskEventDispatcher(), + netEnv.getNetworkBufferPool()) + .create( + netEnv.createShuffleIOOwnerContext( + "TestTask", taskMetricGroup.executionId(), taskMetricGroup), + 0, + gateDesc, + SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER); + } + private static Map createInputGateWithLocalChannels( NettyShuffleEnvironment network, int numberOfGates, From 9df1980642300b4daefd82c5b61eda18c9c701cb Mon Sep 17 00:00:00 2001 From: Ahmed Hamdy Date: Mon, 17 Jan 2022 13:35:16 +0000 Subject: [PATCH 07/49] [FLINK-25810][connector/kinesis] Adding Kinesis data streams SQL client uber jar and end to end test. This closes #18468. --- .../pom.xml | 118 ++++++++ .../src/main/resources/META-INF/NOTICE | 49 ++++ flink-connectors/pom.xml | 1 + .../pom.xml | 117 ++++++++ .../test/KinesisDataStreamsTableApiIT.java | 265 ++++++++++++++++++ .../src/test/resources/log4j2-test.properties | 28 ++ .../src/test/resources/send-orders.sql | 36 +++ flink-end-to-end-tests/pom.xml | 1 + 8 files changed, 615 insertions(+) create mode 100644 flink-connectors/flink-sql-connector-aws-kinesis-data-streams/pom.xml create mode 100644 flink-connectors/flink-sql-connector-aws-kinesis-data-streams/src/main/resources/META-INF/NOTICE create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/pom.xml create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/table/test/KinesisDataStreamsTableApiIT.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/log4j2-test.properties create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/send-orders.sql diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/pom.xml b/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/pom.xml new file mode 100644 index 0000000000000..0e146c91df14a --- /dev/null +++ b/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/pom.xml @@ -0,0 +1,118 @@ + + + + + + flink-connectors + org.apache.flink + 1.15-SNAPSHOT + .. + + 4.0.0 + + flink-sql-connector-aws-kinesis-data-streams + Flink : Connectors : SQL : AWS Kinesis Data Streams + + + + org.apache.flink + flink-connector-aws-kinesis-data-streams + ${project.version} + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-table-common + org.apache.flink:flink-streaming-java + org.apache.flink:flink-connector-base + org.apache.flink:flink-connector-aws-base + org.apache.flink:flink-connector-aws-kinesis-data-streams + software.amazon.awssdk:* + org.reactivestreams:* + com.typesafe.netty:* + org.apache.httpcomponents:* + io.netty:* + + + + + software.amazon + org.apache.flink.kinesis-streams.shaded.software.amazon + + + org.reactivestreams + org.apache.flink.kinesis-streams.shaded.org.reactivestreams + + + com.typesafe.netty + org.apache.flink.kinesis-streams.shaded.com.typesafe.netty + + + org.apache.http + org.apache.flink.kinesis-streams.shaded.org.apache.http + + + io.netty + org.apache.flink.kinesis-streams.shaded.io.netty + + + + + software.amazon.awssdk:* + + META-INF/services/** + + + + org.apache.flink:flink-connector-aws-kinesis-data-streams:* + + profile + + + + org.apache.flink:flink-connector-aws-base:* + + profile + + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..55ec4cac0f12c --- /dev/null +++ b/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/src/main/resources/META-INF/NOTICE @@ -0,0 +1,49 @@ +flink-sql-connector-aws-kinesis-data-streams + +Copyright 2014-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- software.amazon.awssdk:kinesis:2.17.52 +- software.amazon.awssdk:aws-cbor-protocol:2.17.52 +- software.amazon.awssdk:aws-json-protocol:2.17.52 +- software.amazon.awssdk:protocol-core:2.17.52 +- software.amazon.awssdk:profiles:2.17.52 +- software.amazon.awssdk:sdk-core:2.17.52 +- software.amazon.awssdk:auth:2.17.52 +- software.amazon.eventstream:eventstream:1.0.1 +- software.amazon.awssdk:http-client-spi:2.17.52 +- software.amazon.awssdk:regions:2.17.52 +- software.amazon.awssdk:annotations:2.17.52 +- software.amazon.awssdk:utils:2.17.52 +- software.amazon.awssdk:aws-core:2.17.52 +- software.amazon.awssdk:metrics-spi:2.17.52 +- software.amazon.awssdk:apache-client:2.17.52 +- software.amazon.awssdk:netty-nio-client:2.17.52 +- software.amazon.awssdk:sts:2.17.52 +- software.amazon.awssdk:aws-query-protocol:2.17.52 +- software.amazon.awssdk:json-utils:2.17.52 +- software.amazon.awssdk:third-party-jackson-core:2.17.52 +- software.amazon.awssdk:third-party-jackson-dataformat-cbor:2.17.52 +- io.netty:netty-codec-http:4.1.68.Final +- io.netty:netty-codec-http2:4.1.68.Final +- io.netty:netty-codec:4.1.68.Final +- io.netty:netty-transport:4.1.68.Final +- io.netty:netty-resolver:4.1.68.Final +- io.netty:netty-common:4.1.68.Final +- io.netty:netty-buffer:4.1.68.Final +- io.netty:netty-handler:4.1.68.Final +- io.netty:netty-transport-native-epoll:linux-x86_64:4.1.68.Final +- io.netty:netty-transport-native-unix-common:4.1.68.Final +- com.typesafe.netty:netty-reactive-streams-http:2.0.5 +- com.typesafe.netty:netty-reactive-streams:2.0.5 +- org.apache.httpcomponents:httpclient:4.5.13 +- org.apache.httpcomponents:httpcore:4.4.14 + + +This project bundles the following dependencies under the Creative Commons Zero license (https://creativecommons.org/publicdomain/zero/1.0/). + +- org.reactivestreams:reactive-streams:1.0.3 diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index 4e65f8d038791..74954126e0036 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -104,6 +104,7 @@ under the License. flink-sql-connector-hive-2.3.6 flink-sql-connector-hive-3.1.2 flink-sql-connector-kafka + flink-sql-connector-aws-kinesis-data-streams flink-sql-connector-kinesis flink-sql-connector-rabbitmq diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/pom.xml new file mode 100644 index 0000000000000..1818c62beb9fb --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/pom.xml @@ -0,0 +1,117 @@ + + + + + + flink-end-to-end-tests + org.apache.flink + 1.15-SNAPSHOT + .. + + 4.0.0 + + flink-end-to-end-tests-kinesis-streams + Flink : E2E Tests : Kinesis SQL tests + jar + + + + org.apache.flink + flink-streaming-java + ${project.version} + test + + + + org.apache.flink + flink-end-to-end-tests-common + ${project.version} + test + + + + org.apache.flink + flink-connector-aws-kinesis-data-streams + ${project.version} + test + test-jar + + + com.typesafe.netty + netty-reactive-streams-http + + + + + com.fasterxml.jackson.core + jackson-databind + test + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy + pre-integration-test + + copy + + + + + + + org.apache.flink + flink-sql-connector-aws-kinesis-data-streams + ${project.version} + sql-kinesis-streams.jar + jar + ${project.build.directory}/dependencies + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.22.1 + + + + + true + true + true + true + + + + + + + + diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/table/test/KinesisDataStreamsTableApiIT.java b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/table/test/KinesisDataStreamsTableApiIT.java new file mode 100644 index 0000000000000..998c5438b0e2a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/table/test/KinesisDataStreamsTableApiIT.java @@ -0,0 +1,265 @@ +/* + * 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.flink.connector.kinesis.table.test; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.connectors.kinesis.testutils.KinesaliteContainer; +import org.apache.flink.tests.util.TestUtils; +import org.apache.flink.tests.util.flink.SQLJobSubmission; +import org.apache.flink.tests.util.flink.container.FlinkContainers; +import org.apache.flink.util.DockerImageVersions; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.rnorth.ducttape.ratelimits.RateLimiter; +import org.rnorth.ducttape.ratelimits.RateLimiterBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Network; +import org.testcontainers.utility.DockerImageName; +import software.amazon.awssdk.core.SdkSystemSetting; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.kinesis.model.Record; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.services.kinesis.model.StreamStatus; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +import static java.util.concurrent.TimeUnit.SECONDS; + +/** End-to-end test for Kinesis DataStream Table API Sink using Kinesalite. */ +public class KinesisDataStreamsTableApiIT { + + private static final Logger LOGGER = + LoggerFactory.getLogger(KinesisDataStreamsTableApiIT.class); + + private static final String ORDERS_STREAM = "orders"; + private static final String INTER_CONTAINER_KINESALITE_ALIAS = "kinesalite"; + private static final String DEFAULT_FIRST_SHARD_NAME = "shardId-000000000000"; + + private final Path sqlConnectorKinesisJar = TestUtils.getResource(".*kinesis-streams.jar"); + private static final Network network = Network.newNetwork(); + + @ClassRule public static final Timeout TIMEOUT = new Timeout(10, TimeUnit.MINUTES); + + @ClassRule + public static final KinesaliteContainer KINESALITE = + new KinesaliteContainer(DockerImageName.parse(DockerImageVersions.KINESALITE)) + .withNetwork(network) + .withNetworkAliases(INTER_CONTAINER_KINESALITE_ALIAS); + + public static final FlinkContainers FLINK = + FlinkContainers.builder() + .setEnvironmentVariable("AWS_CBOR_DISABLE", "1") + .setEnvironmentVariable( + "FLINK_ENV_JAVA_OPTS", + "-Dorg.apache.flink.kinesis-streams.shaded.com.amazonaws.sdk.disableCertChecking -Daws.cborEnabled=false") + .setNetwork(network) + .setLogger(LOGGER) + .dependsOn(KINESALITE) + .build(); + + @BeforeClass + public static void setupFlink() throws Exception { + FLINK.start(); + } + + @AfterClass + public static void stopFlink() { + FLINK.stop(); + } + + @Before + public void setUp() throws Exception { + prepareStream(ORDERS_STREAM); + System.setProperty(SdkSystemSetting.CBOR_ENABLED.property(), "false"); + } + + @After + public void teardown() { + System.clearProperty(SdkSystemSetting.CBOR_ENABLED.property()); + } + + @Test + public void testTableApiSourceAndSink() throws Exception { + executeSqlStatements(readSqlFile("send-orders.sql")); + List expected = + ImmutableList.of( + new Order("A", 10), + new Order("B", 12), + new Order("C", 14), + new Order("D", 16), + new Order("E", 18)); + // result order is not guaranteed + List result = readAllOrdersFromKinesis(); + Assertions.assertThat(result).containsAll(expected); + } + + private void prepareStream(String streamName) throws Exception { + final RateLimiter rateLimiter = + RateLimiterBuilder.newBuilder() + .withRate(1, SECONDS) + .withConstantThroughput() + .build(); + + KinesisAsyncClient kinesisClient = KINESALITE.getHostClient(); + kinesisClient + .createStream( + CreateStreamRequest.builder().streamName(streamName).shardCount(1).build()) + .get(); + + Deadline deadline = Deadline.fromNow(Duration.ofMinutes(1)); + while (!rateLimiter.getWhenReady(() -> streamExists(streamName))) { + if (deadline.isOverdue()) { + throw new RuntimeException("Failed to create stream within time"); + } + } + } + + private boolean streamExists(final String streamName) { + try { + return KINESALITE + .getHostClient() + .describeStream( + DescribeStreamRequest.builder().streamName(streamName).build()) + .get() + .streamDescription() + .streamStatus() + == StreamStatus.ACTIVE; + } catch (Exception e) { + return false; + } + } + + private List readAllOrdersFromKinesis() throws Exception { + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(10)); + List orders; + do { + orders = + readMessagesFromStream( + recordBytes -> fromJson(new String(recordBytes), Order.class)); + + } while (deadline.hasTimeLeft() && orders.size() < 5); + + return orders; + } + + private void executeSqlStatements(final List sqlLines) throws Exception { + FLINK.submitSQLJob( + new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines) + .addJars(sqlConnectorKinesisJar) + .build()); + } + + private List readSqlFile(final String resourceName) throws Exception { + return Files.readAllLines(Paths.get(getClass().getResource("/" + resourceName).toURI())); + } + + private T fromJson(final String json, final Class type) { + try { + return new ObjectMapper().readValue(json, type); + } catch (JsonProcessingException e) { + throw new RuntimeException("Test Failure.", e); + } + } + + private List readMessagesFromStream(Function deserialiser) throws Exception { + String shardIterator = + KINESALITE + .getHostClient() + .getShardIterator( + GetShardIteratorRequest.builder() + .shardId(DEFAULT_FIRST_SHARD_NAME) + .shardIteratorType(ShardIteratorType.TRIM_HORIZON) + .streamName(KinesisDataStreamsTableApiIT.ORDERS_STREAM) + .build()) + .get() + .shardIterator(); + + List records = + KINESALITE + .getHostClient() + .getRecords( + GetRecordsRequest.builder().shardIterator(shardIterator).build()) + .get() + .records(); + List messages = new ArrayList<>(); + records.forEach(record -> messages.add(deserialiser.apply(record.data().asByteArray()))); + return messages; + } + + /** POJO class for orders used by e2e test. */ + public static class Order { + private final String code; + private final int quantity; + + public Order( + @JsonProperty("code") final String code, @JsonProperty("quantity") int quantity) { + this.code = code; + this.quantity = quantity; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + Order order = (Order) o; + return quantity == order.quantity && Objects.equals(code, order.code); + } + + @Override + public int hashCode() { + return Objects.hash(code, quantity); + } + + @Override + public String toString() { + return String.format("Order{code: %s, quantity: %d}", code, quantity); + } + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/log4j2-test.properties b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/log4j2-test.properties new file mode 100644 index 0000000000000..e463a0e1a088c --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = INFO +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/send-orders.sql b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/send-orders.sql new file mode 100644 index 0000000000000..29f2c885ed3cc --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/send-orders.sql @@ -0,0 +1,36 @@ +--/* +-- * 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. +-- */ + +CREATE TABLE orders ( + `code` STRING, + `quantity` BIGINT +) WITH ( + 'connector' = 'kinesis', + 'stream' = 'orders', + 'aws.region' = 'us-east-1', + 'aws.endpoint' = 'https://kinesalite:4567', + 'aws.credentials.provider' = 'BASIC', + 'aws.credentials.basic.accesskeyid' = 'access key', + 'aws.credentials.basic.secretkey' ='secret key', + 'aws.trust.all.certificates' = 'true', + 'sink.http-client.protocol.version' = 'HTTP1_1', + 'sink.batch.max-size' = '1', + 'format' = 'json' +); + +INSERT INTO orders VALUES ('A', 10),('B', 12),('C', 14),('D', 16),('E', 18); diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index aaf3b94151957..5ad33b8913118 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -80,6 +80,7 @@ under the License. flink-glue-schema-registry-avro-test flink-glue-schema-registry-json-test flink-end-to-end-tests-scala + flink-end-to-end-tests-kinesis-streams From c31452bab1fb3b9a02ab616e4c5fe5e87346dfb4 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 26 Nov 2021 02:07:04 +0100 Subject: [PATCH 08/49] [FLINK-24703][connectors][format] Adds CSV File System decoding format based on file-connector-files StreamFormat. 'flink-connector-file' introduced new interfaces for interacting with the file system. This commit contributes to the goal of migrating all formats used for writing files to this API, in particular, StreamFormat used by FileSource. It also introduces a generic Converter interface to map types between the native format types representation and their corresponding types expected by Flink (in particular Table/SQL API). Co-authored-by: Arvid Heise --- .../apache/flink/formats/csv/CsvCommons.java | 117 ++++++ .../formats/csv/CsvFileFormatFactory.java | 163 ++++++++ .../csv/CsvFileSystemFormatFactory.java | 282 -------------- .../flink/formats/csv/CsvFormatFactory.java | 78 +--- .../flink/formats/csv/CsvReaderFormat.java | 212 +++++++++++ .../formats/csv/CsvToRowDataConverters.java | 10 +- .../org.apache.flink.table.factories.Factory | 2 +- .../formats/csv/CsvFilesystemBatchITCase.java | 17 +- .../formats/csv/CsvFormatFactoryTest.java | 2 +- .../formats/csv/DataStreamCsvITCase.java | 347 ++++++++++++++++++ .../org.apache.flink.table.factories.Factory | 17 + .../flink/formats/common/Converter.java | 42 +++ 12 files changed, 929 insertions(+), 360 deletions(-) create mode 100644 flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvCommons.java create mode 100644 flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java delete mode 100644 flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java create mode 100644 flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java create mode 100644 flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java create mode 100644 flink-formats/flink-csv/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink-formats/flink-format-common/src/main/java/org/apache/flink/formats/common/Converter.java diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvCommons.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvCommons.java new file mode 100644 index 0000000000000..48825748f8251 --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvCommons.java @@ -0,0 +1,117 @@ +/* + * 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.flink.formats.csv; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.commons.lang3.StringEscapeUtils; + +import java.util.HashSet; +import java.util.Set; + +import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS; +import static org.apache.flink.formats.csv.CsvFormatOptions.ARRAY_ELEMENT_DELIMITER; +import static org.apache.flink.formats.csv.CsvFormatOptions.DISABLE_QUOTE_CHARACTER; +import static org.apache.flink.formats.csv.CsvFormatOptions.ESCAPE_CHARACTER; +import static org.apache.flink.formats.csv.CsvFormatOptions.FIELD_DELIMITER; +import static org.apache.flink.formats.csv.CsvFormatOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.csv.CsvFormatOptions.NULL_LITERAL; +import static org.apache.flink.formats.csv.CsvFormatOptions.QUOTE_CHARACTER; + +/** A class with common CSV format constants and utility methods. */ +class CsvCommons { + + public static final String IDENTIFIER = "csv"; + + // ------------------------------------------------------------------------ + // Validation + // ------------------------------------------------------------------------ + + static void validateFormatOptions(ReadableConfig tableOptions) { + final boolean hasQuoteCharacter = tableOptions.getOptional(QUOTE_CHARACTER).isPresent(); + final boolean isDisabledQuoteCharacter = tableOptions.get(DISABLE_QUOTE_CHARACTER); + if (isDisabledQuoteCharacter && hasQuoteCharacter) { + throw new ValidationException( + "Format cannot define a quote character and disabled quote character at the same time."); + } + // Validate the option value must be a single char. + validateCharacterVal(tableOptions, FIELD_DELIMITER, true); + validateCharacterVal(tableOptions, ARRAY_ELEMENT_DELIMITER); + validateCharacterVal(tableOptions, QUOTE_CHARACTER); + validateCharacterVal(tableOptions, ESCAPE_CHARACTER); + } + + /** Validates the option {@code option} value must be a Character. */ + private static void validateCharacterVal( + ReadableConfig tableOptions, ConfigOption option) { + validateCharacterVal(tableOptions, option, false); + } + + /** + * Validates the option {@code option} value must be a Character. + * + * @param tableOptions the table options + * @param option the config option + * @param unescape whether to unescape the option value + */ + private static void validateCharacterVal( + ReadableConfig tableOptions, ConfigOption option, boolean unescape) { + if (!tableOptions.getOptional(option).isPresent()) { + return; + } + + final String value = + unescape + ? StringEscapeUtils.unescapeJava(tableOptions.get(option)) + : tableOptions.get(option); + if (value.length() != 1) { + throw new ValidationException( + String.format( + "Option '%s.%s' must be a string with single character, but was: %s", + IDENTIFIER, option.key(), tableOptions.get(option))); + } + } + + public static Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(FIELD_DELIMITER); + options.add(DISABLE_QUOTE_CHARACTER); + options.add(QUOTE_CHARACTER); + options.add(ALLOW_COMMENTS); + options.add(IGNORE_PARSE_ERRORS); + options.add(ARRAY_ELEMENT_DELIMITER); + options.add(ESCAPE_CHARACTER); + options.add(NULL_LITERAL); + return options; + } + + public static Set> forwardOptions() { + Set> options = new HashSet<>(); + options.add(FIELD_DELIMITER); + options.add(DISABLE_QUOTE_CHARACTER); + options.add(QUOTE_CHARACTER); + options.add(ALLOW_COMMENTS); + options.add(ARRAY_ELEMENT_DELIMITER); + options.add(ESCAPE_CHARACTER); + options.add(NULL_LITERAL); + return options; + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java new file mode 100644 index 0000000000000..de87fa53b1730 --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java @@ -0,0 +1,163 @@ +/* + * 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.flink.formats.csv; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.impl.StreamFormatAdapter; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.table.factories.BulkReaderFormatFactory; +import org.apache.flink.connector.file.table.format.BulkDecodingFormat; +import org.apache.flink.formats.common.Converter; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource.Context; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import org.apache.commons.lang3.StringEscapeUtils; + +import java.util.Collections; +import java.util.Set; + +import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS; +import static org.apache.flink.formats.csv.CsvFormatOptions.ARRAY_ELEMENT_DELIMITER; +import static org.apache.flink.formats.csv.CsvFormatOptions.DISABLE_QUOTE_CHARACTER; +import static org.apache.flink.formats.csv.CsvFormatOptions.ESCAPE_CHARACTER; +import static org.apache.flink.formats.csv.CsvFormatOptions.FIELD_DELIMITER; +import static org.apache.flink.formats.csv.CsvFormatOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.csv.CsvFormatOptions.NULL_LITERAL; +import static org.apache.flink.formats.csv.CsvFormatOptions.QUOTE_CHARACTER; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** CSV format factory for file system. */ +@Internal +public class CsvFileFormatFactory implements BulkReaderFormatFactory { + + @Override + public String factoryIdentifier() { + return CsvCommons.IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + return CsvCommons.optionalOptions(); + } + + @Override + public Set> forwardOptions() { + return CsvCommons.forwardOptions(); + } + + @Override + public BulkDecodingFormat createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + return new CsvBulkDecodingFormat(formatOptions); + } + + private static class CsvBulkDecodingFormat + implements BulkDecodingFormat, + ProjectableDecodingFormat> { + + private final ReadableConfig formatOptions; + + public CsvBulkDecodingFormat(ReadableConfig formatOptions) { + checkNotNull(formatOptions); + this.formatOptions = formatOptions; + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public BulkFormat createRuntimeDecoder( + Context context, DataType physicalDataType, int[][] projections) { + + final DataType projectedDataType = Projection.of(projections).project(physicalDataType); + final RowType projectedRowType = (RowType) projectedDataType.getLogicalType(); + + final RowType physicalRowType = (RowType) physicalDataType.getLogicalType(); + final CsvSchema schema = buildCsvSchema(physicalRowType, formatOptions); + + final boolean ignoreParseErrors = + formatOptions.getOptional(IGNORE_PARSE_ERRORS).isPresent(); + final Converter converter = + (Converter) + new CsvToRowDataConverters(ignoreParseErrors) + .createRowConverter(projectedRowType, true); + CsvReaderFormat csvReaderFormat = + new CsvReaderFormat<>( + new CsvMapper(), + schema, + JsonNode.class, + converter, + context.createTypeInformation(projectedDataType), + ignoreParseErrors); + return new StreamFormatAdapter<>(csvReaderFormat); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + } + + private static CsvSchema buildCsvSchema(RowType rowType, ReadableConfig options) { + final CsvSchema csvSchema = CsvRowSchemaConverter.convert(rowType); + final CsvSchema.Builder csvBuilder = csvSchema.rebuild(); + // format properties + options.getOptional(FIELD_DELIMITER) + .map(s -> StringEscapeUtils.unescapeJava(s).charAt(0)) + .ifPresent(csvBuilder::setColumnSeparator); + + if (options.get(DISABLE_QUOTE_CHARACTER)) { + csvBuilder.disableQuoteChar(); + } else { + options.getOptional(QUOTE_CHARACTER) + .map(s -> s.charAt(0)) + .ifPresent(csvBuilder::setQuoteChar); + } + + options.getOptional(ALLOW_COMMENTS).ifPresent(csvBuilder::setAllowComments); + + options.getOptional(ARRAY_ELEMENT_DELIMITER) + .ifPresent(csvBuilder::setArrayElementSeparator); + + options.getOptional(ESCAPE_CHARACTER) + .map(s -> s.charAt(0)) + .ifPresent(csvBuilder::setEscapeChar); + + options.getOptional(NULL_LITERAL).ifPresent(csvBuilder::setNullValue); + + return csvBuilder.build(); + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java deleted file mode 100644 index 74b2c215fe59f..0000000000000 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java +++ /dev/null @@ -1,282 +0,0 @@ -/* - * 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.flink.formats.csv; - -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.file.table.factories.FileSystemFormatFactory; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.utils.PartitionPathUtils; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; - -import org.apache.commons.lang3.StringEscapeUtils; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.flink.formats.csv.CsvFormatFactory.validateFormatOptions; -import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS; -import static org.apache.flink.formats.csv.CsvFormatOptions.ARRAY_ELEMENT_DELIMITER; -import static org.apache.flink.formats.csv.CsvFormatOptions.DISABLE_QUOTE_CHARACTER; -import static org.apache.flink.formats.csv.CsvFormatOptions.ESCAPE_CHARACTER; -import static org.apache.flink.formats.csv.CsvFormatOptions.FIELD_DELIMITER; -import static org.apache.flink.formats.csv.CsvFormatOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.csv.CsvFormatOptions.NULL_LITERAL; -import static org.apache.flink.formats.csv.CsvFormatOptions.QUOTE_CHARACTER; - -/** CSV format factory for file system. */ -public class CsvFileSystemFormatFactory implements FileSystemFormatFactory { - - public static final String IDENTIFIER = "csv"; - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return new HashSet<>(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(FIELD_DELIMITER); - options.add(DISABLE_QUOTE_CHARACTER); - options.add(QUOTE_CHARACTER); - options.add(ALLOW_COMMENTS); - options.add(IGNORE_PARSE_ERRORS); - options.add(ARRAY_ELEMENT_DELIMITER); - options.add(ESCAPE_CHARACTER); - options.add(NULL_LITERAL); - return options; - } - - @Override - public InputFormat createReader(ReaderContext context) { - ReadableConfig options = context.getFormatOptions(); - validateFormatOptions(options); - - RowType formatRowType = context.getFormatRowType(); - - String[] fieldNames = context.getSchema().getFieldNames(); - List projectFields = - Arrays.stream(context.getProjectFields()) - .mapToObj(idx -> fieldNames[idx]) - .collect(Collectors.toList()); - List csvFields = - Arrays.stream(fieldNames) - .filter(field -> !context.getPartitionKeys().contains(field)) - .collect(Collectors.toList()); - - int[] csvSelectFieldToProjectFieldMapping = - context.getFormatProjectFields().stream() - .mapToInt(projectFields::indexOf) - .toArray(); - int[] csvSelectFieldToCsvFieldMapping = - context.getFormatProjectFields().stream().mapToInt(csvFields::indexOf).toArray(); - - CsvSchema csvSchema = buildCsvSchema(formatRowType, options); - - boolean ignoreParseErrors = options.get(IGNORE_PARSE_ERRORS); - - return new CsvInputFormat( - context.getPaths(), - context.getSchema().getFieldDataTypes(), - context.getSchema().getFieldNames(), - csvSchema, - formatRowType, - context.getProjectFields(), - context.getPartitionKeys(), - context.getDefaultPartName(), - context.getPushedDownLimit(), - csvSelectFieldToProjectFieldMapping, - csvSelectFieldToCsvFieldMapping, - ignoreParseErrors); - } - - private CsvSchema buildCsvSchema(RowType rowType, ReadableConfig options) { - CsvSchema csvSchema = CsvRowSchemaConverter.convert(rowType); - CsvSchema.Builder csvBuilder = csvSchema.rebuild(); - // format properties - options.getOptional(FIELD_DELIMITER) - .map(s -> StringEscapeUtils.unescapeJava(s).charAt(0)) - .ifPresent(csvBuilder::setColumnSeparator); - - options.getOptional(QUOTE_CHARACTER) - .map(s -> s.charAt(0)) - .ifPresent(csvBuilder::setQuoteChar); - - options.getOptional(ALLOW_COMMENTS).ifPresent(csvBuilder::setAllowComments); - - options.getOptional(ARRAY_ELEMENT_DELIMITER) - .ifPresent(csvBuilder::setArrayElementSeparator); - - options.getOptional(ESCAPE_CHARACTER) - .map(s -> s.charAt(0)) - .ifPresent(csvBuilder::setEscapeChar); - - options.getOptional(NULL_LITERAL).ifPresent(csvBuilder::setNullValue); - - return csvBuilder.build(); - } - - /** InputFormat that reads csv record into {@link RowData}. */ - public static class CsvInputFormat extends AbstractCsvInputFormat { - private static final long serialVersionUID = 1L; - - private final RowType formatRowType; - private final DataType[] fieldTypes; - private final String[] fieldNames; - private final int[] selectFields; - private final List partitionKeys; - private final String defaultPartValue; - private final long limit; - private final int[] csvSelectFieldToProjectFieldMapping; - private final int[] csvSelectFieldToCsvFieldMapping; - private final boolean ignoreParseErrors; - - private transient InputStreamReader inputStreamReader; - private transient BufferedReader reader; - private transient boolean end; - private transient long emitted; - // reuse object for per record - private transient GenericRowData rowData; - private transient CsvToRowDataConverters.CsvToRowDataConverter runtimeConverter; - private transient MappingIterator iterator; - - public CsvInputFormat( - Path[] filePaths, - DataType[] fieldTypes, - String[] fieldNames, - CsvSchema csvSchema, - RowType formatRowType, - int[] selectFields, - List partitionKeys, - String defaultPartValue, - long limit, - int[] csvSelectFieldToProjectFieldMapping, - int[] csvSelectFieldToCsvFieldMapping, - boolean ignoreParseErrors) { - super(filePaths, csvSchema); - this.fieldTypes = fieldTypes; - this.fieldNames = fieldNames; - this.formatRowType = formatRowType; - this.partitionKeys = partitionKeys; - this.defaultPartValue = defaultPartValue; - this.selectFields = selectFields; - this.limit = limit; - this.emitted = 0; - this.csvSelectFieldToProjectFieldMapping = csvSelectFieldToProjectFieldMapping; - this.csvSelectFieldToCsvFieldMapping = csvSelectFieldToCsvFieldMapping; - this.ignoreParseErrors = ignoreParseErrors; - } - - @Override - public void open(FileInputSplit split) throws IOException { - super.open(split); - this.end = false; - this.inputStreamReader = new InputStreamReader(csvInputStream); - this.reader = new BufferedReader(inputStreamReader); - this.rowData = - PartitionPathUtils.fillPartitionValueForRecord( - fieldNames, - fieldTypes, - selectFields, - partitionKeys, - currentSplit.getPath(), - defaultPartValue); - this.iterator = - new CsvMapper() - .readerFor(JsonNode.class) - .with(csvSchema) - .readValues(csvInputStream); - prepareRuntimeConverter(); - } - - private void prepareRuntimeConverter() { - this.runtimeConverter = - new CsvToRowDataConverters(ignoreParseErrors) - .createRowConverter(formatRowType, true); - } - - @Override - public boolean reachedEnd() throws IOException { - return emitted >= limit || end; - } - - @Override - public RowData nextRecord(RowData reuse) throws IOException { - GenericRowData csvRow = null; - while (csvRow == null) { - try { - JsonNode root = iterator.nextValue(); - csvRow = (GenericRowData) runtimeConverter.convert(root); - } catch (NoSuchElementException e) { - end = true; - return null; - } catch (Throwable t) { - if (!ignoreParseErrors) { - throw new IOException("Failed to deserialize CSV row.", t); - } - } - } - - GenericRowData returnRecord = rowData; - for (int i = 0; i < csvSelectFieldToCsvFieldMapping.length; i++) { - returnRecord.setField( - csvSelectFieldToProjectFieldMapping[i], - csvRow.getField(csvSelectFieldToCsvFieldMapping[i])); - } - emitted++; - return returnRecord; - } - - @Override - public void close() throws IOException { - super.close(); - if (reader != null) { - reader.close(); - reader = null; - } - if (inputStreamReader != null) { - inputStreamReader.close(); - inputStreamReader = null; - } - } - } -} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java index ddfd685b0d227..0bd8aa6433199 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; @@ -41,7 +40,6 @@ import org.apache.commons.lang3.StringEscapeUtils; import java.util.Collections; -import java.util.HashSet; import java.util.Set; import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS; @@ -61,13 +59,11 @@ public final class CsvFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { - public static final String IDENTIFIER = "csv"; - @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - validateFormatOptions(formatOptions); + CsvCommons.validateFormatOptions(formatOptions); return new DecodingFormat>() { @Override @@ -93,7 +89,7 @@ public ChangelogMode getChangelogMode() { public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - validateFormatOptions(formatOptions); + CsvCommons.validateFormatOptions(formatOptions); return new EncodingFormat>() { @Override @@ -115,7 +111,7 @@ public ChangelogMode getChangelogMode() { @Override public String factoryIdentifier() { - return IDENTIFIER; + return CsvCommons.IDENTIFIER; } @Override @@ -125,76 +121,12 @@ public Set> requiredOptions() { @Override public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(FIELD_DELIMITER); - options.add(DISABLE_QUOTE_CHARACTER); - options.add(QUOTE_CHARACTER); - options.add(ALLOW_COMMENTS); - options.add(IGNORE_PARSE_ERRORS); - options.add(ARRAY_ELEMENT_DELIMITER); - options.add(ESCAPE_CHARACTER); - options.add(NULL_LITERAL); - return options; + return CsvCommons.optionalOptions(); } @Override public Set> forwardOptions() { - Set> options = new HashSet<>(); - options.add(FIELD_DELIMITER); - options.add(DISABLE_QUOTE_CHARACTER); - options.add(QUOTE_CHARACTER); - options.add(ALLOW_COMMENTS); - options.add(ARRAY_ELEMENT_DELIMITER); - options.add(ESCAPE_CHARACTER); - options.add(NULL_LITERAL); - return options; - } - - // ------------------------------------------------------------------------ - // Validation - // ------------------------------------------------------------------------ - - static void validateFormatOptions(ReadableConfig tableOptions) { - final boolean hasQuoteCharacter = tableOptions.getOptional(QUOTE_CHARACTER).isPresent(); - final boolean isDisabledQuoteCharacter = tableOptions.get(DISABLE_QUOTE_CHARACTER); - if (isDisabledQuoteCharacter && hasQuoteCharacter) { - throw new ValidationException( - "Format cannot define a quote character and disabled quote character at the same time."); - } - // Validate the option value must be a single char. - validateCharacterVal(tableOptions, FIELD_DELIMITER, true); - validateCharacterVal(tableOptions, ARRAY_ELEMENT_DELIMITER); - validateCharacterVal(tableOptions, QUOTE_CHARACTER); - validateCharacterVal(tableOptions, ESCAPE_CHARACTER); - } - - /** Validates the option {@code option} value must be a Character. */ - private static void validateCharacterVal( - ReadableConfig tableOptions, ConfigOption option) { - validateCharacterVal(tableOptions, option, false); - } - - /** - * Validates the option {@code option} value must be a Character. - * - * @param tableOptions the table options - * @param option the config option - * @param unescape whether to unescape the option value - */ - private static void validateCharacterVal( - ReadableConfig tableOptions, ConfigOption option, boolean unescape) { - if (tableOptions.getOptional(option).isPresent()) { - final String value = - unescape - ? StringEscapeUtils.unescapeJava(tableOptions.get(option)) - : tableOptions.get(option); - if (value.length() != 1) { - throw new ValidationException( - String.format( - "Option '%s.%s' must be a string with single character, but was: %s", - IDENTIFIER, option.key(), tableOptions.get(option))); - } - } + return CsvCommons.forwardOptions(); } // ------------------------------------------------------------------------ diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java new file mode 100644 index 0000000000000..1ce00fd4d70d8 --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java @@ -0,0 +1,212 @@ +/* + * 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.flink.formats.csv; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.reader.SimpleStreamFormat; +import org.apache.flink.connector.file.src.reader.StreamFormat; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.formats.common.Converter; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@code StreamFormat} for reading CSV files. + * + *

The following example shows how to create a {@code CsvReaderFormat} where the schema for CSV + * parsing is automatically derived based on the fields of a POJO class. + * + *

{@code
+ * CsvReaderFormat csvFormat = CsvReaderFormat.forPojo(SomePojo.class);
+ * FileSource source =
+ *         FileSource.forRecordStreamFormat(csvFormat, Path.fromLocalFile(filesPath)).build();
+ * }
+ * + * Note: you might need to add {@code @JsonPropertyOrder({field1, field2, ...})} annotation from + * the {@code Jackson} library to your class definition with the fields order exactly matching those + * of the CSV file columns). + * + *

If you need more fine-grained control over the CSV schema or the parsing options, use the more + * low-level {@code forSchema} static factory method based on the {@code Jackson} library utilities: + * + *

{@code
+ * CsvMapper mapper = new CsvMapper();
+ * CsvSchema schema = mapper.schemaFor(SomePojo.class)
+ *                          .withColumnSeparator('|');
+ * CsvReaderFormat csvFormat =
+ *          CsvReaderFormat.forSchema(mapper,schema, TypeInformation.of(SomePojo.class));
+ * FileSource source =
+ *         FileSource.forRecordStreamFormat(csvFormat, Path.fromLocalFile(filesPath)).build();
+ * }
+ * + * @param The type of the returned elements. + */ +@PublicEvolving +public class CsvReaderFormat extends SimpleStreamFormat { + + private static final long serialVersionUID = 1L; + + private final CsvMapper mapper; + private final CsvSchema schema; + private final Class rootType; + private final Converter converter; + private final TypeInformation typeInformation; + private boolean ignoreParseErrors; + + @SuppressWarnings("unchecked") + CsvReaderFormat( + CsvMapper mapper, + CsvSchema schema, + Class rootType, + Converter converter, + TypeInformation typeInformation, + boolean ignoreParseErrors) { + this.mapper = checkNotNull(mapper); + this.schema = checkNotNull(schema); + this.rootType = (Class) checkNotNull(rootType); + this.typeInformation = checkNotNull(typeInformation); + this.converter = (Converter) checkNotNull(converter); + this.ignoreParseErrors = ignoreParseErrors; + } + + /** + * Builds a new {@code CsvReaderFormat} using a {@code CsvSchema}. + * + * @param schema The Jackson CSV schema configured for parsing specific CSV files. + * @param typeInformation The Flink type descriptor of the returned elements. + * @param The type of the returned elements. + */ + public static CsvReaderFormat forSchema( + CsvSchema schema, TypeInformation typeInformation) { + return forSchema(new CsvMapper(), schema, typeInformation); + } + + /** + * Builds a new {@code CsvReaderFormat} using a {@code CsvSchema} and a pre-created {@code + * CsvMapper}. + * + * @param mapper The pre-created {@code CsvMapper}. + * @param schema The Jackson CSV schema configured for parsing specific CSV files. + * @param typeInformation The Flink type descriptor of the returned elements. + * @param The type of the returned elements. + */ + public static CsvReaderFormat forSchema( + CsvMapper mapper, CsvSchema schema, TypeInformation typeInformation) { + return new CsvReaderFormat<>( + mapper, + schema, + typeInformation.getTypeClass(), + (value, context) -> value, + typeInformation, + false); + } + + /** + * Builds a new {@code CsvReaderFormat} for reading CSV files mapped to the provided POJO class + * definition. Produced reader uses default mapper and schema settings, use {@code forSchema} if + * you need customizations. + * + * @param pojoType The type class of the POJO. + * @param The type of the returned elements. + */ + public static CsvReaderFormat forPojo(Class pojoType) { + CsvMapper mapper = new CsvMapper(); + return forSchema( + mapper, + mapper.schemaFor(pojoType).withoutQuoteChar(), + TypeInformation.of(pojoType)); + } + + /** + * Returns a new {@code CsvReaderFormat} configured to ignore all parsing errors. All thye other + * options directly carried over from the subject of the method call. + */ + public CsvReaderFormat withIgnoreParseErrors() { + return new CsvReaderFormat( + this.mapper, + this.schema, + this.rootType, + this.converter, + this.typeInformation, + true); + } + + @Override + public StreamFormat.Reader createReader(Configuration config, FSDataInputStream stream) + throws IOException { + return new Reader<>( + mapper.readerFor(rootType).with(schema).readValues(stream), + converter, + ignoreParseErrors); + } + + @Override + public TypeInformation getProducedType() { + return typeInformation; + } + + // ------------------------------------------------------------------------ + + /** The actual reader for the {@code CsvFormat}. */ + private static final class Reader implements StreamFormat.Reader { + private final MappingIterator iterator; + private final Converter converter; + private final boolean ignoreParseErrors; + + public Reader( + MappingIterator iterator, + Converter converter, + boolean ignoreParseErrors) { + this.iterator = checkNotNull(iterator); + this.converter = checkNotNull(converter); + this.ignoreParseErrors = ignoreParseErrors; + } + + @Nullable + @Override + public T read() throws IOException { + while (iterator.hasNext()) { + try { + R nextElement = iterator.next(); + return converter.convert(nextElement, null); + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw t; + } + } + } + return null; + } + + @Override + public void close() throws IOException { + iterator.close(); + } + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java index 526ed5ca96bff..ed307a9a07fb8 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.csv; import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.common.Converter; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericRowData; @@ -65,7 +66,12 @@ public CsvToRowDataConverters(boolean ignoreParseErrors) { * data structures. */ @FunctionalInterface - public interface CsvToRowDataConverter extends Serializable { + interface CsvToRowDataConverter extends Converter { + @Override + default Object convert(JsonNode source, Void context) { + return convert(source); + } + Object convert(JsonNode jsonNode); } @@ -305,7 +311,7 @@ private CsvToRowDataConverter createArrayConverter(ArrayType arrayType) { } private static void validateArity(int expected, int actual, boolean ignoreParseErrors) { - if (expected != actual && !ignoreParseErrors) { + if (expected > actual && !ignoreParseErrors) { throw new RuntimeException( "Row length mismatch. " + expected diff --git a/flink-formats/flink-csv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats/flink-csv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index fdd66f47f9448..0e962ecdd2ecc 100644 --- a/flink-formats/flink-csv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats/flink-csv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,5 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.flink.formats.csv.CsvFileFormatFactory org.apache.flink.formats.csv.CsvFormatFactory -org.apache.flink.formats.csv.CsvFileSystemFormatFactory diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemBatchITCase.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemBatchITCase.java index f6dadc59fbbb9..4b9436998ac72 100644 --- a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemBatchITCase.java +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemBatchITCase.java @@ -80,7 +80,22 @@ public void testParseError() throws Exception { new File(path).mkdirs(); File file = new File(path, "test_file"); file.createNewFile(); - FileUtils.writeFileUtf8(file, "x5,5,1,1\n" + "x5,5,2,2,2\n" + "x5,5,1,1"); + FileUtils.writeFileUtf8( + file, "x5,5,1,1\n" + "x5,5,2,2,2\n" + "x5,5,3,3,3,3\n" + "x5,5,1,1"); + + check( + "select * from nonPartitionedTable", + Arrays.asList(Row.of("x5", 5, 1, 1), Row.of("x5", 5, 1, 1))); + } + + @Test + public void testParseErrorLast() throws Exception { + String path = new URI(resultPath()).getPath(); + new File(path).mkdirs(); + File file = new File(path, "test_file"); + file.createNewFile(); + FileUtils.writeFileUtf8( + file, "x5,5,1,1\n" + "x5,5,2,2,2\n" + "x5,5,1,1\n" + "x5,5,3,3,3,3\n"); check( "select * from nonPartitionedTable", diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatFactoryTest.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatFactoryTest.java index 9c0d4c602b258..54eac4bbf7141 100644 --- a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatFactoryTest.java +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatFactoryTest.java @@ -249,7 +249,7 @@ private static Map getAllOptions() { options.put("target", "MyTarget"); options.put("buffer-size", "1000"); - options.put("format", CsvFormatFactory.IDENTIFIER); + options.put("format", CsvCommons.IDENTIFIER); options.put("csv.field-delimiter", ";"); options.put("csv.quote-character", "'"); options.put("csv.allow-comments", "true"); diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java new file mode 100644 index 0000000000000..23fbfa70cf188 --- /dev/null +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java @@ -0,0 +1,347 @@ +/* + * 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.flink.formats.csv; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.StreamFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.testutils.AllCallbackWrapper; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.collect.ClientAndIterator; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.function.FunctionWithException; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.io.Serializable; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; + +/** MiniCluster-based integration tests CSV data format. */ +@ExtendWith({TestLoggerExtension.class}) +public class DataStreamCsvITCase { + + private static final int PARALLELISM = 4; + + @TempDir File outDir; + + private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + @RegisterExtension + private static AllCallbackWrapper allCallbackWrapper = + new AllCallbackWrapper<>(MINI_CLUSTER_RESOURCE); + + // ------------------------------------------------------------------------ + // test data + // ------------------------------------------------------------------------ + + private static final String[] CSV_LINES = + new String[] { + "Berlin,52.5167,13.3833,Germany,DE,Berlin,primary,3644826", + "San Francisco,37.7562,-122.443,United States,US,California,,3592294", + "Beijing,39.905,116.3914,China,CN,Beijing,primary,19433000" + }; + + private static final String[] CSV_LINES_PIPE_SEPARATED = + new String[] { + "Berlin|52.5167|13.3833|Germany|DE|Berlin|primary|3644826", + "San Francisco|37.7562|-122.443|United States|US|California||3592294", + "Beijing|39.905|116.3914|China|CN|Beijing|primary|19433000" + }; + + private static final String[] CSV_LINES_MALFORMED = + new String[] { + "Berlin,52.5167,13.3833,Germany,DE,Berlin,primary,3644826", + "San Francisco,MALFORMED,3592294", + "Beijing,39.905,116.3914,China,CN,Beijing,primary,19433000" + }; + + static final CityPojo[] POJOS = + new CityPojo[] { + new CityPojo( + "Berlin", + new BigDecimal("52.5167"), + new BigDecimal("13.3833"), + "Germany", + "DE", + "Berlin", + "primary", + 3644826L), + new CityPojo( + "San Francisco", + new BigDecimal("37.7562"), + new BigDecimal("-122.443"), + "United States", + "US", + "California", + "", + 3592294L), + new CityPojo( + "Beijing", + new BigDecimal("39.905"), + new BigDecimal("116.3914"), + "China", + "CN", + "Beijing", + "primary", + 19433000L) + }; + + // ------------------------------------------------------------------------ + // test cases + // ------------------------------------------------------------------------ + @Test + public void testCsvReaderFormatFromPojo() throws Exception { + writeFile(outDir, "data.csv", CSV_LINES); + + final CsvReaderFormat csvFormat = CsvReaderFormat.forPojo(CityPojo.class); + final List result = initializeSourceAndReadData(outDir, csvFormat); + + assertThat(Arrays.asList(POJOS)).isEqualTo(result); + } + + @Test + public void testCsvReaderFormatFromSchema() throws Exception { + writeFile(outDir, "data.csv", CSV_LINES_PIPE_SEPARATED); + + CsvMapper mapper = new CsvMapper(); + CsvSchema schema = + mapper.schemaFor(CityPojo.class).withoutQuoteChar().withColumnSeparator('|'); + + final CsvReaderFormat csvFormat = + CsvReaderFormat.forSchema(mapper, schema, TypeInformation.of(CityPojo.class)); + final List result = initializeSourceAndReadData(outDir, csvFormat); + + assertThat(Arrays.asList(POJOS)).isEqualTo(result); + } + + @Test + public void testCsvReaderFormatMalformed() throws Exception { + writeFile(outDir, "data.csv", CSV_LINES_MALFORMED); + + final CsvReaderFormat csvFormat = + CsvReaderFormat.forPojo(CityPojo.class).withIgnoreParseErrors(); + final List result = initializeSourceAndReadData(outDir, csvFormat); + + List expected = new ArrayList<>(); + expected.add(POJOS[0]); + expected.add(POJOS[2]); + + assertThat(expected).isEqualTo(result); + } + + @JsonPropertyOrder({ + "city", + "lat", + "lng", + "country", + "iso2", + "adminName", + "capital", + "population" + }) + static class CityPojo implements Serializable { + public String city; + public BigDecimal lat; + public BigDecimal lng; + public String country; + public String iso2; + public String adminName; + public String capital; + public long population; + + public CityPojo() {} + + public CityPojo( + String city, + BigDecimal lat, + BigDecimal lng, + String country, + String iso2, + String adminName, + String capital, + long population) { + this.city = city; + this.lat = lat; + this.lng = lng; + this.country = country; + this.iso2 = iso2; + this.adminName = adminName; + this.capital = capital; + this.population = population; + } + + @Override + public String toString() { + return "CitiesPojo{" + + "city='" + + city + + '\'' + + ", lat=" + + lat + + ", lng=" + + lng + + ", country='" + + country + + '\'' + + ", iso2='" + + iso2 + + '\'' + + ", adminName='" + + adminName + + '\'' + + ", capital='" + + capital + + '\'' + + ", population=" + + population + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CityPojo that = (CityPojo) o; + return population == that.population + && Objects.equals(city, that.city) + && Objects.equals(lat, that.lat) + && Objects.equals(lng, that.lng) + && Objects.equals(country, that.country) + && Objects.equals(iso2, that.iso2) + && Objects.equals(adminName, that.adminName) + && Objects.equals(capital, that.capital); + } + + @Override + public int hashCode() { + return Objects.hash(city, lat, lng, country, iso2, adminName, capital, population); + } + } + + private static List initializeSourceAndReadData(File testDir, StreamFormat csvFormat) + throws Exception { + final FileSource source = + FileSource.forRecordStreamFormat(csvFormat, Path.fromLocalFile(testDir)).build(); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + + final DataStream stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); + + return getResultsFromStream(stream); + } + + @NotNull + private static List getResultsFromStream(DataStream stream) throws Exception { + final ClientAndIterator client = + DataStreamUtils.collectWithClient(stream, "Bounded Results Fetch"); + + final List result = new ArrayList<>(); + while (client.iterator.hasNext()) { + T next = client.iterator.next(); + result.add(next); + } + return result; + } + + // ------------------------------------------------------------------------ + // Write data utils + // ------------------------------------------------------------------------ + + private static void writeFile(File testDir, String fileName, String[] lines) + throws IOException { + final File file = new File(testDir, fileName); + writeFileAtomically(file, lines); + } + + private static void writeFileAtomically(final File file, final String[] lines) + throws IOException { + writeFileAtomically(file, lines, (v) -> v); + } + + private static void writeFileAtomically( + final File file, + final String[] lines, + final FunctionWithException + streamEncoderFactory) + throws IOException { + + final File stagingFile = + new File(file.getParentFile(), ".tmp-" + UUID.randomUUID().toString()); + + try (final FileOutputStream fileOut = new FileOutputStream(stagingFile); + final OutputStream out = streamEncoderFactory.apply(fileOut); + final OutputStreamWriter encoder = + new OutputStreamWriter(out, StandardCharsets.UTF_8); + final PrintWriter writer = new PrintWriter(encoder)) { + + for (String line : lines) { + writer.println(line); + } + } + + final File parent = file.getParentFile(); + + assertThat(parent.mkdirs() || parent.exists()).isTrue(); + assertThat(stagingFile.renameTo(file)).isTrue(); + } +} diff --git a/flink-formats/flink-csv/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats/flink-csv/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000000..0e962ecdd2ecc --- /dev/null +++ b/flink-formats/flink-csv/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,17 @@ +# 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. + +org.apache.flink.formats.csv.CsvFileFormatFactory +org.apache.flink.formats.csv.CsvFormatFactory diff --git a/flink-formats/flink-format-common/src/main/java/org/apache/flink/formats/common/Converter.java b/flink-formats/flink-format-common/src/main/java/org/apache/flink/formats/common/Converter.java new file mode 100644 index 0000000000000..6584227f2e1d2 --- /dev/null +++ b/flink-formats/flink-format-common/src/main/java/org/apache/flink/formats/common/Converter.java @@ -0,0 +1,42 @@ +/* + * 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.flink.formats.common; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * A generic interface for converting data types. + * + * @param The type of the element to be converted. + * @param The output type. + * @param The context for passing optional conversion instructions. + */ +@PublicEvolving +public interface Converter extends Serializable { + /** + * Converts elements of type {@code From} into elements of type {@code To}. + * + * @param source The element to be converted. + * @param context The context with conversion instructions and utilities. + * @return The element in the target type. + */ + To convert(From source, C context); +} From ca58a700bbc0522f3c62e9db720f9f89c8bd8313 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 26 Nov 2021 02:20:40 +0100 Subject: [PATCH 09/49] [FLINK-24703][connectors][format] Add CSV File System encoding format based on BulkWriter. 'flink-connector-file' introduced new interfaces for interacting with the file system. This commit contributes to the goal of migrating all formats used for writing files to this API, in particular, BulkWriter used by FileSink. It also makes use of the generic Converter interface to map types between the types used by Flink (in particular by Table/SQL API) and the native format types of the format encoder. This commit also adds tests that use the reader added and the previous commit and the writer added in this commit together. --- .../flink/formats/csv/CsvBulkWriter.java | 112 ++++++++++ .../formats/csv/CsvFileFormatFactory.java | 41 +++- .../csv/CsvRowDataSerializationSchema.java | 10 +- .../formats/csv/RowDataToCsvConverters.java | 27 ++- .../formats/csv/DataStreamCsvITCase.java | 64 ++++++ .../formats/csv/TableCsvFormatITCase.java | 204 ++++++++++++++++++ .../table/planner/utils/JsonPlanTestBase.java | 37 +++- 7 files changed, 480 insertions(+), 15 deletions(-) create mode 100644 flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java create mode 100644 flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/TableCsvFormatITCase.java diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java new file mode 100644 index 0000000000000..69b3f9bce283f --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java @@ -0,0 +1,112 @@ +/* + * 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.flink.formats.csv; + +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.formats.common.Converter; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A simple {@link BulkWriter} implementation based on Jackson CSV transformations. */ +class CsvBulkWriter implements BulkWriter { + + private final FSDataOutputStream stream; + private final Converter converter; + @Nullable private final C converterContext; + private final ObjectWriter csvWriter; + + CsvBulkWriter( + CsvMapper mapper, + CsvSchema schema, + Converter converter, + @Nullable C converterContext, + FSDataOutputStream stream) { + checkNotNull(mapper); + checkNotNull(schema); + + this.converter = checkNotNull(converter); + this.stream = checkNotNull(stream); + this.converterContext = converterContext; + this.csvWriter = mapper.writer(schema); + + // Prevent Jackson's writeValue() method calls from closing the stream. + mapper.getFactory().disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET); + } + + /** + * Builds a writer with Jackson schema and a type converter. + * + * @param mapper The specialized mapper for producing CSV. + * @param schema The schema that defined the mapping properties. + * @param converter The type converter that converts incoming elements of type {@code } into + * elements of type JsonNode. + * @param stream The output stream. + * @param The type of the elements accepted by this writer. + * @param The type of the converter context. + * @param The type of the elements produced by this writer. + */ + static CsvBulkWriter forSchema( + CsvMapper mapper, + CsvSchema schema, + Converter converter, + @Nullable C converterContext, + FSDataOutputStream stream) { + return new CsvBulkWriter<>(mapper, schema, converter, converterContext, stream); + } + + /** + * Builds a writer based on a POJO class definition. + * + * @param pojoClass The class of the POJO. + * @param stream The output stream. + * @param The type of the elements accepted by this writer. + */ + static CsvBulkWriter forPojo(Class pojoClass, FSDataOutputStream stream) { + final Converter converter = (value, context) -> value; + final CsvMapper csvMapper = new CsvMapper(); + final CsvSchema schema = csvMapper.schemaFor(pojoClass).withoutQuoteChar(); + return new CsvBulkWriter<>(csvMapper, schema, converter, null, stream); + } + + @Override + public void addElement(T element) throws IOException { + final R r = converter.convert(element, converterContext); + csvWriter.writeValue(stream, r); + } + + @Override + public void flush() throws IOException { + stream.flush(); + } + + @Override + public void finish() throws IOException { + stream.sync(); + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java index de87fa53b1730..8ea2280eaf1ef 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java @@ -19,17 +19,23 @@ package org.apache.flink.formats.csv; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.common.serialization.BulkWriter.Factory; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.file.src.FileSourceSplit; import org.apache.flink.connector.file.src.impl.StreamFormatAdapter; import org.apache.flink.connector.file.src.reader.BulkFormat; import org.apache.flink.connector.file.table.factories.BulkReaderFormatFactory; +import org.apache.flink.connector.file.table.factories.BulkWriterFormatFactory; import org.apache.flink.connector.file.table.format.BulkDecodingFormat; import org.apache.flink.formats.common.Converter; +import org.apache.flink.formats.csv.RowDataToCsvConverters.RowDataToCsvConverter; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.format.ProjectableDecodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.source.DynamicTableSource.Context; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DynamicTableFactory; @@ -37,6 +43,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; @@ -57,7 +64,7 @@ /** CSV format factory for file system. */ @Internal -public class CsvFileFormatFactory implements BulkReaderFormatFactory { +public class CsvFileFormatFactory implements BulkReaderFormatFactory, BulkWriterFormatFactory { @Override public String factoryIdentifier() { @@ -131,6 +138,38 @@ public ChangelogMode getChangelogMode() { } } + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + return new EncodingFormat>() { + @Override + public BulkWriter.Factory createRuntimeEncoder( + DynamicTableSink.Context context, DataType physicalDataType) { + + final RowType rowType = (RowType) physicalDataType.getLogicalType(); + final CsvSchema schema = buildCsvSchema(rowType, formatOptions); + + final RowDataToCsvConverter converter = + RowDataToCsvConverters.createRowConverter(rowType); + + final CsvMapper mapper = new CsvMapper(); + final ObjectNode container = mapper.createObjectNode(); + + final RowDataToCsvConverter.RowDataToCsvFormatConverterContext converterContext = + new RowDataToCsvConverter.RowDataToCsvFormatConverterContext( + mapper, container); + + return out -> + CsvBulkWriter.forSchema(mapper, schema, converter, converterContext, out); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + private static CsvSchema buildCsvSchema(RowType rowType, ReadableConfig options) { final CsvSchema csvSchema = CsvRowSchemaConverter.convert(rowType); final CsvSchema.Builder csvBuilder = csvSchema.rebuild(); diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataSerializationSchema.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataSerializationSchema.java index 87abdbaaebbe4..16d7c966a52b6 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataSerializationSchema.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataSerializationSchema.java @@ -65,6 +65,11 @@ public final class CsvRowDataSerializationSchema implements SerializationSchema< /** Reusable object node. */ private transient ObjectNode root; + /** Reusable converter context. */ + private transient RowDataToCsvConverters.RowDataToCsvConverter + .RowDataToCsvFormatConverterContext + converterContext; + private CsvRowDataSerializationSchema(RowType rowType, CsvSchema csvSchema) { this.rowType = rowType; this.runtimeConverter = RowDataToCsvConverters.createRowConverter(rowType); @@ -132,9 +137,12 @@ public CsvRowDataSerializationSchema build() { public byte[] serialize(RowData row) { if (root == null) { root = csvMapper.createObjectNode(); + converterContext = + new RowDataToCsvConverters.RowDataToCsvConverter + .RowDataToCsvFormatConverterContext(csvMapper, root); } try { - runtimeConverter.convert(csvMapper, root, row); + runtimeConverter.convert(row, converterContext); return objectWriter.writeValueAsBytes(root); } catch (Throwable t) { throw new RuntimeException(String.format("Could not serialize row '%s'.", row), t); diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowDataToCsvConverters.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowDataToCsvConverters.java index ea1bd8d626877..693e89198e2cb 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowDataToCsvConverters.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowDataToCsvConverters.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.csv; import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.common.Converter; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.RowData; @@ -57,8 +58,23 @@ public class RowDataToCsvConverters implements Serializable { * Runtime converter that converts objects of Flink Table & SQL internal data structures to * corresponding {@link JsonNode}s. */ - public interface RowDataToCsvConverter extends Serializable { - JsonNode convert(CsvMapper csvMapper, ContainerNode container, RowData row); + interface RowDataToCsvConverter + extends Converter< + RowData, JsonNode, RowDataToCsvConverter.RowDataToCsvFormatConverterContext> { + /** + * Converter context for passing the {@code CsvMapper} and the {@code container} that can be + * reused between transformations of the individual elements for performance reasons. + */ + class RowDataToCsvFormatConverterContext implements Serializable { + CsvMapper csvMapper; + ContainerNode container; + + public RowDataToCsvFormatConverterContext( + CsvMapper csvMapper, ContainerNode container) { + this.csvMapper = csvMapper; + this.container = container; + } + } } private interface RowFieldConverter extends Serializable { @@ -80,14 +96,15 @@ public static RowDataToCsvConverter createRowConverter(RowType type) { .map(RowDataToCsvConverters::createNullableRowFieldConverter) .toArray(RowFieldConverter[]::new); final int rowArity = type.getFieldCount(); - return (csvMapper, container, row) -> { + return (row, context) -> { // top level reuses the object node container - final ObjectNode objectNode = (ObjectNode) container; + final ObjectNode objectNode = (ObjectNode) context.container; for (int i = 0; i < rowArity; i++) { try { objectNode.set( fieldNames[i], - fieldConverters[i].convert(csvMapper, container, row, i)); + fieldConverters[i].convert( + context.csvMapper, context.container, row, i)); } catch (Throwable t) { throw new RuntimeException( String.format("Fail to serialize at field: %s.", fieldNames[i]), t); diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java index 23fbfa70cf188..a9a64e71d35c4 100644 --- a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java @@ -20,17 +20,21 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.serialization.BulkWriter; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.file.sink.FileSink; import org.apache.flink.connector.file.src.FileSource; import org.apache.flink.connector.file.src.reader.StreamFormat; import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.AllCallbackWrapper; +import org.apache.flink.formats.common.Converter; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.MiniClusterExtension; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamUtils; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner; import org.apache.flink.streaming.api.operators.collect.ClientAndIterator; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.function.FunctionWithException; @@ -39,6 +43,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; +import org.apache.commons.io.FileUtils; import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -56,9 +61,13 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.UUID; +import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; @@ -182,6 +191,61 @@ public void testCsvReaderFormatMalformed() throws Exception { assertThat(expected).isEqualTo(result); } + @Test + public void testCustomBulkWriter() throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + // fromCollection is not bounded, using fromSequence instead + final List pojosList = Arrays.asList(POJOS); // needs to be Serializable + final DataStream sequence = + env.fromSequence(0, POJOS.length - 1).map(Long::intValue); + final DataStream stream = sequence.map(pojosList::get).returns(CityPojo.class); + + FileSink sink = + FileSink.forBulkFormat(new Path(outDir.toURI()), factoryForPojo(CityPojo.class)) + .withBucketAssigner(new BasePathBucketAssigner<>()) + .build(); + + stream.sinkTo(sink); + env.execute(); + + String[] result = getResultsFromSinkFiles(outDir); + + assertThat(result).containsExactlyInAnyOrder(CSV_LINES); + } + + @NotNull + private String[] getResultsFromSinkFiles(File outDir) throws IOException { + final Map contents = getFileContentByPath(outDir); + + List resultList = + contents.entrySet().stream() + .flatMap(e -> Arrays.stream(e.getValue().split("\n"))) + .collect(Collectors.toList()); + + String[] result = resultList.toArray(new String[0]); + return result; + } + + private static BulkWriter.Factory factoryForPojo(Class pojoClass) { + final Converter converter = (value, context) -> value; + final CsvMapper csvMapper = new CsvMapper(); + final CsvSchema schema = csvMapper.schemaFor(pojoClass).withoutQuoteChar(); + return (out) -> new CsvBulkWriter<>(csvMapper, schema, converter, null, out); + } + + private static Map getFileContentByPath(File directory) throws IOException { + Map contents = new HashMap<>(4); + + final Collection filesInBucket = FileUtils.listFiles(directory, null, true); + for (File file : filesInBucket) { + contents.put(file, FileUtils.readFileToString(file)); + } + return contents; + } + @JsonPropertyOrder({ "city", "lat", diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/TableCsvFormatITCase.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/TableCsvFormatITCase.java new file mode 100644 index 0000000000000..d51fb9e87c1fb --- /dev/null +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/TableCsvFormatITCase.java @@ -0,0 +1,204 @@ +/* + * 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.flink.formats.csv; + +import org.apache.flink.formats.common.TimeFormats; +import org.apache.flink.table.planner.runtime.utils.TestData; +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.JsonPlanTestBase; + +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.utils.DateTimeUtils.toLocalDateTime; + +/** Tests for the CSV file format. */ +public class TableCsvFormatITCase extends JsonPlanTestBase { + + @Test + public void testProjectPushDown() throws Exception { + List data = Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello world"); + createSourceTable("MyTable", data, "a bigint", "b int not null", "c varchar"); + File sinkPath = createSinkTable("MySink", "a bigint", "c varchar"); + + String jsonPlan = tableEnv.getJsonPlan("insert into MySink select a, c from MyTable"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult(Arrays.asList("1,hi", "2,hello", "3,hello world"), sinkPath); + } + + @Test + public void testReadingMetadata() throws Exception { + createTestValuesSourceTable( + "MyTable", + JavaScalaConversionUtil.toJava(TestData.smallData3()), + new String[] {"a int", "b bigint", "m varchar metadata"}, + new HashMap() { + { + put("readable-metadata", "m:STRING"); + } + }); + + File sinkPath = createSinkTable("MySink", "a bigint", "m varchar"); + + String jsonPlan = tableEnv.getJsonPlan("insert into MySink select a, m from MyTable"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult(Arrays.asList("1,Hi", "2,Hello", "3,Hello world"), sinkPath); + } + + @Test + public void testFilterPushDown() throws Exception { + List data = Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello world"); + createSourceTable("MyTable", data, "a bigint", "b int not null", "c varchar"); + File sinkPath = createSinkTable("MySink", "a bigint", "b int", "c varchar"); + + String jsonPlan = + tableEnv.getJsonPlan("insert into MySink select * from MyTable where a > 1"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult(Arrays.asList("2,1,hello", "3,2,hello world"), sinkPath); + } + + @Test + public void testPartitionPushDown() throws Exception { + createTestValuesSourceTable( + "MyTable", + JavaScalaConversionUtil.toJava(TestData.smallData3()), + new String[] {"a int", "p bigint", "c varchar"}, + "p", + new HashMap() { + { + put("partition-list", "p:1;p:2"); + } + }); + File sinkPath = createSinkTable("MySink", "a int", "p bigint", "c varchar"); + + String jsonPlan = + tableEnv.getJsonPlan("insert into MySink select * from MyTable where p = 2"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult(Arrays.asList("2,2,Hello", "3,2,Hello world"), sinkPath); + } + + @Test + public void testWatermarkPushDown() throws Exception { + createTestValuesSourceTable( + "MyTable", + JavaScalaConversionUtil.toJava(TestData.data3WithTimestamp()), + new String[] { + "a int", + "b bigint", + "c varchar", + "ts timestamp(3)", + "watermark for ts as ts - interval '5' second" + }, + new HashMap() { + { + put("enable-watermark-push-down", "true"); + } + }); + + File sinkPath = createSinkTable("MySink", "a int", "b bigint", "ts timestamp(3)"); + + String jsonPlan = + tableEnv.getJsonPlan("insert into MySink select a, b, ts from MyTable where b = 3"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult( + Arrays.asList( + "4,3," + formatSqlTimestamp(4000L), + "5,3," + formatSqlTimestamp(5000L), + "6,3," + formatSqlTimestamp(6000L)), + sinkPath); + } + + @Test + public void testPushDowns() throws Exception { + createTestValuesSourceTable( + "MyTable", + JavaScalaConversionUtil.toJava(TestData.data3WithTimestamp()), + new String[] { + "a int", + "b bigint", + "c varchar", + "ts timestamp(3)", + "watermark for ts as ts - interval '5' second" + }, + "b", + new HashMap() { + { + put("readable-metadata", "a:INT"); + put("filterable-fields", "a"); + put("enable-watermark-push-down", "true"); + put("partition-list", "b:1;b:2;b:3;b:4;b:5;b:6"); + } + }); + + File sinkPath = createSinkTable("MySink", "a int", "ts timestamp(3)"); + + String jsonPlan = + tableEnv.getJsonPlan( + "insert into MySink select a, ts from MyTable where b = 3 and a > 4"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult( + Arrays.asList("5," + formatSqlTimestamp(5000L), "6," + formatSqlTimestamp(6000L)), + sinkPath); + } + + private static String formatSqlTimestamp(long timestamp) { + return TimeFormats.SQL_TIMESTAMP_FORMAT.format(toLocalDateTime(timestamp)); + } + + private void createSourceTable(String tableName, List data, String... fieldNameAndTypes) + throws IOException { + File sourceFile = TEMPORARY_FOLDER.newFile(); + Collections.shuffle(data); + Files.write(sourceFile.toPath(), String.join("\n", data).getBytes()); + + Map properties = new HashMap<>(); + properties.put("connector", "filesystem"); + properties.put("path", sourceFile.getAbsolutePath()); + properties.put("format", "csv"); + + createTestSourceTable(tableName, fieldNameAndTypes, null, properties); + } + + private File createSinkTable(String tableName, String... fieldNameAndTypes) throws IOException { + File sinkPath = TEMPORARY_FOLDER.newFolder(); + + Map properties = new HashMap<>(); + properties.put("connector", "filesystem"); + properties.put("path", sinkPath.getAbsolutePath()); + properties.put("format", "csv"); + properties.put("csv.disable-quote-character", "true"); + + createTestSinkTable(tableName, fieldNameAndTypes, null, properties); + return sinkPath; + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java index 7bda9f3fbcfaf..71089c589dda6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java @@ -92,10 +92,7 @@ protected void createTestValuesSourceTable( @Nullable String partitionFields, Map extraProperties) { checkArgument(fieldNameAndTypes.length > 0); - String partitionedBy = - StringUtils.isNullOrWhitespaceOnly(partitionFields) - ? "" - : "\n partitioned by (" + partitionFields + ") \n"; + String dataId = TestValuesTableFactory.registerData(data); Map properties = new HashMap<>(); properties.put("connector", "values"); @@ -103,6 +100,19 @@ protected void createTestValuesSourceTable( properties.put("bounded", "true"); properties.put("disable-lookup", "true"); properties.putAll(extraProperties); + createTestSourceTable(tableName, fieldNameAndTypes, partitionFields, properties); + } + + protected void createTestSourceTable( + String tableName, + String[] fieldNameAndTypes, + @Nullable String partitionFields, + Map properties) { + checkArgument(fieldNameAndTypes.length > 0); + String partitionedBy = + StringUtils.isNullOrWhitespaceOnly(partitionFields) + ? "" + : "\n partitioned by (" + partitionFields + ") \n"; String ddl = String.format( "CREATE TABLE %s (\n" + "%s\n" + ") %s with (\n%s)", @@ -140,15 +150,26 @@ protected void createTestValuesSinkTable( String tableName, String[] fieldNameAndTypes, @Nullable String partitionFields, - Map extraProperties) { + Map properties) { + + Map extraProperties = new HashMap<>(); + extraProperties.put("connector", "values"); + + properties.putAll(extraProperties); + + createTestSinkTable(tableName, fieldNameAndTypes, partitionFields, properties); + } + + protected void createTestSinkTable( + String tableName, + String[] fieldNameAndTypes, + @Nullable String partitionFields, + Map properties) { checkArgument(fieldNameAndTypes.length > 0); String partitionedBy = StringUtils.isNullOrWhitespaceOnly(partitionFields) ? "" : "\n partitioned by (" + partitionFields + ") \n"; - Map properties = new HashMap<>(); - properties.put("connector", "values"); - properties.putAll(extraProperties); String ddl = String.format( "CREATE TABLE %s (\n" + "%s\n" + ") %s with (\n%s)", From 8d5a4dd5ba8f74935524787335112885250a6da0 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 22 Dec 2021 13:25:16 +0100 Subject: [PATCH 10/49] [hotfix] Adds fail to DispatcherTest --- .../java/org/apache/flink/runtime/dispatcher/DispatcherTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 4f9985f61f90a..bc5687bd80288 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -758,6 +758,7 @@ public void clearJob(JobID jobID) { queue.offer(Optional.of(testException)); try { dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + fail("A FlinkException is expected"); } catch (Throwable expectedException) { assertThat(expectedException, containsCause(FlinkException.class)); assertThat(expectedException, containsMessage(testException.getMessage())); From bdb365741c0412447aa95672aa2d562a15c10bb9 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 15 Nov 2021 11:17:41 +0100 Subject: [PATCH 11/49] [hotfix] Adds missing JavaDoc --- .../org/apache/flink/runtime/testutils/TestingJobGraphStore.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java index 4c31f78554699..e9a797f0f47f2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java @@ -141,6 +141,7 @@ public static Builder newBuilder() { return new Builder(); } + /** {@code Builder} for creating {@code TestingJobGraphStore} instances. */ public static class Builder { private ThrowingConsumer startConsumer = ignored -> {}; From 38148676715371a42ab9e8f990b77da1cea4d602 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 15 Nov 2021 10:42:02 +0100 Subject: [PATCH 12/49] [hotfix] Makes intention of comment clearer --- .../main/java/org/apache/flink/runtime/blob/BlobServer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java index 24be572862e2d..944031eabdec0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java @@ -885,7 +885,8 @@ public boolean cleanupJob(JobID jobId, boolean cleanupBlobStoreFiles) { try { FileUtils.deleteDirectory(jobDir); - // NOTE: Instead of going through blobExpiryTimes, keep lingering entries - they + // NOTE on why blobExpiryTimes are not cleaned up: + // Instead of going through blobExpiryTimes, keep lingering entries - they // will be cleaned up by the timer task which tolerates non-existing files // If inserted again with the same IDs (via put()), the TTL will be updated // again. From cfeaff195dd19f7a9544f927e7f99764ebbe4431 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 30 Dec 2021 15:14:36 +0100 Subject: [PATCH 13/49] [hotfix] Migrates ApplicationStatusTest to JUnit5 and AssertJ --- .../clusterframework/ApplicationStatus.java | 2 +- .../ApplicationStatusTest.java | 20 +++++++++---------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java index a971e69a456d6..16606966938e8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java @@ -40,7 +40,7 @@ public enum ApplicationStatus { /** The associated process exit code */ private final int processExitCode; - private ApplicationStatus(int exitCode) { + ApplicationStatus(int exitCode) { this.processExitCode = exitCode; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java index 9c164c2bd4fcf..24f1f70f0ffbd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java @@ -18,41 +18,39 @@ package org.apache.flink.runtime.clusterframework; -import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.not; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link ApplicationStatus}. */ -public class ApplicationStatusTest extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +public class ApplicationStatusTest { private static final int SUCCESS_EXIT_CODE = 0; @Test public void succeededStatusMapsToSuccessExitCode() { int exitCode = ApplicationStatus.SUCCEEDED.processExitCode(); - assertThat(exitCode, is(equalTo(SUCCESS_EXIT_CODE))); + assertThat(exitCode).isEqualTo(SUCCESS_EXIT_CODE); } @Test public void cancelledStatusMapsToSuccessExitCode() { int exitCode = ApplicationStatus.CANCELED.processExitCode(); - assertThat(exitCode, is(equalTo(SUCCESS_EXIT_CODE))); + assertThat(exitCode).isEqualTo(SUCCESS_EXIT_CODE); } @Test public void notSucceededNorCancelledStatusMapsToNonSuccessExitCode() { Iterable exitCodes = exitCodes(notSucceededNorCancelledStatus()); - assertThat(exitCodes, not(contains(SUCCESS_EXIT_CODE))); + assertThat(exitCodes).doesNotContain(SUCCESS_EXIT_CODE); } private static Iterable exitCodes(Iterable statuses) { From bf1a3108fd2bf8d59ca2f06d83f2bf2a355dc75b Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 21 Jan 2022 17:33:49 +0100 Subject: [PATCH 14/49] [hotfix] Adds TestLogger extension to ApplicationDispatcherBootstrapTest --- .../application/ApplicationDispatcherBootstrapTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java index f6fd8db2eea21..b9f1111db599e 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java @@ -45,7 +45,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.SerializedThrowable; -import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ScheduledExecutor; import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; @@ -53,6 +53,7 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -79,7 +80,8 @@ import static org.junit.jupiter.api.Assertions.fail; /** Tests for the {@link ApplicationDispatcherBootstrap}. */ -public class ApplicationDispatcherBootstrapTest extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +public class ApplicationDispatcherBootstrapTest { private static final int TIMEOUT_SECONDS = 10; From 46952ef7e7c41c021bb8e69fabe4f4e4fa31ecbd Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 21 Jan 2022 15:30:12 +0100 Subject: [PATCH 15/49] [hotfix] Introduces TestingPartialDispatcherServices --- .../TestingPartialDispatcherServices.java | 184 ++++++++++++++++++ .../runner/DefaultDispatcherRunnerITCase.java | 24 +-- 2 files changed, 188 insertions(+), 20 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingPartialDispatcherServices.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingPartialDispatcherServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingPartialDispatcherServices.java new file mode 100644 index 0000000000000..ba986291aabec --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingPartialDispatcherServices.java @@ -0,0 +1,184 @@ +/* + * 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.flink.runtime.dispatcher; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.BlobStore; +import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; + +/** + * {@code TestingPartialDispatcherServices} implements {@link PartialDispatcherServices} to be used + * in test contexts. + */ +public class TestingPartialDispatcherServices extends PartialDispatcherServices { + public TestingPartialDispatcherServices( + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + JobManagerMetricGroupFactory jobManagerMetricGroupFactory, + ExecutionGraphInfoStore executionGraphInfoStore, + FatalErrorHandler fatalErrorHandler, + HistoryServerArchivist historyServerArchivist, + @Nullable String metricQueryServiceAddress, + Executor ioExecutor, + DispatcherOperationCaches operationCaches) { + super( + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + blobServer, + heartbeatServices, + jobManagerMetricGroupFactory, + executionGraphInfoStore, + fatalErrorHandler, + historyServerArchivist, + metricQueryServiceAddress, + ioExecutor, + operationCaches); + } + + public static Builder builder() { + return new Builder(); + } + + /** {@code Builder} for creating {@code TestingPartialDispatcherServices} instances. */ + public static class Builder { + + private HighAvailabilityServices highAvailabilityServices = + new TestingHighAvailabilityServicesBuilder().build(); + private GatewayRetriever resourceManagerGatewayRetriever = + CompletableFuture::new; + private BlobStore blobStore = new TestingBlobStoreBuilder().createTestingBlobStore(); + private HeartbeatServices heartbeatServices = new TestingHeartbeatServices(); + private JobManagerMetricGroupFactory jobManagerMetricGroupFactory = + UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup; + private ExecutionGraphInfoStore executionGraphInfoStore = + new MemoryExecutionGraphInfoStore(); + private FatalErrorHandler fatalErrorHandler = NoOpFatalErrorHandler.INSTANCE; + private HistoryServerArchivist historyServerArchivist = VoidHistoryServerArchivist.INSTANCE; + @Nullable private String metricQueryServiceAddress = null; + private DispatcherOperationCaches operationCaches = new DispatcherOperationCaches(); + private Executor ioExecutor = ForkJoinPool.commonPool(); + + private Builder() {} + + public Builder withHighAvailabilityServices( + HighAvailabilityServices highAvailabilityServices) { + this.highAvailabilityServices = highAvailabilityServices; + return this; + } + + public Builder withResourceManagerGatewayRetriever( + GatewayRetriever resourceManagerGatewayRetriever) { + this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; + return this; + } + + public Builder withBlobStore(BlobStore blobStore) { + this.blobStore = blobStore; + return this; + } + + public Builder withHeartbeatServices(HeartbeatServices heartbeatServices) { + this.heartbeatServices = heartbeatServices; + return this; + } + + public Builder withJobManagerMetricGroupFactory( + JobManagerMetricGroupFactory jobManagerMetricGroupFactory) { + this.jobManagerMetricGroupFactory = jobManagerMetricGroupFactory; + return this; + } + + public Builder withExecutionGraphInfoStore( + ExecutionGraphInfoStore executionGraphInfoStore) { + this.executionGraphInfoStore = executionGraphInfoStore; + return this; + } + + public Builder withFatalErrorHandler(FatalErrorHandler fatalErrorHandler) { + this.fatalErrorHandler = fatalErrorHandler; + return this; + } + + public Builder withHistoryServerArchivist(HistoryServerArchivist historyServerArchivist) { + this.historyServerArchivist = historyServerArchivist; + return this; + } + + public Builder withMetricQueryServiceAddress(@Nullable String metricQueryServiceAddress) { + this.metricQueryServiceAddress = metricQueryServiceAddress; + return this; + } + + public Builder withOperationCaches(DispatcherOperationCaches operationCaches) { + this.operationCaches = operationCaches; + return this; + } + + public Builder withIoExecutor(Executor ioExecutor) { + this.ioExecutor = ioExecutor; + return this; + } + + public TestingPartialDispatcherServices build(File storageDir, Configuration configuration) + throws IOException { + try (BlobServer blobServer = new BlobServer(configuration, storageDir, blobStore)) { + return build(blobServer, configuration); + } + } + + public TestingPartialDispatcherServices build( + BlobServer blobServer, Configuration configuration) { + return new TestingPartialDispatcherServices( + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + blobServer, + heartbeatServices, + jobManagerMetricGroupFactory, + executionGraphInfoStore, + fatalErrorHandler, + historyServerArchivist, + metricQueryServiceAddress, + ioExecutor, + operationCaches); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index 03bf6dff8c3ee..64790d31d5c87 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -26,25 +26,20 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; -import org.apache.flink.runtime.dispatcher.DispatcherOperationCaches; import org.apache.flink.runtime.dispatcher.DispatcherServices; import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; -import org.apache.flink.runtime.dispatcher.MemoryExecutionGraphInfoStore; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; -import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; -import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.dispatcher.TestingPartialDispatcherServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; import org.apache.flink.runtime.testutils.TestingJobGraphStore; @@ -66,7 +61,6 @@ import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import static org.hamcrest.Matchers.contains; @@ -110,19 +104,9 @@ public void setup() { jobGraphStore = TestingJobGraphStore.newBuilder().build(); partialDispatcherServices = - new PartialDispatcherServices( - new Configuration(), - new TestingHighAvailabilityServicesBuilder().build(), - CompletableFuture::new, - blobServerResource.getBlobServer(), - new TestingHeartbeatServices(), - UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup, - new MemoryExecutionGraphInfoStore(), - fatalErrorHandler, - VoidHistoryServerArchivist.INSTANCE, - null, - ForkJoinPool.commonPool(), - new DispatcherOperationCaches()); + TestingPartialDispatcherServices.builder() + .withFatalErrorHandler(fatalErrorHandler) + .build(blobServerResource.getBlobServer(), new Configuration()); } @After From 8d235d86e1504863843f60d775c0cd929c224219 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 29 Nov 2021 18:56:51 +0100 Subject: [PATCH 16/49] [hotfix] Removes @Nonnull annotations from DispatcherServices and PartialDispatcherServicesWithJobGraphStore Instead, null checks are added --- .../dispatcher/DispatcherServices.java | 107 ++++++++---------- ...alDispatcherServicesWithJobGraphStore.java | 28 +++-- 2 files changed, 63 insertions(+), 72 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java index b18321c44e69a..9f19e7c0fb85e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java @@ -27,8 +27,8 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.Preconditions; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.concurrent.Executor; @@ -36,106 +36,104 @@ /** {@link Dispatcher} services container. */ public class DispatcherServices { - @Nonnull private final Configuration configuration; + private final Configuration configuration; - @Nonnull private final HighAvailabilityServices highAvailabilityServices; + private final HighAvailabilityServices highAvailabilityServices; - @Nonnull private final GatewayRetriever resourceManagerGatewayRetriever; + private final GatewayRetriever resourceManagerGatewayRetriever; - @Nonnull private final BlobServer blobServer; + private final BlobServer blobServer; - @Nonnull private final HeartbeatServices heartbeatServices; + private final HeartbeatServices heartbeatServices; - @Nonnull private final JobManagerMetricGroup jobManagerMetricGroup; + private final JobManagerMetricGroup jobManagerMetricGroup; - @Nonnull private final ExecutionGraphInfoStore executionGraphInfoStore; + private final ExecutionGraphInfoStore executionGraphInfoStore; - @Nonnull private final FatalErrorHandler fatalErrorHandler; + private final FatalErrorHandler fatalErrorHandler; - @Nonnull private final HistoryServerArchivist historyServerArchivist; + private final HistoryServerArchivist historyServerArchivist; @Nullable private final String metricQueryServiceAddress; - @Nonnull private final DispatcherOperationCaches operationCaches; + private final DispatcherOperationCaches operationCaches; - @Nonnull private final JobGraphWriter jobGraphWriter; + private final JobGraphWriter jobGraphWriter; - @Nonnull private final JobManagerRunnerFactory jobManagerRunnerFactory; + private final JobManagerRunnerFactory jobManagerRunnerFactory; - @Nonnull private final Executor ioExecutor; + private final Executor ioExecutor; DispatcherServices( - @Nonnull Configuration configuration, - @Nonnull HighAvailabilityServices highAvailabilityServices, - @Nonnull GatewayRetriever resourceManagerGatewayRetriever, - @Nonnull BlobServer blobServer, - @Nonnull HeartbeatServices heartbeatServices, - @Nonnull ExecutionGraphInfoStore executionGraphInfoStore, - @Nonnull FatalErrorHandler fatalErrorHandler, - @Nonnull HistoryServerArchivist historyServerArchivist, + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + ExecutionGraphInfoStore executionGraphInfoStore, + FatalErrorHandler fatalErrorHandler, + HistoryServerArchivist historyServerArchivist, @Nullable String metricQueryServiceAddress, - @Nonnull DispatcherOperationCaches operationCaches, - @Nonnull JobManagerMetricGroup jobManagerMetricGroup, - @Nonnull JobGraphWriter jobGraphWriter, - @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory, - @Nonnull Executor ioExecutor) { - this.configuration = configuration; - this.highAvailabilityServices = highAvailabilityServices; - this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; - this.blobServer = blobServer; - this.heartbeatServices = heartbeatServices; - this.executionGraphInfoStore = executionGraphInfoStore; - this.fatalErrorHandler = fatalErrorHandler; - this.historyServerArchivist = historyServerArchivist; + DispatcherOperationCaches operationCaches, + JobManagerMetricGroup jobManagerMetricGroup, + JobGraphWriter jobGraphWriter, + JobManagerRunnerFactory jobManagerRunnerFactory, + Executor ioExecutor) { + this.configuration = Preconditions.checkNotNull(configuration, "Configuration"); + this.highAvailabilityServices = + Preconditions.checkNotNull(highAvailabilityServices, "HighAvailabilityServices"); + this.resourceManagerGatewayRetriever = + Preconditions.checkNotNull( + resourceManagerGatewayRetriever, "ResourceManagerGatewayRetriever"); + this.blobServer = Preconditions.checkNotNull(blobServer, "BlobServer"); + this.heartbeatServices = Preconditions.checkNotNull(heartbeatServices, "HeartBeatServices"); + this.executionGraphInfoStore = + Preconditions.checkNotNull(executionGraphInfoStore, "ExecutionGraphInfoStore"); + this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler, "FatalErrorHandler"); + this.historyServerArchivist = + Preconditions.checkNotNull(historyServerArchivist, "HistoryServerArchivist"); this.metricQueryServiceAddress = metricQueryServiceAddress; - this.operationCaches = operationCaches; - this.jobManagerMetricGroup = jobManagerMetricGroup; - this.jobGraphWriter = jobGraphWriter; - this.jobManagerRunnerFactory = jobManagerRunnerFactory; - this.ioExecutor = ioExecutor; + this.operationCaches = Preconditions.checkNotNull(operationCaches, "OperationCaches"); + this.jobManagerMetricGroup = + Preconditions.checkNotNull(jobManagerMetricGroup, "JobManagerMetricGroup"); + this.jobGraphWriter = Preconditions.checkNotNull(jobGraphWriter, "JobGraphWriter"); + this.jobManagerRunnerFactory = + Preconditions.checkNotNull(jobManagerRunnerFactory, "JobManagerRunnerFactory"); + this.ioExecutor = Preconditions.checkNotNull(ioExecutor, "IOExecutor"); } - @Nonnull public Configuration getConfiguration() { return configuration; } - @Nonnull public HighAvailabilityServices getHighAvailabilityServices() { return highAvailabilityServices; } - @Nonnull public GatewayRetriever getResourceManagerGatewayRetriever() { return resourceManagerGatewayRetriever; } - @Nonnull public BlobServer getBlobServer() { return blobServer; } - @Nonnull public HeartbeatServices getHeartbeatServices() { return heartbeatServices; } - @Nonnull public JobManagerMetricGroup getJobManagerMetricGroup() { return jobManagerMetricGroup; } - @Nonnull public ExecutionGraphInfoStore getArchivedExecutionGraphStore() { return executionGraphInfoStore; } - @Nonnull public FatalErrorHandler getFatalErrorHandler() { return fatalErrorHandler; } - @Nonnull public HistoryServerArchivist getHistoryServerArchivist() { return historyServerArchivist; } @@ -145,31 +143,26 @@ public String getMetricQueryServiceAddress() { return metricQueryServiceAddress; } - @Nonnull public DispatcherOperationCaches getOperationCaches() { return operationCaches; } - @Nonnull public JobGraphWriter getJobGraphWriter() { return jobGraphWriter; } - @Nonnull JobManagerRunnerFactory getJobManagerRunnerFactory() { return jobManagerRunnerFactory; } - @Nonnull public Executor getIoExecutor() { return ioExecutor; } public static DispatcherServices from( - @Nonnull - PartialDispatcherServicesWithJobGraphStore - partialDispatcherServicesWithJobGraphStore, - @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { + PartialDispatcherServicesWithJobGraphStore + partialDispatcherServicesWithJobGraphStore, + JobManagerRunnerFactory jobManagerRunnerFactory) { return new DispatcherServices( partialDispatcherServicesWithJobGraphStore.getConfiguration(), partialDispatcherServicesWithJobGraphStore.getHighAvailabilityServices(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java index c3fbecdc9c2b2..038fcb871542f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java @@ -27,7 +27,6 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.concurrent.Executor; @@ -35,22 +34,22 @@ /** {@link DispatcherFactory} services container. */ public class PartialDispatcherServicesWithJobGraphStore extends PartialDispatcherServices { - @Nonnull private final JobGraphWriter jobGraphWriter; + private final JobGraphWriter jobGraphWriter; private PartialDispatcherServicesWithJobGraphStore( - @Nonnull Configuration configuration, - @Nonnull HighAvailabilityServices highAvailabilityServices, - @Nonnull GatewayRetriever resourceManagerGatewayRetriever, - @Nonnull BlobServer blobServer, - @Nonnull HeartbeatServices heartbeatServices, - @Nonnull JobManagerMetricGroupFactory jobManagerMetricGroupFactory, - @Nonnull ExecutionGraphInfoStore executionGraphInfoStore, - @Nonnull FatalErrorHandler fatalErrorHandler, - @Nonnull HistoryServerArchivist historyServerArchivist, + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + JobManagerMetricGroupFactory jobManagerMetricGroupFactory, + ExecutionGraphInfoStore executionGraphInfoStore, + FatalErrorHandler fatalErrorHandler, + HistoryServerArchivist historyServerArchivist, @Nullable String metricQueryServiceAddress, - @Nonnull Executor ioExecutor, - @Nonnull DispatcherOperationCaches operationCaches, - @Nonnull JobGraphWriter jobGraphWriter) { + Executor ioExecutor, + DispatcherOperationCaches operationCaches, + JobGraphWriter jobGraphWriter) { super( configuration, highAvailabilityServices, @@ -67,7 +66,6 @@ private PartialDispatcherServicesWithJobGraphStore( this.jobGraphWriter = jobGraphWriter; } - @Nonnull public JobGraphWriter getJobGraphWriter() { return jobGraphWriter; } From 5c39a7ea2c538d0aa81533f708fb8b5e7fb77dd4 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 25 Jan 2022 15:40:47 +0100 Subject: [PATCH 17/49] [FLINK-25430][runtime] Add JobResultStore This commit adds the JobResultStore interface, a in-memory implementation EmbeddedJobResultStoreand, and corresponding test, container and utility classes. (FLIP-194) --- .../highavailability/JobResultEntry.java | 44 ++++ .../highavailability/JobResultStore.java | 111 ++++++++++ .../embedded/EmbeddedJobResultStore.java | 131 +++++++++++ .../embedded/EmbeddedJobResultStoreTest.java | 204 ++++++++++++++++++ .../testutils/TestingJobResultStore.java | 184 ++++++++++++++++ 5 files changed, 674 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultEntry.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStoreTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultEntry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultEntry.java new file mode 100644 index 0000000000000..d4cda9a9033ac --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultEntry.java @@ -0,0 +1,44 @@ +/* + * 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.flink.runtime.highavailability; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.Preconditions; + +/** + * {@code JobResultEntry} is the entity managed by the {@link JobResultStore}. It collects + * information about a globally terminated job (e.g. {@link JobResult}). + */ +public class JobResultEntry { + + private final JobResult jobResult; + + public JobResultEntry(JobResult jobResult) { + this.jobResult = Preconditions.checkNotNull(jobResult); + } + + public JobResult getJobResult() { + return jobResult; + } + + public JobID getJobId() { + return jobResult.getJobId(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java new file mode 100644 index 0000000000000..077826760f5f8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java @@ -0,0 +1,111 @@ +/* + * 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.flink.runtime.highavailability; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobmaster.JobResult; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.Set; + +/** + * A storage for the results of globally terminated jobs. These results can have the following + * states: + * + *
    + *
  • {@code dirty} - indicating that the corresponding job is not properly cleaned up, yet. + *
  • {@code clean} - indicating that the cleanup of the corresponding job is performed and no + * further actions need to be applied. + *
+ */ +@Internal +public interface JobResultStore { + + /** + * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that + * clean-up operations still need to be performed. Once the job resource cleanup has been + * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link + * #markResultAsClean(JobID)}. + * + * @param jobResultEntry The job result we wish to persist. + * @throws IOException if the creation of the dirty result failed for IO reasons. + * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID} + * attached that is already registered in this {@code JobResultStore}. + */ + void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException; + + /** + * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more + * resource cleanup steps need to be performed. No actions should be triggered if the passed + * {@code JobID} belongs to a job that was already marked as clean. + * + * @param jobId Ident of the job we wish to mark as clean. + * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean} + * failed for IO reasons. + * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the + * store for the given {@code JobID}. + */ + void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException; + + /** + * Returns whether the store already contains an entry for a job. + * + * @param jobId Ident of the job we wish to check the store for. + * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for + * the given {@code JobID}; otherwise {@code false}. + * @throws IOException if determining whether a job entry is present in the store failed for IO + * reasons. + */ + default boolean hasJobResultEntry(JobID jobId) throws IOException { + return hasDirtyJobResultEntry(jobId) || hasCleanJobResultEntry(jobId); + } + + /** + * Returns whether the store already contains a {@code dirty} entry for the given {@code JobID}. + * + * @param jobId Ident of the job we wish to check the store for. + * @return {@code true}, if a {@code dirty} entry exists for the given {@code JobID}; otherwise + * {@code false}. + * @throws IOException if determining whether a job entry is present in the store failed for IO + * reasons. + */ + boolean hasDirtyJobResultEntry(JobID jobId) throws IOException; + + /** + * Returns whether the store already contains a {@code clean} entry for the given {@code JobID}. + * + * @param jobId Ident of the job we wish to check the store for. + * @return {@code true}, if a {@code clean} entry exists for the given {@code JobID}; otherwise + * {@code false}. + * @throws IOException if determining whether a job entry is present in the store failed for IO + * reasons. + */ + boolean hasCleanJobResultEntry(JobID jobId) throws IOException; + + /** + * Get the persisted {@link JobResult} instances that are marked as {@code dirty}. This is + * useful for recovery of finalization steps. + * + * @return A set of dirty {@code JobResults} from the store. + * @throws IOException if collecting the set of dirty results failed for IO reasons. + */ + Set getDirtyResults() throws IOException; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java new file mode 100644 index 0000000000000..1f54fd62c4cc3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java @@ -0,0 +1,131 @@ +/* + * 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.flink.runtime.highavailability.nonha.embedded; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; + +import java.util.HashMap; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** A thread-safe in-memory implementation of the {@link JobResultStore}. */ +public class EmbeddedJobResultStore implements JobResultStore { + + private static final Logger LOG = LoggerFactory.getLogger(EmbeddedJobResultStore.class); + + private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + + @GuardedBy("readWriteLock") + @VisibleForTesting + final Map dirtyJobResults = new HashMap<>(); + + @GuardedBy("readWriteLock") + @VisibleForTesting + final Map cleanJobResults = new HashMap<>(); + + @Override + public void createDirtyResult(JobResultEntry jobResultEntry) { + Preconditions.checkState( + !hasJobResultEntry(jobResultEntry.getJobId()), + "There is already a job registered under the passed ID {}.", + jobResultEntry.getJobId()); + + withWriteLock(() -> dirtyJobResults.put(jobResultEntry.getJobId(), jobResultEntry)); + } + + @Override + public void markResultAsClean(JobID jobId) throws NoSuchElementException { + if (hasCleanJobResultEntry(jobId)) { + LOG.debug("The job {} is already marked as clean. No action required.", jobId); + + return; + } + + withWriteLock( + () -> { + final JobResultEntry jobResultEntry = dirtyJobResults.remove(jobId); + if (jobResultEntry != null) { + cleanJobResults.put(jobId, jobResultEntry); + } else { + throw new NoSuchElementException( + String.format( + "Could not mark job %s as clean as it is not present in the job result store.", + jobId)); + } + }); + } + + @Override + public boolean hasJobResultEntry(JobID jobId) { + return withReadLock( + () -> dirtyJobResults.containsKey(jobId) || cleanJobResults.containsKey(jobId)); + } + + @Override + public boolean hasDirtyJobResultEntry(JobID jobId) { + return withReadLock(() -> dirtyJobResults.containsKey(jobId)); + } + + @Override + public boolean hasCleanJobResultEntry(JobID jobId) { + return withReadLock(() -> cleanJobResults.containsKey(jobId)); + } + + @Override + public Set getDirtyResults() { + return withReadLock( + () -> + dirtyJobResults.values().stream() + .map(JobResultEntry::getJobResult) + .collect(Collectors.toSet())); + } + + private void withWriteLock(Runnable callback) { + readWriteLock.writeLock().lock(); + try { + callback.run(); + } finally { + readWriteLock.writeLock().unlock(); + } + } + + private T withReadLock(Supplier callback) { + readWriteLock.readLock().lock(); + try { + return callback.get(); + } finally { + readWriteLock.readLock().unlock(); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStoreTest.java new file mode 100644 index 0000000000000..45644dc4381a5 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStoreTest.java @@ -0,0 +1,204 @@ +/* + * 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.flink.runtime.highavailability.nonha.embedded; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.testutils.TestingJobResultStore; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.NoSuchElementException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.entry; + +/** Tests for the {@link EmbeddedJobResultStore}. */ +@ExtendWith(TestLoggerExtension.class) +public class EmbeddedJobResultStoreTest { + + private static final JobResultEntry DUMMY_JOB_RESULT_ENTRY = + new JobResultEntry(TestingJobResultStore.DUMMY_JOB_RESULT); + + private EmbeddedJobResultStore embeddedJobResultStore; + + @BeforeEach + public void setupTest() { + embeddedJobResultStore = new EmbeddedJobResultStore(); + } + + @Test + public void testStoreDirtyJobResult() throws Exception { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + assertThat(embeddedJobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + + assertThat(embeddedJobResultStore.dirtyJobResults) + .containsExactly(entry(DUMMY_JOB_RESULT_ENTRY.getJobId(), DUMMY_JOB_RESULT_ENTRY)); + assertThat(embeddedJobResultStore.cleanJobResults).isEmpty(); + } + + @Test + public void testStoreDirtyJobResultTwice() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + + assertThatThrownBy(() -> embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY)) + .isInstanceOf(IllegalStateException.class); + } + + @Test + public void testStoreDirtyJobResultForCleanJobEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThatThrownBy(() -> embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY)) + .isInstanceOf(IllegalStateException.class); + } + + @Test + public void testCleanDirtyJobResult() throws Exception { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThat(embeddedJobResultStore.dirtyJobResults).isEmpty(); + assertThat(embeddedJobResultStore.cleanJobResults) + .containsExactly(entry(DUMMY_JOB_RESULT_ENTRY.getJobId(), DUMMY_JOB_RESULT_ENTRY)); + } + + @Test + public void testCleanDirtyJobResultTwice() { + final JobID jobId = DUMMY_JOB_RESULT_ENTRY.getJobId(); + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(jobId); + + assertThat(embeddedJobResultStore.cleanJobResults) + .containsExactly(entry(jobId, DUMMY_JOB_RESULT_ENTRY)); + embeddedJobResultStore.markResultAsClean(jobId); + assertThat(embeddedJobResultStore.cleanJobResults) + .as("Marking the same job %s as clean should be idempotent.", jobId) + .containsExactly(entry(jobId, DUMMY_JOB_RESULT_ENTRY)); + } + + @Test + public void testCleanNonExistentJobResult() throws Exception { + assertThatThrownBy(() -> embeddedJobResultStore.markResultAsClean(new JobID())) + .isInstanceOf(NoSuchElementException.class); + } + + @Test + public void testHasJobResultEntryWithNoEntry() { + assertThat(embeddedJobResultStore.hasJobResultEntry(new JobID())).isFalse(); + } + + @Test + public void testHasJobResultEntryWithDirtyEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + + assertThat(embeddedJobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + } + + @Test + public void testHasJobResultEntryWithCleanEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThat(embeddedJobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + } + + @Test + public void testHasDirtyJobResultEntryWithNoDirtyEntry() { + assertThat(embeddedJobResultStore.hasDirtyJobResultEntry(new JobID())).isFalse(); + } + + @Test + public void testHasDirtyJobResultEntryWithDirtyEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + + assertThat(embeddedJobResultStore.hasDirtyJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + } + + @Test + public void testHasDirtyJobResultEntryWithCleanEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThat(embeddedJobResultStore.hasDirtyJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isFalse(); + } + + @Test + public void testHasCleanJobResultEntryWithNoEntry() { + assertThat(embeddedJobResultStore.hasCleanJobResultEntry(new JobID())).isFalse(); + } + + @Test + public void testHasCleanJobResultEntryWithDirtyEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + assertThat(embeddedJobResultStore.hasCleanJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isFalse(); + } + + @Test + public void testHasCleanJobResultEntryWithCleanEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThat(embeddedJobResultStore.hasCleanJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + } + + @Test + public void testGetDirtyResultsWithNoEntry() { + assertThat(embeddedJobResultStore.getDirtyResults()).isEmpty(); + } + + @Test + public void testGetDirtyResultsWithDirtyEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + + assertThat(embeddedJobResultStore.getDirtyResults()) + .containsExactlyInAnyOrder(DUMMY_JOB_RESULT_ENTRY.getJobResult()); + } + + @Test + public void testGetDirtyResultsWithDirtyAndCleanEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + final JobResultEntry otherDirtyJobResultEntry = + new JobResultEntry(TestingJobResultStore.createSuccessfulJobResult(new JobID())); + embeddedJobResultStore.createDirtyResult(otherDirtyJobResultEntry); + + assertThat(embeddedJobResultStore.dirtyJobResults) + .containsExactly( + entry(otherDirtyJobResultEntry.getJobId(), otherDirtyJobResultEntry)); + assertThat(embeddedJobResultStore.cleanJobResults) + .containsExactly(entry(DUMMY_JOB_RESULT_ENTRY.getJobId(), DUMMY_JOB_RESULT_ENTRY)); + + assertThat(embeddedJobResultStore.getDirtyResults()) + .containsExactlyInAnyOrder(otherDirtyJobResultEntry.getJobResult()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java new file mode 100644 index 0000000000000..a73759196f49d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java @@ -0,0 +1,184 @@ +/* + * 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.flink.runtime.testutils; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.SupplierWithException; +import org.apache.flink.util.function.ThrowingConsumer; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; + +/** + * {@code TestingJobResultStore} is a {@link JobResultStore} implementation that can be used in + * tests. + */ +public class TestingJobResultStore implements JobResultStore { + + public static final JobResult DUMMY_JOB_RESULT = createSuccessfulJobResult(new JobID()); + + public static JobResult createSuccessfulJobResult(JobID jobId) { + return createJobResult(jobId, ApplicationStatus.SUCCEEDED); + } + + public static JobResult createJobResult(JobID jobId, ApplicationStatus applicationStatus) { + return new JobResult.Builder() + .jobId(jobId) + .applicationStatus(applicationStatus) + .netRuntime(1) + .build(); + } + + private final ThrowingConsumer createDirtyResultConsumer; + private final ThrowingConsumer markResultAsCleanConsumer; + private final FunctionWithException + hasJobResultEntryFunction; + private final FunctionWithException + hasDirtyJobResultEntryFunction; + private final FunctionWithException + hasCleanJobResultEntryFunction; + private final SupplierWithException, ? extends IOException> + getDirtyResultsSupplier; + + private TestingJobResultStore( + ThrowingConsumer createDirtyResultConsumer, + ThrowingConsumer markResultAsCleanConsumer, + FunctionWithException hasJobResultEntryFunction, + FunctionWithException + hasDirtyJobResultEntryFunction, + FunctionWithException + hasCleanJobResultEntryFunction, + SupplierWithException, ? extends IOException> getDirtyResultsSupplier) { + this.createDirtyResultConsumer = createDirtyResultConsumer; + this.markResultAsCleanConsumer = markResultAsCleanConsumer; + this.hasJobResultEntryFunction = hasJobResultEntryFunction; + this.hasDirtyJobResultEntryFunction = hasDirtyJobResultEntryFunction; + this.hasCleanJobResultEntryFunction = hasCleanJobResultEntryFunction; + this.getDirtyResultsSupplier = getDirtyResultsSupplier; + } + + @Override + public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException { + createDirtyResultConsumer.accept(jobResultEntry); + } + + @Override + public void markResultAsClean(JobID jobId) throws IOException { + markResultAsCleanConsumer.accept(jobId); + } + + @Override + public boolean hasJobResultEntry(JobID jobId) throws IOException { + return hasJobResultEntryFunction.apply(jobId); + } + + @Override + public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException { + return hasDirtyJobResultEntryFunction.apply(jobId); + } + + @Override + public boolean hasCleanJobResultEntry(JobID jobId) throws IOException { + return hasCleanJobResultEntryFunction.apply(jobId); + } + + @Override + public Set getDirtyResults() throws IOException { + return getDirtyResultsSupplier.get(); + } + + public static TestingJobResultStore.Builder builder() { + return new Builder(); + } + + /** {@code Builder} for instantiating {@code TestingJobResultStore} instances. */ + public static class Builder { + + private ThrowingConsumer createDirtyResultConsumer = + ignored -> {}; + private ThrowingConsumer markResultAsCleanConsumer = + ignored -> {}; + + private FunctionWithException + hasJobResultEntryFunction = ignored -> false; + private FunctionWithException + hasDirtyJobResultEntryFunction = ignored -> false; + private FunctionWithException + hasCleanJobResultEntryFunction = ignored -> false; + + private SupplierWithException, ? extends IOException> + getDirtyResultsSupplier = Collections::emptySet; + + public Builder withCreateDirtyResultConsumer( + ThrowingConsumer createDirtyResultConsumer) { + this.createDirtyResultConsumer = createDirtyResultConsumer; + return this; + } + + public Builder withMarkResultAsCleanConsumer( + ThrowingConsumer markResultAsCleanConsumer) { + this.markResultAsCleanConsumer = markResultAsCleanConsumer; + return this; + } + + public Builder withHasJobResultEntryFunction( + FunctionWithException + hasJobResultEntryFunction) { + this.hasJobResultEntryFunction = hasJobResultEntryFunction; + return this; + } + + public Builder withHasDirtyJobResultEntryFunction( + FunctionWithException + hasDirtyJobResultEntryFunction) { + this.hasDirtyJobResultEntryFunction = hasDirtyJobResultEntryFunction; + return this; + } + + public Builder withHasCleanJobResultEntryFunction( + FunctionWithException + hasCleanJobResultEntryFunction) { + this.hasCleanJobResultEntryFunction = hasCleanJobResultEntryFunction; + return this; + } + + public Builder withGetDirtyResultsSupplier( + SupplierWithException, ? extends IOException> + getDirtyResultsSupplier) { + this.getDirtyResultsSupplier = getDirtyResultsSupplier; + return this; + } + + public TestingJobResultStore build() { + return new TestingJobResultStore( + createDirtyResultConsumer, + markResultAsCleanConsumer, + hasJobResultEntryFunction, + hasDirtyJobResultEntryFunction, + hasCleanJobResultEntryFunction, + getDirtyResultsSupplier); + } + } +} From 01b14fc4b9a9487a144f515bb7d4f6ad14cbe013 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 25 Jan 2022 16:30:40 +0100 Subject: [PATCH 18/49] [FLINK-25430][runtime] Integrates JobResultStoreinto the Dispatcher Integrates the JobResultStore (using the EmbeddedJobResultStore implementation) into the Dispatcher through HighAvailabilityServices. It replaces the RunningJobsRegistry but doesn't add any other functionality. --- .../ApplicationDispatcherBootstrapTest.java | 4 +- .../KubernetesHaServices.java | 10 +- ...ipleComponentLeaderElectionHaServices.java | 9 +- .../KubernetesRunningJobsRegistry.java | 157 ----------------- .../flink/kubernetes/utils/Constants.java | 1 - .../KubernetesRunningJobsRegistryTest.java | 123 ------------- .../flink/runtime/dispatcher/Dispatcher.java | 59 +++++-- ...bMasterServiceLeadershipRunnerFactory.java | 8 +- .../highavailability/AbstractHaServices.java | 27 ++- .../HighAvailabilityServices.java | 7 +- .../highavailability/RunningJobsRegistry.java | 92 ---------- .../nonha/AbstractNonHaServices.java | 12 +- .../StandaloneRunningJobsRegistry.java | 70 -------- .../AbstractZooKeeperHaServices.java | 10 +- .../ZooKeeperRunningJobsRegistry.java | 153 ---------------- .../JobMasterServiceLeadershipRunner.java | 59 ++----- .../dispatcher/AbstractDispatcherTest.java | 2 + .../dispatcher/DispatcherFailoverITCase.java | 6 +- .../DispatcherResourceCleanupTest.java | 163 ++++++++++++++---- .../runtime/dispatcher/DispatcherTest.java | 81 ++++----- .../runtime/dispatcher/TestingDispatcher.java | 10 +- .../ZooKeeperDefaultDispatcherRunnerTest.java | 3 - .../AbstractHaServicesTest.java | 18 +- .../TestingHighAvailabilityServices.java | 12 +- ...estingHighAvailabilityServicesBuilder.java | 11 +- ...TestingManualHighAvailabilityServices.java | 6 +- .../zookeeper/ZooKeeperHaServicesTest.java | 7 - .../zookeeper/ZooKeeperRegistryTest.java | 97 ----------- .../JobMasterServiceLeadershipRunnerTest.java | 21 ++- 29 files changed, 310 insertions(+), 928 deletions(-) delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistry.java delete mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistryTest.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java index b9f1111db599e..9d6ae78c5b9e1 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java @@ -643,7 +643,7 @@ public void testDuplicateJobSubmissionWithTerminatedJobId() throws Throwable { * In this scenario, job result is no longer present in the {@link * org.apache.flink.runtime.dispatcher.Dispatcher dispatcher} (job has terminated and job * manager failed over), but we know that job has already terminated from {@link - * org.apache.flink.runtime.highavailability.RunningJobsRegistry running jobs registry}. + * org.apache.flink.runtime.highavailability.JobResultStore}. */ @Test public void testDuplicateJobSubmissionWithTerminatedJobIdWithUnknownResult() throws Throwable { @@ -677,7 +677,7 @@ public void testDuplicateJobSubmissionWithTerminatedJobIdWithUnknownResult() thr * In this scenario, job result is no longer present in the {@link * org.apache.flink.runtime.dispatcher.Dispatcher dispatcher} (job has terminated and job * manager failed over), but we know that job has already terminated from {@link - * org.apache.flink.runtime.highavailability.RunningJobsRegistry running jobs registry}. + * org.apache.flink.runtime.highavailability.JobResultStore}. */ @Test public void testDuplicateJobSubmissionWithTerminatedJobIdWithUnknownResultAttached() diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java index e66973f872d6e..7f70e85f81fff 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.blob.BlobStoreService; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.AbstractHaServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -96,8 +96,8 @@ public class KubernetesHaServices extends AbstractHaServices { Executor executor, Configuration config, BlobStoreService blobStoreService) { + super(config, executor, blobStoreService, new EmbeddedJobResultStore()); - super(config, executor, blobStoreService); this.kubeClient = checkNotNull(kubeClient); this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID)); @@ -145,12 +145,6 @@ public JobGraphStore createJobGraphStore() throws Exception { configuration, kubeClient, getLeaderPathForDispatcher(), lockIdentity); } - @Override - public RunningJobsRegistry createRunningJobsRegistry() { - return new KubernetesRunningJobsRegistry( - kubeClient, getLeaderPathForDispatcher(), lockIdentity); - } - @Override public void internalClose() { configMapSharedWatcher.close(); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.java index 1a208d25b4a43..d1a313393a355 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.blob.BlobStoreService; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.AbstractHaServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService; import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService; @@ -83,7 +83,7 @@ public class KubernetesMultipleComponentLeaderElectionHaServices extends Abstrac BlobStoreService blobStoreService, FatalErrorHandler fatalErrorHandler) { - super(config, executor, blobStoreService); + super(config, executor, blobStoreService, new EmbeddedJobResultStore()); this.kubeClient = checkNotNull(kubeClient); this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID)); this.fatalErrorHandler = checkNotNull(fatalErrorHandler); @@ -167,11 +167,6 @@ private String getClusterConfigMap() { return clusterId + NAME_SEPARATOR + "cluster-config-map"; } - @Override - protected RunningJobsRegistry createRunningJobsRegistry() { - return new KubernetesRunningJobsRegistry(kubeClient, getClusterConfigMap(), lockIdentity); - } - @Override public void internalClose() throws Exception { Exception exception = null; diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistry.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistry.java deleted file mode 100644 index 7f75c3bbbc245..0000000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistry.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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.flink.kubernetes.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.util.StringUtils; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Optional; - -import static org.apache.flink.kubernetes.utils.Constants.RUNNING_JOBS_REGISTRY_KEY_PREFIX; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * {@link RunningJobsRegistry} implementation for Kubernetes. All the running jobs will be stored in - * Dispatcher-leader ConfigMap. The key is the job id with prefix {@link - * org.apache.flink.kubernetes.utils.Constants#RUNNING_JOBS_REGISTRY_KEY_PREFIX}, and value is job - * status. - */ -public class KubernetesRunningJobsRegistry implements RunningJobsRegistry { - - private static final Logger LOG = LoggerFactory.getLogger(KubernetesRunningJobsRegistry.class); - - private final FlinkKubeClient kubeClient; - - private final String configMapName; - - private final String lockIdentity; - - public KubernetesRunningJobsRegistry( - FlinkKubeClient kubeClient, String configMapName, String lockIdentity) { - this.kubeClient = checkNotNull(kubeClient); - this.configMapName = checkNotNull(configMapName); - this.lockIdentity = checkNotNull(lockIdentity); - } - - @Override - public void setJobRunning(JobID jobID) throws IOException { - checkNotNull(jobID); - - writeJobStatusToConfigMap(jobID, JobSchedulingStatus.RUNNING); - } - - @Override - public void setJobFinished(JobID jobID) throws IOException { - checkNotNull(jobID); - - writeJobStatusToConfigMap(jobID, JobSchedulingStatus.DONE); - } - - @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException { - checkNotNull(jobID); - - return kubeClient - .getConfigMap(configMapName) - .map( - configMap -> - getJobStatus(configMap, jobID).orElse(JobSchedulingStatus.PENDING)) - .orElseThrow( - () -> new IOException("ConfigMap " + configMapName + " does not exist.")); - } - - @Override - public void clearJob(JobID jobID) throws IOException { - checkNotNull(jobID); - - try { - kubeClient - .checkAndUpdateConfigMap( - configMapName, - configMap -> { - if (KubernetesLeaderElector.hasLeadership( - configMap, lockIdentity)) { - if (configMap.getData().remove(getKeyForJobId(jobID)) != null) { - return Optional.of(configMap); - } - } - return Optional.empty(); - }) - .get(); - } catch (Exception e) { - throw new IOException( - "Failed to clear job state in ConfigMap " + configMapName + " for job " + jobID, - e); - } - } - - private void writeJobStatusToConfigMap(JobID jobID, JobSchedulingStatus status) - throws IOException { - LOG.debug("Setting scheduling state for job {} to {}.", jobID, status); - final String key = getKeyForJobId(jobID); - try { - kubeClient - .checkAndUpdateConfigMap( - configMapName, - configMap -> { - if (KubernetesLeaderElector.hasLeadership( - configMap, lockIdentity)) { - final Optional optional = - getJobStatus(configMap, jobID); - if (!optional.isPresent() || optional.get() != status) { - configMap.getData().put(key, status.name()); - return Optional.of(configMap); - } - } - return Optional.empty(); - }) - .get(); - } catch (Exception e) { - throw new IOException( - "Failed to set " - + status.name() - + " state in ConfigMap " - + configMapName - + " for job " - + jobID, - e); - } - } - - private Optional getJobStatus(KubernetesConfigMap configMap, JobID jobId) { - final String key = getKeyForJobId(jobId); - final String status = configMap.getData().get(key); - if (!StringUtils.isNullOrWhitespaceOnly(status)) { - return Optional.of(JobSchedulingStatus.valueOf(status)); - } - return Optional.empty(); - } - - private String getKeyForJobId(JobID jobId) { - return RUNNING_JOBS_REGISTRY_KEY_PREFIX + jobId.toString(); - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java index 14fdc2860c938..6e0f25e48540f 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java @@ -96,7 +96,6 @@ public class Constants { public static final String LEADER_SESSION_ID_KEY = "sessionId"; public static final String JOB_GRAPH_STORE_KEY_PREFIX = "jobGraph-"; public static final String SUBMITTED_JOBGRAPH_FILE_PREFIX = "submittedJobGraph"; - public static final String RUNNING_JOBS_REGISTRY_KEY_PREFIX = "runningJobsRegistry-"; public static final String CHECKPOINT_COUNTER_KEY = "counter"; public static final String CHECKPOINT_ID_KEY_PREFIX = "checkpointID-"; public static final String COMPLETED_CHECKPOINT_FILE_SUFFIX = "completedCheckpoint"; diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistryTest.java deleted file mode 100644 index d77c572df478f..0000000000000 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistryTest.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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.flink.kubernetes.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.core.testutils.FlinkMatchers; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; - -import org.junit.Test; - -import java.io.IOException; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.fail; - -/** Tests for {@link KubernetesRunningJobsRegistry} operations. */ -public class KubernetesRunningJobsRegistryTest extends KubernetesHighAvailabilityTestBase { - - private final JobID jobID = JobID.generate(); - - @Test - public void testSetAndGetJobStatus() throws Exception { - new Context() { - { - runTest( - () -> { - leaderCallbackGrantLeadership(); - - final KubernetesRunningJobsRegistry runningJobsRegistry = - new KubernetesRunningJobsRegistry( - flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); - runningJobsRegistry.setJobRunning(jobID); - assertThat( - runningJobsRegistry.getJobSchedulingStatus(jobID), - is(RunningJobsRegistry.JobSchedulingStatus.RUNNING)); - }); - } - }; - } - - @Test - public void testGetJobStatusNonExisting() throws Exception { - new Context() { - { - runTest( - () -> { - leaderCallbackGrantLeadership(); - - final KubernetesRunningJobsRegistry runningJobsRegistry = - new KubernetesRunningJobsRegistry( - flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); - final JobID jobId = JobID.generate(); - assertThat( - runningJobsRegistry.getJobSchedulingStatus(jobId), - is(RunningJobsRegistry.JobSchedulingStatus.PENDING)); - }); - } - }; - } - - @Test - public void testGetJobStatusConfigMapNotExist() throws Exception { - new Context() { - { - runTest( - () -> { - final KubernetesRunningJobsRegistry runningJobsRegistry = - new KubernetesRunningJobsRegistry( - flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); - try { - runningJobsRegistry.getJobSchedulingStatus(JobID.generate()); - fail("Exception should be thrown."); - } catch (IOException ex) { - final String msg = - "ConfigMap " + LEADER_CONFIGMAP_NAME + " does not exist"; - assertThat(ex, FlinkMatchers.containsMessage(msg)); - } - }); - } - }; - } - - @Test - public void testClearJob() throws Exception { - new Context() { - { - runTest( - () -> { - leaderCallbackGrantLeadership(); - - final KubernetesRunningJobsRegistry runningJobsRegistry = - new KubernetesRunningJobsRegistry( - flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); - runningJobsRegistry.setJobFinished(jobID); - assertThat( - runningJobsRegistry.getJobSchedulingStatus(jobID), - is(RunningJobsRegistry.JobSchedulingStatus.DONE)); - runningJobsRegistry.clearJob(jobID); - assertThat( - runningJobsRegistry.getJobSchedulingStatus(jobID), - is(RunningJobsRegistry.JobSchedulingStatus.PENDING)); - }); - } - }; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index fdfc505351614..6eb12adedccc4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -36,7 +36,8 @@ import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -110,7 +111,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint resourceManagerGatewayRetriever; @@ -178,7 +179,7 @@ public Dispatcher( JobManagerSharedServices.fromConfiguration( configuration, blobServer, fatalErrorHandler); - this.runningJobsRegistry = highAvailabilityServices.getRunningJobsRegistry(); + this.jobResultStore = highAvailabilityServices.getJobResultStore(); runningJobs = new HashMap<>(16); @@ -372,9 +373,7 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException { */ private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException { try { - final RunningJobsRegistry.JobSchedulingStatus schedulingStatus = - runningJobsRegistry.getJobSchedulingStatus(jobId); - return schedulingStatus == RunningJobsRegistry.JobSchedulingStatus.DONE; + return jobResultStore.hasJobResultEntry(jobId); } catch (IOException e) { throw new FlinkException( String.format("Failed to retrieve job scheduling status for job %s.", jobId), @@ -830,7 +829,12 @@ private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJo .thenCompose( jobGraphRemoved -> job.closeAsync().thenApply(ignored -> jobGraphRemoved)) .thenAcceptAsync( - jobGraphRemoved -> cleanUpRemainingJobData(jobId, jobGraphRemoved), + jobGraphRemoved -> { + cleanUpRemainingJobData(jobId, jobGraphRemoved); + if (jobGraphRemoved) { + markJobAsClean(jobId); + } + }, ioExecutor); } @@ -867,14 +871,6 @@ private boolean cleanUpJobGraph(JobID jobId, boolean cleanupHA) { private void cleanUpRemainingJobData(JobID jobId, boolean jobGraphRemoved) { jobManagerMetricGroup.removeJob(jobId); if (jobGraphRemoved) { - try { - runningJobsRegistry.clearJob(jobId); - } catch (IOException e) { - log.warn( - "Could not properly remove job {} from the running jobs registry.", - jobId, - e); - } try { highAvailabilityServices.cleanupJobData(jobId); } catch (Exception e) { @@ -885,6 +881,16 @@ private void cleanUpRemainingJobData(JobID jobId, boolean jobGraphRemoved) { blobServer.cleanupJob(jobId, jobGraphRemoved); } + private void markJobAsClean(JobID jobId) { + try { + jobResultStore.markResultAsClean(jobId); + log.debug( + "Cleanup for the job '{}' has finished. Job has been marked as clean.", jobId); + } catch (IOException e) { + log.warn("Could not properly mark job {} result as clean.", jobId, e); + } + } + private void cleanUpHighAvailabilityJobData(JobID jobId) { final boolean jobGraphRemoved = cleanUpJobGraph(jobId, true); cleanUpRemainingJobData(jobId, jobGraphRemoved); @@ -952,6 +958,29 @@ protected CleanupJobState jobReachedTerminalState(ExecutionGraphInfo executionGr archiveExecutionGraph(executionGraphInfo); + if (terminalJobStatus.isGloballyTerminalState()) { + final JobID jobId = executionGraphInfo.getJobId(); + try { + if (jobResultStore.hasCleanJobResultEntry(jobId)) { + log.warn( + "Job {} is already marked as clean but clean up was triggered again.", + jobId); + } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) { + jobResultStore.createDirtyResult( + new JobResultEntry( + JobResult.createFrom( + executionGraphInfo.getArchivedExecutionGraph()))); + } + } catch (IOException e) { + fatalErrorHandler.onFatalError( + new FlinkException( + String.format( + "The job %s couldn't be marked as pre-cleanup finished in JobResultStore.", + jobId), + e)); + } + } + return terminalJobStatus.isGloballyTerminalState() ? CleanupJobState.GLOBAL : CleanupJobState.LOCAL; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java index f025261030a18..e670f2d0cdd1e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory; import org.apache.flink.runtime.jobmaster.JobManagerRunner; @@ -64,8 +64,8 @@ public JobManagerRunner createJobManagerRunner( final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); - final RunningJobsRegistry runningJobsRegistry = - highAvailabilityServices.getRunningJobsRegistry(); + final JobResultStore jobResultStore = highAvailabilityServices.getJobResultStore(); + final LeaderElectionService jobManagerLeaderElectionService = highAvailabilityServices.getJobManagerLeaderElectionService(jobGraph.getJobID()); @@ -118,7 +118,7 @@ public JobManagerRunner createJobManagerRunner( return new JobMasterServiceLeadershipRunner( jobMasterServiceProcessFactory, jobManagerLeaderElectionService, - runningJobsRegistry, + jobResultStore, classLoaderLease, fatalErrorHandler); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java index da8fb83266f5e..d6c0418f37995 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java @@ -47,7 +47,7 @@ * resources. * *

The abstract class is also responsible for determining which component service should be - * reused. For example, {@link #runningJobsRegistry} is created once and could be reused many times. + * reused. For example, {@link #jobResultStore} is created once and could be reused many times. */ public abstract class AbstractHaServices implements HighAvailabilityServices { @@ -62,15 +62,18 @@ public abstract class AbstractHaServices implements HighAvailabilityServices { /** Store for arbitrary blobs. */ private final BlobStoreService blobStoreService; - /** The distributed storage based running jobs registry. */ - private RunningJobsRegistry runningJobsRegistry; + private final JobResultStore jobResultStore; - public AbstractHaServices( - Configuration config, Executor ioExecutor, BlobStoreService blobStoreService) { + protected AbstractHaServices( + Configuration config, + Executor ioExecutor, + BlobStoreService blobStoreService, + JobResultStore jobResultStore) { this.configuration = checkNotNull(config); this.ioExecutor = checkNotNull(ioExecutor); this.blobStoreService = checkNotNull(blobStoreService); + this.jobResultStore = checkNotNull(jobResultStore); } @Override @@ -130,11 +133,8 @@ public JobGraphStore getJobGraphStore() throws Exception { } @Override - public RunningJobsRegistry getRunningJobsRegistry() { - if (runningJobsRegistry == null) { - this.runningJobsRegistry = createRunningJobsRegistry(); - } - return runningJobsRegistry; + public JobResultStore getJobResultStore() throws Exception { + return jobResultStore; } @Override @@ -243,13 +243,6 @@ public void cleanupJobData(JobID jobID) throws Exception { */ protected abstract JobGraphStore createJobGraphStore() throws Exception; - /** - * Create the registry that holds information about whether jobs are currently running. - * - * @return Running job registry to retrieve running jobs - */ - protected abstract RunningJobsRegistry createRunningJobsRegistry(); - /** * Closes the components which is used for external operations(e.g. Zookeeper Client, Kubernetes * Client). diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index 47a73276703d0..8df9227ce22b6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -169,11 +169,12 @@ default LeaderElectionService getWebMonitorLeaderElectionService() { JobGraphStore getJobGraphStore() throws Exception; /** - * Gets the registry that holds information about whether jobs are currently running. + * Gets the store that holds information about the state of finished jobs. * - * @return Running job registry to retrieve running jobs + * @return Store of finished job results + * @throws Exception if job result store could not be created */ - RunningJobsRegistry getRunningJobsRegistry() throws Exception; + JobResultStore getJobResultStore() throws Exception; /** * Creates the BLOB store in which BLOBs are stored in a highly-available fashion. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java deleted file mode 100644 index a2c1aa326fc8a..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.flink.runtime.highavailability; - -import org.apache.flink.api.common.JobID; - -import java.io.IOException; - -/** - * A simple registry that tracks if a certain job is pending execution, running, or completed. - * - *

This registry is used in highly-available setups with multiple master nodes, to determine - * whether a new leader should attempt to recover a certain job (because the job is still running), - * or whether the job has already finished successfully (in case of a finite job) and the leader has - * only been granted leadership because the previous leader quit cleanly after the job was finished. - * - *

In addition, the registry can help to determine whether a newly assigned leader JobManager - * should attempt reconciliation with running TaskManagers, or immediately schedule the job from the - * latest checkpoint/savepoint. - */ -public interface RunningJobsRegistry { - - /** The scheduling status of a job, as maintained by the {@code RunningJobsRegistry}. */ - enum JobSchedulingStatus { - - /** Job has not been scheduled, yet. */ - PENDING, - - /** Job has been scheduled and is not yet finished. */ - RUNNING, - - /** Job has been finished, successfully or unsuccessfully. */ - DONE - } - - // ------------------------------------------------------------------------ - - /** - * Marks a job as running. Requesting the job's status via the {@link - * #getJobSchedulingStatus(JobID)} method will return {@link JobSchedulingStatus#RUNNING}. - * - * @param jobID The id of the job. - * @throws IOException Thrown when the communication with the highly-available storage or - * registry failed and could not be retried. - */ - void setJobRunning(JobID jobID) throws IOException; - - /** - * Marks a job as completed. Requesting the job's status via the {@link - * #getJobSchedulingStatus(JobID)} method will return {@link JobSchedulingStatus#DONE}. - * - * @param jobID The id of the job. - * @throws IOException Thrown when the communication with the highly-available storage or - * registry failed and could not be retried. - */ - void setJobFinished(JobID jobID) throws IOException; - - /** - * Gets the scheduling status of a job. - * - * @param jobID The id of the job to check. - * @return The job scheduling status. - * @throws IOException Thrown when the communication with the highly-available storage or - * registry failed and could not be retried. - */ - JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException; - - /** - * Clear job state form the registry, usually called after job finish. - * - * @param jobID The id of the job to check. - * @throws IOException Thrown when the communication with the highly-available storage or - * registry failed and could not be retried. - */ - void clearJob(JobID jobID) throws IOException; -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java index 188bdbbfd1c42..ddb62b368e5b3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java @@ -23,8 +23,8 @@ import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; @@ -43,14 +43,14 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices { protected final Object lock = new Object(); - private final RunningJobsRegistry runningJobsRegistry; + private final JobResultStore jobResultStore; private final VoidBlobStore voidBlobStore; private boolean shutdown; public AbstractNonHaServices() { - this.runningJobsRegistry = new StandaloneRunningJobsRegistry(); + this.jobResultStore = new EmbeddedJobResultStore(); this.voidBlobStore = new VoidBlobStore(); shutdown = false; @@ -79,11 +79,11 @@ public JobGraphStore getJobGraphStore() throws Exception { } @Override - public RunningJobsRegistry getRunningJobsRegistry() throws Exception { + public JobResultStore getJobResultStore() throws Exception { synchronized (lock) { checkNotShutdown(); - return runningJobsRegistry; + return jobResultStore; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java deleted file mode 100644 index f5055bab02a71..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.flink.runtime.highavailability.nonha.standalone; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; - -import java.util.HashMap; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** A registry for running jobs, not-highly available. */ -public class StandaloneRunningJobsRegistry implements RunningJobsRegistry { - - /** The currently running jobs. */ - private final HashMap jobStatus = new HashMap<>(); - - @Override - public void setJobRunning(JobID jobID) { - checkNotNull(jobID); - - synchronized (jobStatus) { - jobStatus.put(jobID, JobSchedulingStatus.RUNNING); - } - } - - @Override - public void setJobFinished(JobID jobID) { - checkNotNull(jobID); - - synchronized (jobStatus) { - jobStatus.put(jobID, JobSchedulingStatus.DONE); - } - } - - @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) { - checkNotNull(jobID); - - synchronized (jobStatus) { - JobSchedulingStatus status = jobStatus.get(jobID); - return status == null ? JobSchedulingStatus.PENDING : status; - } - } - - @Override - public void clearJob(JobID jobID) { - checkNotNull(jobID); - - synchronized (jobStatus) { - jobStatus.remove(jobID); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java index 13f729e5369d7..b127d848a3451 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.AbstractHaServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.util.ZooKeeperUtils; @@ -46,7 +46,7 @@ public AbstractZooKeeperHaServices( Executor executor, Configuration configuration, BlobStoreService blobStoreService) { - super(configuration, executor, blobStoreService); + super(configuration, executor, blobStoreService, new EmbeddedJobResultStore()); this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper); } @@ -69,12 +69,6 @@ public JobGraphStore createJobGraphStore() throws Exception { curatorFrameworkWrapper.asCuratorFramework(), configuration); } - @Override - public RunningJobsRegistry createRunningJobsRegistry() { - return new ZooKeeperRunningJobsRegistry( - curatorFrameworkWrapper.asCuratorFramework(), configuration); - } - @Override protected void internalClose() throws Exception { curatorFrameworkWrapper.close(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java deleted file mode 100644 index cf8c3363e7910..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * 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.flink.runtime.highavailability.zookeeper; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; - -import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework; -import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException; -import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.nio.charset.Charset; -import java.util.Arrays; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** A zookeeper based registry for running jobs, highly available. */ -public class ZooKeeperRunningJobsRegistry implements RunningJobsRegistry { - - private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperRunningJobsRegistry.class); - - private static final Charset ENCODING = Charset.forName("utf-8"); - - /** The ZooKeeper client to use. */ - private final CuratorFramework client; - - private final String runningJobPath; - - public ZooKeeperRunningJobsRegistry( - final CuratorFramework client, final Configuration configuration) { - this.client = checkNotNull(client, "client"); - this.runningJobPath = - configuration.getString( - HighAvailabilityOptions.ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH); - } - - @Override - public void setJobRunning(JobID jobID) throws IOException { - checkNotNull(jobID); - - try { - writeEnumToZooKeeper(jobID, JobSchedulingStatus.RUNNING); - } catch (Exception e) { - throw new IOException("Failed to set RUNNING state in ZooKeeper for job " + jobID, e); - } - } - - @Override - public void setJobFinished(JobID jobID) throws IOException { - checkNotNull(jobID); - - try { - writeEnumToZooKeeper(jobID, JobSchedulingStatus.DONE); - } catch (Exception e) { - throw new IOException("Failed to set DONE state in ZooKeeper for job " + jobID, e); - } - } - - @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException { - checkNotNull(jobID); - - try { - final String zkPath = createZkPath(jobID); - final Stat stat = client.checkExists().forPath(zkPath); - if (stat != null) { - // found some data, try to parse it - final byte[] data = client.getData().forPath(zkPath); - if (data != null) { - try { - final String name = new String(data, ENCODING); - return JobSchedulingStatus.valueOf(name); - } catch (IllegalArgumentException e) { - throw new IOException( - "Found corrupt data in ZooKeeper: " - + Arrays.toString(data) - + " is no valid job status"); - } - } - } - - // nothing found, yet, must be in status 'PENDING' - return JobSchedulingStatus.PENDING; - } catch (Exception e) { - throw new IOException("Get finished state from zk fail for job " + jobID.toString(), e); - } - } - - @Override - public void clearJob(JobID jobID) throws IOException { - checkNotNull(jobID); - - final String zkPath = createZkPath(jobID); - - try { - final Stat stat = this.client.checkExists().forPath(zkPath); - if (stat != null) { - this.client.delete().forPath(zkPath); - } - } catch (KeeperException.NoNodeException ignored) { - // The zNode has been deleted by somebody else. This works as well. - } catch (Exception e) { - throw new IOException("Failed to clear job state from ZooKeeper for job " + jobID, e); - } - } - - private String createZkPath(JobID jobID) { - return runningJobPath + jobID.toString(); - } - - private void writeEnumToZooKeeper(JobID jobID, JobSchedulingStatus status) throws Exception { - LOG.debug("Setting scheduling state for job {} to {}.", jobID, status); - final String zkPath = createZkPath(jobID); - while (true) { - try { - final Stat stat = this.client.checkExists().forPath(zkPath); - if (stat != null) { - this.client.setData().forPath(zkPath, status.name().getBytes(ENCODING)); - } else { - this.client - .create() - .creatingParentContainersIfNeeded() - .forPath(zkPath, status.name().getBytes(ENCODING)); - } - return; - } catch (KeeperException.NoNodeException | KeeperException.NodeExistsException e) { - LOG.debug("Retrying failure to set job state from ZooKeeper for job {}", jobID, e); - } - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java index 45782d4aff405..39998bed9f785 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceProcessFactory; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -81,7 +81,7 @@ public class JobMasterServiceLeadershipRunner implements JobManagerRunner, Leade private final LeaderElectionService leaderElectionService; - private final RunningJobsRegistry runningJobsRegistry; + private final JobResultStore jobResultStore; private final LibraryCacheManager.ClassLoaderLease classLoaderLease; @@ -111,12 +111,12 @@ public class JobMasterServiceLeadershipRunner implements JobManagerRunner, Leade public JobMasterServiceLeadershipRunner( JobMasterServiceProcessFactory jobMasterServiceProcessFactory, LeaderElectionService leaderElectionService, - RunningJobsRegistry runningJobsRegistry, + JobResultStore jobResultStore, LibraryCacheManager.ClassLoaderLease classLoaderLease, FatalErrorHandler fatalErrorHandler) { this.jobMasterServiceProcessFactory = jobMasterServiceProcessFactory; this.leaderElectionService = leaderElectionService; - this.runningJobsRegistry = runningJobsRegistry; + this.jobResultStore = jobResultStore; this.classLoaderLease = classLoaderLease; this.fatalErrorHandler = fatalErrorHandler; } @@ -269,13 +269,17 @@ private void startJobMasterServiceProcessAsync(UUID leaderSessionId) { @GuardedBy("lock") private void verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(UUID leaderSessionId) throws FlinkException { - final RunningJobsRegistry.JobSchedulingStatus jobSchedulingStatus = - getJobSchedulingStatus(); - - if (jobSchedulingStatus == RunningJobsRegistry.JobSchedulingStatus.DONE) { - jobAlreadyDone(); - } else { - createNewJobMasterServiceProcess(leaderSessionId); + try { + if (jobResultStore.hasJobResultEntry(getJobID())) { + jobAlreadyDone(); + } else { + createNewJobMasterServiceProcess(leaderSessionId); + } + } catch (IOException e) { + throw new FlinkException( + String.format( + "Could not retrieve the job scheduling status for job %s.", getJobID()), + e); } } @@ -293,17 +297,6 @@ private void jobAlreadyDone() { new JobAlreadyDoneException(getJobID()))))); } - private RunningJobsRegistry.JobSchedulingStatus getJobSchedulingStatus() throws FlinkException { - try { - return runningJobsRegistry.getJobSchedulingStatus(getJobID()); - } catch (IOException e) { - throw new FlinkException( - String.format( - "Could not retrieve the job scheduling status for job %s.", getJobID()), - e); - } - } - @GuardedBy("lock") private void createNewJobMasterServiceProcess(UUID leaderSessionId) throws FlinkException { Preconditions.checkState(jobMasterServiceProcess.closeAsync().isDone()); @@ -312,16 +305,6 @@ private void createNewJobMasterServiceProcess(UUID leaderSessionId) throws Flink "Create new JobMasterServiceProcess because we were granted leadership under {}.", leaderSessionId); - try { - runningJobsRegistry.setJobRunning(getJobID()); - } catch (IOException e) { - throw new FlinkException( - String.format( - "Failed to set the job %s to running in the running jobs registry.", - getJobID()), - e); - } - jobMasterServiceProcess = jobMasterServiceProcessFactory.create(leaderSessionId); forwardIfValidLeader( @@ -382,17 +365,7 @@ private void onJobCompletion( "Could not retrieve JobMasterGateway because the JobMaster failed.", throwable)); } else { - if (jobManagerRunnerResult.isSuccess()) { - try { - runningJobsRegistry.setJobFinished(getJobID()); - } catch (IOException e) { - LOG.error( - "Could not un-register from high-availability services job {}." - + "Other JobManager's may attempt to recover it and re-execute it.", - getJobID(), - e); - } - } else { + if (!jobManagerRunnerResult.isSuccess()) { jobMasterGatewayFuture.completeExceptionally( new FlinkException( "Could not retrieve JobMasterGateway because the JobMaster initialization failed.", diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java index 345aae7d1e9bb..67ec96e566eea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; @@ -106,6 +107,7 @@ public void setUp() throws Exception { haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); haServices.setJobGraphStore(new StandaloneJobGraphStore()); + haServices.setJobResultStore(new EmbeddedJobResultStore()); configuration = new Configuration(); blobServer = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java index 67271d6b2249b..535709d26e9a4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.checkpoint.PerJobCheckpointRecoveryFactory; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphBuilder; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -223,7 +223,9 @@ private TestingDispatcher createRecoveredDispatcher( for (JobID jobId : haServices.getJobGraphStore().getJobIds()) { jobGraphs.add(haServices.getJobGraphStore().recoverJobGraph(jobId)); } - haServices.setRunningJobsRegistry(new StandaloneRunningJobsRegistry()); + // we need to reinstantiate the JobResultStore here to simulate a not-persisting + // JobResultStore + haServices.setJobResultStore(new EmbeddedJobResultStore()); final TestingDispatcher dispatcher = new TestingDispatcherBuilder() .setJobManagerRunnerFactory( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 80123ea3f1c7d..ed96fae91e83b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -32,16 +32,19 @@ import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.messages.Acknowledge; @@ -53,6 +56,7 @@ import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.runtime.testutils.TestingJobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -60,6 +64,7 @@ import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.hamcrest.core.IsInstanceOf; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -70,7 +75,7 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; @@ -79,6 +84,7 @@ import java.util.Collections; import java.util.Optional; import java.util.Queue; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ForkJoinPool; @@ -88,6 +94,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** Tests the resource cleanup by the {@link Dispatcher}. */ @@ -111,7 +118,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { private Configuration configuration; - private SingleRunningJobsRegistry runningJobsRegistry; + private JobResultStore jobResultStore; private TestingHighAvailabilityServices highAvailabilityServices; @@ -147,8 +154,8 @@ public void setup() throws Exception { highAvailabilityServices = new TestingHighAvailabilityServices(); clearedJobLatch = new OneShotLatch(); - runningJobsRegistry = new SingleRunningJobsRegistry(jobId, clearedJobLatch); - highAvailabilityServices.setRunningJobsRegistry(runningJobsRegistry); + jobResultStore = new SingleJobResultStore(jobId, clearedJobLatch); + highAvailabilityServices.setJobResultStore(jobResultStore); cleanupJobHADataFuture = new CompletableFuture<>(); highAvailabilityServices.setCleanupJobDataFuture(cleanupJobHADataFuture); @@ -371,16 +378,19 @@ public void testHACleanupWhenJobFinishedWhileClosingDispatcher() throws Exceptio } /** - * Tests that the {@link RunningJobsRegistry} entries are cleared after the job reached a + * Tests that the {@link JobResultStore} entries are marked as clean after the job reached a * terminal state. */ @Test - public void testRunningJobsRegistryCleanup() throws Exception { + public void testJobResultStoreCleanup() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); - runningJobsRegistry.setJobRunning(jobId); - assertThat(runningJobsRegistry.contains(jobId), is(true)); + final JobResult jobResult = + TestingJobResultStore.createJobResult(jobId, ApplicationStatus.UNKNOWN); + + jobResultStore.createDirtyResult(new JobResultEntry(jobResult)); + assertTrue(jobResultStore.hasJobResultEntry(jobId)); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); @@ -394,7 +404,8 @@ public void testRunningJobsRegistryCleanup() throws Exception { // wait for the clearing clearedJobLatch.await(); - assertThat(runningJobsRegistry.contains(jobId), is(false)); + assertTrue(jobResultStore.hasJobResultEntry(jobId)); + assertTrue(jobResultStore.getDirtyResults().isEmpty()); } /** @@ -406,7 +417,6 @@ public void testJobSubmissionUnderSameJobId() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); - runningJobsRegistry.setJobRunning(jobId); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); suspendJob(testingJobManagerRunner); @@ -545,27 +555,23 @@ public void testDispatcherTerminationWaitsForJobMasterTerminations() throws Exce dispatcherTerminationFuture.get(); } - private static final class SingleRunningJobsRegistry implements RunningJobsRegistry { - - @Nonnull private final JobID expectedJobId; + private static final class SingleJobResultStore implements JobResultStore { - @Nonnull private final OneShotLatch clearedJobLatch; + private final JobID expectedJobId; + @Nullable private JobResultEntry actualJobResultEntry; + private boolean isDirty = true; - private JobSchedulingStatus jobSchedulingStatus = JobSchedulingStatus.PENDING; + private final OneShotLatch clearedJobLatch; - private boolean containsJob = false; - - private SingleRunningJobsRegistry( - @Nonnull JobID expectedJobId, @Nonnull OneShotLatch clearedJobLatch) { + private SingleJobResultStore(JobID expectedJobId, OneShotLatch clearedJobLatch) { this.expectedJobId = expectedJobId; this.clearedJobLatch = clearedJobLatch; } @Override - public void setJobRunning(JobID jobID) { - checkJobId(jobID); - containsJob = true; - jobSchedulingStatus = JobSchedulingStatus.RUNNING; + public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException { + checkJobId(jobResultEntry.getJobId()); + this.actualJobResultEntry = jobResultEntry; } private void checkJobId(JobID jobID) { @@ -573,28 +579,38 @@ private void checkJobId(JobID jobID) { } @Override - public void setJobFinished(JobID jobID) { - checkJobId(jobID); - containsJob = true; - jobSchedulingStatus = JobSchedulingStatus.DONE; + public void markResultAsClean(JobID jobId) throws IOException { + checkJobId(jobId); + Preconditions.checkNotNull(actualJobResultEntry); + isDirty = false; + clearedJobLatch.trigger(); } @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) { - checkJobId(jobID); - return jobSchedulingStatus; + public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException { + if (actualJobResultEntry == null) { + return false; + } + + checkJobId(jobId); + return isDirty; } - public boolean contains(JobID jobId) { + @Override + public boolean hasCleanJobResultEntry(JobID jobId) throws IOException { + if (actualJobResultEntry == null) { + return false; + } + checkJobId(jobId); - return containsJob; + return !isDirty; } @Override - public void clearJob(JobID jobID) { - checkJobId(jobID); - containsJob = false; - clearedJobLatch.trigger(); + public Set getDirtyResults() throws IOException { + return actualJobResultEntry != null && isDirty + ? Collections.singleton(actualJobResultEntry.getJobResult()) + : Collections.emptySet(); } } @@ -645,6 +661,79 @@ public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); } + @Test + public void testFatalErrorIfJobCannotBeMarkedDirtyInJobResultStore() throws Exception { + jobResultStore = + TestingJobResultStore.builder() + .withCreateDirtyResultConsumer( + jobResult -> { + throw new IOException("Expected IOException."); + }) + .build(); + highAvailabilityServices.setJobResultStore(jobResultStore); + + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = + startDispatcherAndSubmitJob(); + + ArchivedExecutionGraph executionGraph = + new ArchivedExecutionGraphBuilder() + .setJobID(jobId) + .setState(JobStatus.FINISHED) + .build(); + + final TestingJobManagerRunner testingJobManagerRunner = + jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); + + final CompletableFuture errorFuture = + this.testingFatalErrorHandlerResource.getFatalErrorHandler().getErrorFuture(); + assertThat( + errorFuture.get(100, TimeUnit.MILLISECONDS), + IsInstanceOf.instanceOf(FlinkException.class)); + testingFatalErrorHandlerResource.getFatalErrorHandler().clearError(); + } + + @Test + public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws Exception { + final CompletableFuture dirtyJobFuture = new CompletableFuture<>(); + jobResultStore = + TestingJobResultStore.builder() + .withCreateDirtyResultConsumer(dirtyJobFuture::complete) + .withMarkResultAsCleanConsumer( + jobId -> { + throw new IOException("Expected IOException."); + }) + .build(); + highAvailabilityServices.setJobResultStore(jobResultStore); + + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = + startDispatcherAndSubmitJob(); + + ArchivedExecutionGraph executionGraph = + new ArchivedExecutionGraphBuilder() + .setJobID(jobId) + .setState(JobStatus.FINISHED) + .build(); + + final TestingJobManagerRunner testingJobManagerRunner = + jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); + + final CompletableFuture errorFuture = + this.testingFatalErrorHandlerResource.getFatalErrorHandler().getErrorFuture(); + try { + final Throwable unexpectedError = errorFuture.get(100, TimeUnit.MILLISECONDS); + fail( + "No error should have been reported but an " + + unexpectedError.getClass() + + " was handled."); + } catch (TimeoutException e) { + // expected + } + + assertThat(dirtyJobFuture.get().getJobId(), is(jobId)); + } + private static final class TestingBlobServer extends BlobServer { private final CompletableFuture cleanupJobFuture; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index bc5687bd80288..c327cbc2fe55f 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -38,9 +38,9 @@ import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphBuilder; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; @@ -80,6 +80,7 @@ import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.TestingJobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -90,6 +91,7 @@ import org.assertj.core.api.Assertions; import org.hamcrest.Matchers; +import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -144,7 +146,7 @@ public class DispatcherTest extends AbstractDispatcherTest { private static final String CLEANUP_JOB_GRAPH_RELEASE = "job-graph-release"; private static final String CLEANUP_JOB_MANAGER_RUNNER = "job-manager-runner"; private static final String CLEANUP_HA_SERVICES = "ha-services"; - private static final String CLEANUP_RUNNING_JOBS_REGISTRY = "running-jobs-registry"; + private static final String CLEANUP_JOB_RESULT_STORE = "job-result-store"; private JobGraph jobGraph; @@ -216,7 +218,10 @@ public void testJobSubmission() throws Exception { @Test public void testDuplicateJobSubmissionWithGloballyTerminatedJobId() throws Exception { - haServices.getRunningJobsRegistry().setJobFinished(jobGraph.getJobID()); + final JobResult jobResult = + TestingJobResultStore.createJobResult( + jobGraph.getJobID(), ApplicationStatus.UNKNOWN); + haServices.getJobResultStore().createDirtyResult(new JobResultEntry(jobResult)); dispatcher = createAndStartDispatcher( heartbeatServices, @@ -736,16 +741,12 @@ public void testFailingJobManagerRunnerCleanup() throws Exception { jobGraphStore.start(null); haServices.setJobGraphStore(jobGraphStore); - // Track cleanup - running jobs registry - haServices.setRunningJobsRegistry( - new StandaloneRunningJobsRegistry() { - - @Override - public void clearJob(JobID jobID) { - super.clearJob(jobID); - cleanUpEvents.add(CLEANUP_RUNNING_JOBS_REGISTRY); - } - }); + // Track cleanup - job result store + haServices.setJobResultStore( + TestingJobResultStore.builder() + .withMarkResultAsCleanConsumer( + jobID -> cleanUpEvents.add(CLEANUP_JOB_RESULT_STORE)) + .build()); dispatcher = createAndStartDispatcher( @@ -765,11 +766,7 @@ public void clearJob(JobID jobID) { // make sure we've cleaned up in correct order (including HA) assertThat( new ArrayList<>(cleanUpEvents), - equalTo( - Arrays.asList( - CLEANUP_JOB_GRAPH_REMOVE, - CLEANUP_RUNNING_JOBS_REGISTRY, - CLEANUP_HA_SERVICES))); + equalTo(Arrays.asList(CLEANUP_JOB_GRAPH_REMOVE, CLEANUP_HA_SERVICES))); } // don't fail this time @@ -1140,16 +1137,19 @@ private void testJobDataAreCleanedUpInCorrectOrder(JobStatus jobStatus) throws E jobGraphStore.start(null); haServices.setJobGraphStore(jobGraphStore); - // Track cleanup - running jobs registry - haServices.setRunningJobsRegistry( - new StandaloneRunningJobsRegistry() { - - @Override - public void clearJob(JobID jobID) { - super.clearJob(jobID); - cleanUpEvents.add(CLEANUP_RUNNING_JOBS_REGISTRY); - } - }); + // Track cleanup - job result store + haServices.setJobResultStore( + TestingJobResultStore.builder() + .withMarkResultAsCleanConsumer( + jobID -> + assertThat( + "All cleanup tasks should have been finished before marking the job as clean.", + cleanUpEvents, + IsIterableContainingInAnyOrder.containsInAnyOrder( + CLEANUP_HA_SERVICES, + CLEANUP_JOB_GRAPH_REMOVE, + CLEANUP_JOB_MANAGER_RUNNER))) + .build()); final CompletableFuture resultFuture = new CompletableFuture<>(); dispatcher = @@ -1165,19 +1165,14 @@ public void clearJob(JobID jobID) { resultFuture.complete( JobManagerRunnerResult.forSuccess( new ExecutionGraphInfo( - new ArchivedExecutionGraphBuilder().setState(jobStatus).build()))); + new ArchivedExecutionGraphBuilder() + .setState(jobStatus) + .setFailureCause( + new ErrorInfo(new RuntimeException("expected"), 1L)) + .build()))); // Wait for job to terminate. dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); - - assertThat( - new ArrayList<>(cleanUpEvents), - equalTo( - Arrays.asList( - CLEANUP_JOB_GRAPH_REMOVE, - CLEANUP_JOB_MANAGER_RUNNER, - CLEANUP_RUNNING_JOBS_REGISTRY, - CLEANUP_HA_SERVICES))); } private static class JobManagerRunnerWithBlockingJobMasterFactory @@ -1234,7 +1229,7 @@ public JobManagerRunner createJobManagerRunner( })), highAvailabilityServices.getJobManagerLeaderElectionService( jobGraph.getJobID()), - highAvailabilityServices.getRunningJobsRegistry(), + highAvailabilityServices.getJobResultStore(), jobManagerServices .getLibraryCacheManager() .registerClassLoaderLease(jobGraph.getJobID()), @@ -1286,7 +1281,7 @@ public JobManagerRunner createJobManagerRunner( new TestingJobMasterServiceFactory()), highAvailabilityServices.getJobManagerLeaderElectionService( jobGraph.getJobID()), - highAvailabilityServices.getRunningJobsRegistry(), + highAvailabilityServices.getJobResultStore(), jobManagerServices .getLibraryCacheManager() .registerClassLoaderLease(jobGraph.getJobID()), @@ -1309,13 +1304,13 @@ public BlockingTerminationJobManagerService( CompletableFuture future, JobMasterServiceProcessFactory jobMasterServiceProcessFactory, LeaderElectionService leaderElectionService, - RunningJobsRegistry runningJobsRegistry, + JobResultStore jobResultStore, LibraryCacheManager.ClassLoaderLease classLoaderLease, FatalErrorHandler fatalErrorHandler) { super( jobMasterServiceProcessFactory, leaderElectionService, - runningJobsRegistry, + jobResultStore, classLoaderLease, fatalErrorHandler); this.future = future; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java index 918d4f3cce0ba..aab1760b794b6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java @@ -28,6 +28,7 @@ import java.util.Collection; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.function.Function; /** {@link Dispatcher} implementation used for testing purposes. */ @@ -65,7 +66,14 @@ public void onStart() throws Exception { } void completeJobExecution(ExecutionGraphInfo executionGraphInfo) { - runAsync(() -> jobReachedTerminalState(executionGraphInfo)); + runAsync( + () -> { + try { + jobReachedTerminalState(executionGraphInfo); + } catch (Exception e) { + throw new CompletionException(e); + } + }); } CompletableFuture getJobTerminationFuture(@Nonnull JobID jobId, @Nonnull Time timeout) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index 6f0322145e961..da58dd8bd8ee2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -37,7 +37,6 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; -import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperRunningJobsRegistry; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -151,8 +150,6 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { .asCuratorFramework(); try (final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() - .setRunningJobsRegistry( - new ZooKeeperRunningJobsRegistry(client, configuration)) .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) .setJobMasterLeaderRetrieverFunction( jobId -> ZooKeeperUtils.createLeaderRetrievalService(client)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java index 630ba1f326487..a105cf8bf842d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; @@ -193,7 +194,17 @@ private TestingHaServices( Queue closeOperations, RunnableWithException internalCleanupRunnable, Consumer internalJobCleanupConsumer) { - super(config, ioExecutor, blobStoreService); + super( + config, + ioExecutor, + blobStoreService, + TestingJobResultStore.builder() + .withMarkResultAsCleanConsumer( + ignoredJobId -> { + throw new AssertionError( + "Marking the job as clean shouldn't happen in the HaServices cleanup"); + }) + .build()); this.closeOperations = closeOperations; this.internalCleanupRunnable = internalCleanupRunnable; this.internalJobCleanupConsumer = internalJobCleanupConsumer; @@ -219,11 +230,6 @@ protected JobGraphStore createJobGraphStore() throws Exception { throw new UnsupportedOperationException("Not supported by this test implementation."); } - @Override - protected RunningJobsRegistry createRunningJobsRegistry() { - throw new UnsupportedOperationException("Not supported by this test implementation."); - } - @Override protected void internalClose() { closeOperations.offer(CloseOperations.HA_CLOSE); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java index 4434eefc852be..d98150a5fb5c6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -22,7 +22,7 @@ import org.apache.flink.runtime.blob.BlobStore; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -67,7 +67,7 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private volatile JobGraphStore jobGraphStore; - private volatile RunningJobsRegistry runningJobsRegistry = new StandaloneRunningJobsRegistry(); + private volatile JobResultStore jobResultStore = new EmbeddedJobResultStore(); private CompletableFuture closeFuture = new CompletableFuture<>(); @@ -125,8 +125,8 @@ public void setJobGraphStore(JobGraphStore jobGraphStore) { this.jobGraphStore = jobGraphStore; } - public void setRunningJobsRegistry(RunningJobsRegistry runningJobsRegistry) { - this.runningJobsRegistry = runningJobsRegistry; + public void setJobResultStore(JobResultStore jobResultStore) { + this.jobResultStore = jobResultStore; } public void setJobMasterLeaderElectionServiceFunction( @@ -262,8 +262,8 @@ public JobGraphStore getJobGraphStore() { } @Override - public RunningJobsRegistry getRunningJobsRegistry() { - return runningJobsRegistry; + public JobResultStore getJobResultStore() { + return jobResultStore; } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServicesBuilder.java index dbf52699c54fd..88557c02c85d5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServicesBuilder.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -69,7 +69,7 @@ public class TestingHighAvailabilityServicesBuilder { private JobGraphStore jobGraphStore = new StandaloneJobGraphStore(); - private RunningJobsRegistry runningJobsRegistry = new StandaloneRunningJobsRegistry(); + private JobResultStore jobResultStore = new EmbeddedJobResultStore(); private CompletableFuture closeFuture = new CompletableFuture<>(); @@ -99,7 +99,7 @@ public TestingHighAvailabilityServices build() { testingHighAvailabilityServices.setCheckpointRecoveryFactory(checkpointRecoveryFactory); testingHighAvailabilityServices.setJobGraphStore(jobGraphStore); - testingHighAvailabilityServices.setRunningJobsRegistry(runningJobsRegistry); + testingHighAvailabilityServices.setJobResultStore(jobResultStore); testingHighAvailabilityServices.setCloseFuture(closeFuture); testingHighAvailabilityServices.setCloseAndCleanupAllDataFuture( @@ -167,9 +167,8 @@ public TestingHighAvailabilityServicesBuilder setJobGraphStore(JobGraphStore job return this; } - public TestingHighAvailabilityServicesBuilder setRunningJobsRegistry( - RunningJobsRegistry runningJobsRegistry) { - this.runningJobsRegistry = runningJobsRegistry; + public TestingHighAvailabilityServicesBuilder setJobResultStore(JobResultStore jobResultStore) { + this.jobResultStore = jobResultStore; return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java index 6cbb92c8cc3fc..940c8d5331c0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -117,8 +117,8 @@ public JobGraphStore getJobGraphStore() throws Exception { } @Override - public RunningJobsRegistry getRunningJobsRegistry() throws Exception { - return new StandaloneRunningJobsRegistry(); + public JobResultStore getJobResultStore() { + return new EmbeddedJobResultStore(); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java index 72b41a0ab633d..b95fb74f020da 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java @@ -23,7 +23,6 @@ import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobStoreService; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingContender; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -235,9 +234,6 @@ private void runCleanupTestWithJob( final LeaderElectionService jobManagerLeaderElectionService = zooKeeperHaServices.getJobManagerLeaderElectionService(jobId); - final RunningJobsRegistry runningJobsRegistry = - zooKeeperHaServices.getRunningJobsRegistry(); - final LeaderRetrievalUtils.LeaderConnectionInfoListener resourceManagerLeaderListener = new LeaderRetrievalUtils.LeaderConnectionInfoListener(); resourceManagerLeaderElectionService.start( @@ -253,8 +249,6 @@ private void runCleanupTestWithJob( "unused-jobmanager-address", jobManagerLeaderElectionService)); jobManagerLeaderRetriever.start(jobManagerLeaderListener); - runningJobsRegistry.setJobRunning(jobId); - // Make sure that the respective zNodes have been properly created resourceManagerLeaderListener.getLeaderConnectionInfoFuture().join(); jobManagerLeaderListener.getLeaderConnectionInfoFuture().join(); @@ -263,7 +257,6 @@ private void runCleanupTestWithJob( resourceManagerLeaderElectionService.stop(); jobManagerLeaderRetriever.stop(); jobManagerLeaderElectionService.stop(); - runningJobsRegistry.clearJob(jobId); zooKeeperHaServicesConsumer.accept(zooKeeperHaServices); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java deleted file mode 100644 index 7849274500a52..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.flink.runtime.highavailability.zookeeper; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.runtime.blob.VoidBlobStore; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus; -import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; -import org.apache.flink.runtime.util.ZooKeeperUtils; -import org.apache.flink.util.TestLogger; -import org.apache.flink.util.concurrent.Executors; - -import org.apache.curator.test.TestingServer; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class ZooKeeperRegistryTest extends TestLogger { - - private TestingServer testingServer; - - @Before - public void before() throws Exception { - testingServer = new TestingServer(); - } - - @After - public void after() throws Exception { - testingServer.stop(); - testingServer = null; - } - - /** - * Tests that the function of ZookeeperRegistry, setJobRunning(), setJobFinished(), - * isJobRunning(). - */ - @Test - public void testZooKeeperRegistry() throws Exception { - Configuration configuration = new Configuration(); - configuration.setString( - HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString()); - configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); - - final HighAvailabilityServices zkHaService = - new ZooKeeperHaServices( - ZooKeeperUtils.startCuratorFramework( - configuration, NoOpFatalErrorHandler.INSTANCE), - Executors.directExecutor(), - configuration, - new VoidBlobStore()); - - final RunningJobsRegistry zkRegistry = zkHaService.getRunningJobsRegistry(); - - try { - JobID jobID = JobID.generate(); - assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID)); - - // set when znode does not exist for job - zkRegistry.setJobRunning(jobID); - assertEquals(JobSchedulingStatus.RUNNING, zkRegistry.getJobSchedulingStatus(jobID)); - - // set when znode does exist for job - zkRegistry.setJobFinished(jobID); - assertEquals(JobSchedulingStatus.DONE, zkRegistry.getJobSchedulingStatus(jobID)); - - zkRegistry.clearJob(jobID); - assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID)); - - // clear when znode does not exist for job - zkRegistry.clearJob(jobID); - } finally { - zkHaService.closeAndCleanupAllData(); - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java index f79893d0c26ec..0b2782cfcb9ec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java @@ -23,11 +23,13 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.execution.librarycache.TestingClassLoaderLease; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -38,6 +40,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -88,7 +91,7 @@ public class JobMasterServiceLeadershipRunnerTest extends TestLogger { private TestingFatalErrorHandler fatalErrorHandler; - private RunningJobsRegistry runningJobsRegistry; + private JobResultStore jobResultStore; @BeforeClass public static void setupClass() { @@ -101,7 +104,7 @@ public static void setupClass() { @Before public void setup() { leaderElectionService = new TestingLeaderElectionService(); - runningJobsRegistry = new StandaloneRunningJobsRegistry(); + jobResultStore = new EmbeddedJobResultStore(); fatalErrorHandler = new TestingFatalErrorHandler(); } @@ -662,15 +665,17 @@ public void testJobMasterServiceProcessCreationFailureIsForwardedToResultFuture( @Test public void testJobAlreadyDone() throws Exception { - JobID jobID = new JobID(); + final JobID jobId = new JobID(); + final JobResult jobResult = + TestingJobResultStore.createJobResult(jobId, ApplicationStatus.UNKNOWN); + jobResultStore.createDirtyResult(new JobResultEntry(jobResult)); try (JobManagerRunner jobManagerRunner = newJobMasterServiceLeadershipRunnerBuilder() .setJobMasterServiceProcessFactory( TestingJobMasterServiceProcessFactory.newBuilder() - .setJobId(jobID) + .setJobId(jobId) .build()) .build()) { - runningJobsRegistry.setJobFinished(jobID); jobManagerRunner.start(); leaderElectionService.isLeader(UUID.randomUUID()); @@ -721,7 +726,7 @@ public JobMasterServiceLeadershipRunner build() { return new JobMasterServiceLeadershipRunner( jobMasterServiceProcessFactory, leaderElectionService, - runningJobsRegistry, + jobResultStore, classLoaderLease, fatalErrorHandler); } From adbe87271ea1c9b1c091ee1152b66d6b6937bd5a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 29 Nov 2021 13:02:44 +0100 Subject: [PATCH 19/49] [FLINK-25430][runtime] Renames JobGraphStoreFactory into JobPersistenceComponentFactory Separate commit to prepare the further JobResultStore integration. JobGraphStoreFactory becomes JobPersistenceComponentFactory because we want to integrate the initialization analogously to how the JobGraphStore initialization is integrated. --- ...licationDispatcherLeaderProcessFactoryFactory.java | 9 ++++++--- .../runner/DefaultDispatcherRunnerFactory.java | 6 +++--- .../runner/DispatcherLeaderProcessFactoryFactory.java | 4 ++-- .../dispatcher/runner/DispatcherRunnerFactory.java | 4 ++-- .../JobDispatcherLeaderProcessFactoryFactory.java | 4 ++-- .../runner/SessionDispatcherLeaderProcessFactory.java | 10 +++++----- .../SessionDispatcherLeaderProcessFactoryFactory.java | 6 +++--- ...aultDispatcherResourceManagerComponentFactory.java | 4 ++-- ... => HaServicesJobPersistenceComponentFactory.java} | 11 ++++++----- ...ctory.java => JobPersistenceComponentFactory.java} | 4 ++-- .../runner/ZooKeeperDefaultDispatcherRunnerTest.java | 6 +++--- .../runtime/entrypoint/ClusterEntrypointTest.java | 4 ++-- 12 files changed, 38 insertions(+), 34 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/{HaServicesJobGraphStoreFactory.java => HaServicesJobPersistenceComponentFactory.java} (79%) rename flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/{JobGraphStoreFactory.java => JobPersistenceComponentFactory.java} (91%) diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherLeaderProcessFactoryFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherLeaderProcessFactoryFactory.java index 2c1ac37187f23..03b94a88ef9e2 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherLeaderProcessFactoryFactory.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherLeaderProcessFactoryFactory.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.dispatcher.runner.DispatcherLeaderProcessFactory; import org.apache.flink.runtime.dispatcher.runner.DispatcherLeaderProcessFactoryFactory; import org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcessFactory; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -59,7 +59,7 @@ private ApplicationDispatcherLeaderProcessFactoryFactory( @Override public DispatcherLeaderProcessFactory createFactory( - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, @@ -74,7 +74,10 @@ public DispatcherLeaderProcessFactory createFactory( partialDispatcherServices); return new SessionDispatcherLeaderProcessFactory( - dispatcherServiceFactory, jobGraphStoreFactory, ioExecutor, fatalErrorHandler); + dispatcherServiceFactory, + jobPersistenceComponentFactory, + ioExecutor, + fatalErrorHandler); } public static ApplicationDispatcherLeaderProcessFactoryFactory create( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java index 9d5c4d62dd4ff..2050271db1634 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -44,7 +44,7 @@ public DefaultDispatcherRunnerFactory( public DispatcherRunner createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) @@ -52,7 +52,7 @@ public DispatcherRunner createDispatcherRunner( final DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory = dispatcherLeaderProcessFactoryFactory.createFactory( - jobGraphStoreFactory, + jobPersistenceComponentFactory, ioExecutor, rpcService, partialDispatcherServices, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java index 6792fa32ecccc..222c349870100 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -31,7 +31,7 @@ public interface DispatcherLeaderProcessFactoryFactory { DispatcherLeaderProcessFactory createFactory( - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java index a03be4cf42b48..bebafa2bbf0c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -32,7 +32,7 @@ public interface DispatcherRunnerFactory { DispatcherRunner createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java index 7be59105478be..424774c3b4b70 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java @@ -22,7 +22,7 @@ import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkException; @@ -42,7 +42,7 @@ private JobDispatcherLeaderProcessFactoryFactory(JobGraphRetriever jobGraphRetri @Override public DispatcherLeaderProcessFactory createFactory( - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java index 95cf4e49e10ee..18a6111194d01 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import java.util.UUID; @@ -31,18 +31,18 @@ public class SessionDispatcherLeaderProcessFactory implements DispatcherLeaderPr private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; - private final JobGraphStoreFactory jobGraphStoreFactory; + private final JobPersistenceComponentFactory jobPersistenceComponentFactory; private final Executor ioExecutor; private final FatalErrorHandler fatalErrorHandler; public SessionDispatcherLeaderProcessFactory( AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; - this.jobGraphStoreFactory = jobGraphStoreFactory; + this.jobPersistenceComponentFactory = jobPersistenceComponentFactory; this.ioExecutor = ioExecutor; this.fatalErrorHandler = fatalErrorHandler; } @@ -52,7 +52,7 @@ public DispatcherLeaderProcess create(UUID leaderSessionID) { return SessionDispatcherLeaderProcess.create( leaderSessionID, dispatcherGatewayServiceFactory, - jobGraphStoreFactory.create(), + jobPersistenceComponentFactory.createJobGraphStore(), ioExecutor, fatalErrorHandler); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java index 29789650ba4f3..ee2fb081647fe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -38,7 +38,7 @@ private SessionDispatcherLeaderProcessFactoryFactory(DispatcherFactory dispatche @Override public DispatcherLeaderProcessFactory createFactory( - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, @@ -50,7 +50,7 @@ public DispatcherLeaderProcessFactory createFactory( return new SessionDispatcherLeaderProcessFactory( dispatcherGatewayServiceFactory, - jobGraphStoreFactory, + jobPersistenceComponentFactory, ioExecutor, fatalErrorHandler); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java index 0d514f5183b18..f235caa554dc0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java @@ -36,7 +36,7 @@ import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.jobmanager.HaServicesJobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.HaServicesJobPersistenceComponentFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; @@ -221,7 +221,7 @@ public DispatcherResourceManagerComponent create( dispatcherRunnerFactory.createDispatcherRunner( highAvailabilityServices.getDispatcherLeaderElectionService(), fatalErrorHandler, - new HaServicesJobGraphStoreFactory(highAvailabilityServices), + new HaServicesJobPersistenceComponentFactory(highAvailabilityServices), ioExecutor, rpcService, partialDispatcherServices); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java similarity index 79% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java index ae8d6ca5cc7a7..a59a83eee14d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java @@ -22,18 +22,19 @@ import org.apache.flink.util.FlinkRuntimeException; /** - * {@link JobGraphStoreFactory} implementation which creates a {@link JobGraphStore} using the - * provided {@link HighAvailabilityServices}. + * {@link JobPersistenceComponentFactory} implementation which creates a {@link JobGraphStore} using + * the provided {@link HighAvailabilityServices}. */ -public class HaServicesJobGraphStoreFactory implements JobGraphStoreFactory { +public class HaServicesJobPersistenceComponentFactory implements JobPersistenceComponentFactory { private final HighAvailabilityServices highAvailabilityServices; - public HaServicesJobGraphStoreFactory(HighAvailabilityServices highAvailabilityServices) { + public HaServicesJobPersistenceComponentFactory( + HighAvailabilityServices highAvailabilityServices) { this.highAvailabilityServices = highAvailabilityServices; } @Override - public JobGraphStore create() { + public JobGraphStore createJobGraphStore() { try { return highAvailabilityServices.getJobGraphStore(); } catch (Exception e) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java similarity index 91% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java index 1530509003981..b4ce131ade655 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java @@ -19,12 +19,12 @@ package org.apache.flink.runtime.jobmanager; /** Factory for {@link JobGraphStore}. */ -public interface JobGraphStoreFactory { +public interface JobPersistenceComponentFactory { /** * Creates a {@link JobGraphStore}. * * @return a {@link JobGraphStore} instance */ - JobGraphStore create(); + JobGraphStore createJobGraphStore(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index da58dd8bd8ee2..484c41e60f3cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -41,7 +41,7 @@ import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobmanager.JobGraphStore; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; @@ -226,14 +226,14 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { private DispatcherRunner createDispatcherRunner( TestingRpcService rpcService, TestingLeaderElectionService dispatcherLeaderElectionService, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, PartialDispatcherServices partialDispatcherServices, DispatcherRunnerFactory dispatcherRunnerFactory) throws Exception { return dispatcherRunnerFactory.createDispatcherRunner( dispatcherLeaderElectionService, fatalErrorHandler, - jobGraphStoreFactory, + jobPersistenceComponentFactory, TestingUtils.defaultExecutor(), rpcService, partialDispatcherServices); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointTest.java index 160119bb8ed20..5b3926996df6c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointTest.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; @@ -418,7 +418,7 @@ private TestingDispatcherRunnerFactory( public DispatcherRunner createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) From 6c5ae20baaf28775d5c6e42717462ec21ddbd2d6 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 29 Nov 2021 18:54:55 +0100 Subject: [PATCH 20/49] [FLINK-25430][runtime] Renames PartialDispatcherServicesWithJobGraphStore into PartialDispatcherServicesWithJobPersistenceComponents Separate commit to prepare the further JobResultStore integration. PartialDispatcherServicesWithJobGraphStore becomes PartialDispatcherServicesWithJobPersistenceComponents because we want to integrate the initialization analogously to how the JobGraphStore initialization is integrated. --- ...cationDispatcherGatewayServiceFactory.java | 4 +-- .../runtime/dispatcher/DispatcherFactory.java | 3 +- .../dispatcher/DispatcherServices.java | 33 ++++++++++--------- .../dispatcher/JobDispatcherFactory.java | 7 ++-- ...ServicesWithJobPersistenceComponents.java} | 9 ++--- .../dispatcher/SessionDispatcherFactory.java | 5 +-- ...efaultDispatcherGatewayServiceFactory.java | 4 +-- .../runner/DefaultDispatcherRunnerITCase.java | 9 ++--- 8 files changed, 41 insertions(+), 33 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/{PartialDispatcherServicesWithJobGraphStore.java => PartialDispatcherServicesWithJobPersistenceComponents.java} (92%) diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java index f1d08e184ac6e..53d998b1e6d06 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; import org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess; import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherGatewayService; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -102,7 +102,7 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( dispatcherGateway, scheduledExecutor, errorHandler), - PartialDispatcherServicesWithJobGraphStore.from( + PartialDispatcherServicesWithJobPersistenceComponents.from( partialDispatcherServices, jobGraphWriter)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java index b9a7d7bb7c660..9fa086e36d407 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java @@ -32,6 +32,7 @@ Dispatcher createDispatcher( DispatcherId fencingToken, Collection recoveredJobs, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java index 9f19e7c0fb85e..60fda8219b9a1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java @@ -160,25 +160,28 @@ public Executor getIoExecutor() { } public static DispatcherServices from( - PartialDispatcherServicesWithJobGraphStore - partialDispatcherServicesWithJobGraphStore, + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents, JobManagerRunnerFactory jobManagerRunnerFactory) { return new DispatcherServices( - partialDispatcherServicesWithJobGraphStore.getConfiguration(), - partialDispatcherServicesWithJobGraphStore.getHighAvailabilityServices(), - partialDispatcherServicesWithJobGraphStore.getResourceManagerGatewayRetriever(), - partialDispatcherServicesWithJobGraphStore.getBlobServer(), - partialDispatcherServicesWithJobGraphStore.getHeartbeatServices(), - partialDispatcherServicesWithJobGraphStore.getArchivedExecutionGraphStore(), - partialDispatcherServicesWithJobGraphStore.getFatalErrorHandler(), - partialDispatcherServicesWithJobGraphStore.getHistoryServerArchivist(), - partialDispatcherServicesWithJobGraphStore.getMetricQueryServiceAddress(), - partialDispatcherServicesWithJobGraphStore.getOperationCaches(), - partialDispatcherServicesWithJobGraphStore + partialDispatcherServicesWithJobPersistenceComponents.getConfiguration(), + partialDispatcherServicesWithJobPersistenceComponents.getHighAvailabilityServices(), + partialDispatcherServicesWithJobPersistenceComponents + .getResourceManagerGatewayRetriever(), + partialDispatcherServicesWithJobPersistenceComponents.getBlobServer(), + partialDispatcherServicesWithJobPersistenceComponents.getHeartbeatServices(), + partialDispatcherServicesWithJobPersistenceComponents + .getArchivedExecutionGraphStore(), + partialDispatcherServicesWithJobPersistenceComponents.getFatalErrorHandler(), + partialDispatcherServicesWithJobPersistenceComponents.getHistoryServerArchivist(), + partialDispatcherServicesWithJobPersistenceComponents + .getMetricQueryServiceAddress(), + partialDispatcherServicesWithJobPersistenceComponents.getOperationCaches(), + partialDispatcherServicesWithJobPersistenceComponents .getJobManagerMetricGroupFactory() .create(), - partialDispatcherServicesWithJobGraphStore.getJobGraphWriter(), + partialDispatcherServicesWithJobPersistenceComponents.getJobGraphWriter(), jobManagerRunnerFactory, - partialDispatcherServicesWithJobGraphStore.getIoExecutor()); + partialDispatcherServicesWithJobPersistenceComponents.getIoExecutor()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java index 82c225b3a554b..223093c088b51 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java @@ -39,12 +39,13 @@ public MiniDispatcher createDispatcher( DispatcherId fencingToken, Collection recoveredJobs, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents) throws Exception { final JobGraph jobGraph = Iterables.getOnlyElement(recoveredJobs); final Configuration configuration = - partialDispatcherServicesWithJobGraphStore.getConfiguration(); + partialDispatcherServicesWithJobPersistenceComponents.getConfiguration(); final String executionModeValue = configuration.getString(INTERNAL_CLUSTER_EXECUTION_MODE); final ClusterEntrypoint.ExecutionMode executionMode = ClusterEntrypoint.ExecutionMode.valueOf(executionModeValue); @@ -53,7 +54,7 @@ public MiniDispatcher createDispatcher( rpcService, fencingToken, DispatcherServices.from( - partialDispatcherServicesWithJobGraphStore, + partialDispatcherServicesWithJobPersistenceComponents, JobMasterServiceLeadershipRunnerFactory.INSTANCE), jobGraph, dispatcherBootstrapFactory, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java similarity index 92% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java index 038fcb871542f..07b87a9637232 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java @@ -32,11 +32,12 @@ import java.util.concurrent.Executor; /** {@link DispatcherFactory} services container. */ -public class PartialDispatcherServicesWithJobGraphStore extends PartialDispatcherServices { +public class PartialDispatcherServicesWithJobPersistenceComponents + extends PartialDispatcherServices { private final JobGraphWriter jobGraphWriter; - private PartialDispatcherServicesWithJobGraphStore( + private PartialDispatcherServicesWithJobPersistenceComponents( Configuration configuration, HighAvailabilityServices highAvailabilityServices, GatewayRetriever resourceManagerGatewayRetriever, @@ -70,9 +71,9 @@ public JobGraphWriter getJobGraphWriter() { return jobGraphWriter; } - public static PartialDispatcherServicesWithJobGraphStore from( + public static PartialDispatcherServicesWithJobPersistenceComponents from( PartialDispatcherServices partialDispatcherServices, JobGraphWriter jobGraphWriter) { - return new PartialDispatcherServicesWithJobGraphStore( + return new PartialDispatcherServicesWithJobPersistenceComponents( partialDispatcherServices.getConfiguration(), partialDispatcherServices.getHighAvailabilityServices(), partialDispatcherServices.getResourceManagerGatewayRetriever(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java index 55d92efa366ec..88261ae6ed78f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java @@ -33,7 +33,8 @@ public StandaloneDispatcher createDispatcher( DispatcherId fencingToken, Collection recoveredJobs, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents) throws Exception { // create the default dispatcher return new StandaloneDispatcher( @@ -42,7 +43,7 @@ public StandaloneDispatcher createDispatcher( recoveredJobs, dispatcherBootstrapFactory, DispatcherServices.from( - partialDispatcherServicesWithJobGraphStore, + partialDispatcherServicesWithJobPersistenceComponents, JobMasterServiceLeadershipRunnerFactory.INSTANCE)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java index e521c4a75e635..181b60785f4ef 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.NoOpDispatcherBootstrap; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.rpc.RpcService; @@ -65,7 +65,7 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( recoveredJobs, (dispatcherGateway, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), - PartialDispatcherServicesWithJobGraphStore.from( + PartialDispatcherServicesWithJobPersistenceComponents.from( partialDispatcherServices, jobGraphWriter)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index 64790d31d5c87..f349cea10afc5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -29,7 +29,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherServices; import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; @@ -225,8 +225,8 @@ public Dispatcher createDispatcher( DispatcherId fencingToken, Collection recoveredJobs, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobGraphStore - partialDispatcherServicesWithJobGraphStore) + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents) throws Exception { return new StandaloneDispatcher( rpcService, @@ -234,7 +234,8 @@ public Dispatcher createDispatcher( recoveredJobs, dispatcherBootstrapFactory, DispatcherServices.from( - partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); + partialDispatcherServicesWithJobPersistenceComponents, + jobManagerRunnerFactory)); } } From 31d9caa01f4e9ff00575891731326aac498bc26c Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 29 Nov 2021 13:32:55 +0100 Subject: [PATCH 21/49] [FLINK-25430][runtime] Integrates JobResultStore initialization along JobGraphStore initialization The goal is to do the recovery of dirty JobResults from the JobResultStore along the JobGraph recovery. Both will be pass through the call chain up to the Dispatcher. The *DispatcherLeaderProcess implementations take care of this recovery and make sure that any JobGraph that is recovered but has a matching dirty JobResult is not passed on into the Dispatcher. There's a dedicated Precondition checking this in the Dispatcher constructor itself. --- ...cationDispatcherGatewayServiceFactory.java | 9 +- .../ApplicationDispatcherBootstrapITCase.java | 65 ++++ .../testjar/ErrorHandlingSubmissionJob.java | 75 +++++ .../org/apache/flink/util/CollectionUtil.java | 5 + .../apache/flink/util/CollectionUtilTest.java | 29 +- .../flink/runtime/dispatcher/Dispatcher.java | 26 +- .../runtime/dispatcher/DispatcherFactory.java | 2 + .../dispatcher/DispatcherServices.java | 10 + .../dispatcher/JobDispatcherFactory.java | 14 +- .../runtime/dispatcher/MiniDispatcher.java | 10 +- ...rServicesWithJobPersistenceComponents.java | 17 +- .../dispatcher/SessionDispatcherFactory.java | 3 + .../dispatcher/StandaloneDispatcher.java | 3 + .../AbstractDispatcherLeaderProcess.java | 26 +- ...efaultDispatcherGatewayServiceFactory.java | 9 +- .../runner/JobDispatcherLeaderProcess.java | 26 +- .../JobDispatcherLeaderProcessFactory.java | 34 ++- ...DispatcherLeaderProcessFactoryFactory.java | 75 ++++- .../SessionDispatcherLeaderProcess.java | 80 ++++- ...SessionDispatcherLeaderProcessFactory.java | 1 + ...ervicesJobPersistenceComponentFactory.java | 15 +- .../JobPersistenceComponentFactory.java | 13 +- .../dispatcher/AbstractDispatcherTest.java | 26 +- .../dispatcher/DispatcherFailoverITCase.java | 2 + .../DispatcherResourceCleanupTest.java | 2 + .../runtime/dispatcher/DispatcherTest.java | 13 + .../dispatcher/MiniDispatcherTest.java | 49 ++- .../runtime/dispatcher/TestingDispatcher.java | 3 + .../TestingJobManagerRunnerFactory.java | 4 + .../runner/DefaultDispatcherRunnerITCase.java | 11 +- ...atcherLeaderProcessFactoryFactoryTest.java | 134 +++++++++ .../SessionDispatcherLeaderProcessTest.java | 282 ++++++++++++------ .../TestingDispatcherServiceFactory.java | 86 ------ .../ZooKeeperDefaultDispatcherRunnerTest.java | 14 +- ...TestingJobPersistenceComponentFactory.java | 48 +++ 35 files changed, 968 insertions(+), 253 deletions(-) create mode 100644 flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java index 53d998b1e6d06..6df7118aaa1dd 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java @@ -29,8 +29,10 @@ import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; import org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess; import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherGatewayService; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkRuntimeException; @@ -83,7 +85,9 @@ public ApplicationDispatcherGatewayServiceFactory( public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, - JobGraphWriter jobGraphWriter) { + Collection recoveredDirtyJobResults, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore) { final List recoveredJobIds = getRecoveredJobIds(recoveredJobs); @@ -94,6 +98,7 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, (dispatcherGateway, scheduledExecutor, errorHandler) -> new ApplicationDispatcherBootstrap( application, @@ -103,7 +108,7 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( scheduledExecutor, errorHandler), PartialDispatcherServicesWithJobPersistenceComponents.from( - partialDispatcherServices, jobGraphWriter)); + partialDispatcherServices, jobGraphWriter, jobResultStore)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java index 8c0067b279e49..8caceb1db5fad 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java @@ -26,11 +26,14 @@ import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.client.testjar.BlockingJob; +import org.apache.flink.client.testjar.ErrorHandlingSubmissionJob; import org.apache.flink.client.testjar.FailingJob; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.PipelineOptionsInternal; +import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunnerFactory; @@ -38,7 +41,10 @@ import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.executiongraph.ErrorInfo; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; @@ -48,6 +54,7 @@ import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; import org.apache.flink.runtime.rest.JobRestEndpointFactory; import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -153,6 +160,64 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc } } + @Test + public void testDirtyJobResultRecoveryInApplicationMode() throws Exception { + final Deadline deadline = Deadline.fromNow(TIMEOUT); + final Configuration configuration = new Configuration(); + configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); + configuration.set(DeploymentOptions.TARGET, EmbeddedExecutor.NAME); + configuration.set(ClientOptions.CLIENT_RETRY_PERIOD, Duration.ofMillis(100)); + final TestingMiniClusterConfiguration clusterConfiguration = + TestingMiniClusterConfiguration.newBuilder() + .setConfiguration(configuration) + .build(); + + // having a dirty entry in the JobResultStore should make the ApplicationDispatcherBootstrap + // implementation fail to submit the job + final JobResultStore jobResultStore = new EmbeddedJobResultStore(); + jobResultStore.createDirtyResult( + new JobResultEntry( + TestingJobResultStore.createSuccessfulJobResult( + ApplicationDispatcherBootstrap.ZERO_JOB_ID))); + final EmbeddedHaServicesWithLeadershipControl haServices = + new EmbeddedHaServicesWithLeadershipControl(TestingUtils.defaultExecutor()) { + + @Override + public JobResultStore getJobResultStore() { + return jobResultStore; + } + }; + + final TestingMiniCluster.Builder clusterBuilder = + TestingMiniCluster.newBuilder(clusterConfiguration) + .setHighAvailabilityServicesSupplier(() -> haServices) + .setDispatcherResourceManagerComponentFactorySupplier( + createApplicationModeDispatcherResourceManagerComponentFactorySupplier( + clusterConfiguration.getConfiguration(), + ErrorHandlingSubmissionJob.createPackagedProgram())); + try (final MiniCluster cluster = clusterBuilder.build()) { + // start mini cluster and submit the job + cluster.start(); + + // the cluster should shut down automatically once the application completes + awaitClusterStopped(cluster, deadline); + } + + FlinkAssertions.assertThatChainOfCauses(ErrorHandlingSubmissionJob.getSubmissionException()) + .as( + "The job's main method shouldn't have been succeeded due to a DuplicateJobSubmissionException.") + .hasAtLeastOneElementOfType(DuplicateJobSubmissionException.class); + + assertThat( + jobResultStore.hasDirtyJobResultEntry( + ApplicationDispatcherBootstrap.ZERO_JOB_ID)) + .isTrue(); + assertThat( + jobResultStore.hasCleanJobResultEntry( + ApplicationDispatcherBootstrap.ZERO_JOB_ID)) + .isFalse(); + } + @Test public void testSubmitFailedJobOnApplicationError() throws Exception { final Deadline deadline = Deadline.fromNow(TIMEOUT); diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java new file mode 100644 index 0000000000000..83492c257eea2 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java @@ -0,0 +1,75 @@ +/* + * 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.flink.client.testjar; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.client.cli.CliFrontendTestUtils; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.util.FlinkException; + +import java.io.File; +import java.io.FileNotFoundException; +import java.net.MalformedURLException; +import java.util.Arrays; +import java.util.Collections; +import java.util.concurrent.atomic.AtomicReference; + +/** + * {@code ErrorHandlingSubmissionJob} provides a factory method for creating a {@link + * PackagedProgram} that monitors the job submission within the job's {@code main} method. + */ +public class ErrorHandlingSubmissionJob { + + private static final AtomicReference SUBMISSION_EXCEPTION = new AtomicReference<>(); + + public static PackagedProgram createPackagedProgram() throws FlinkException { + try { + return PackagedProgram.newBuilder() + .setUserClassPaths( + Collections.singletonList( + new File(CliFrontendTestUtils.getTestJarPath()) + .toURI() + .toURL())) + .setEntryPointClassName(ErrorHandlingSubmissionJob.class.getName()) + .build(); + } catch (ProgramInvocationException | FileNotFoundException | MalformedURLException e) { + throw new FlinkException("Could not load the provided entrypoint class.", e); + } + } + + public static void main(String[] args) throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.fromCollection(Arrays.asList(1, 2, 3)) + .map(element -> element + 1) + .output(new DiscardingOutputFormat<>()); + + try { + env.execute(); + } catch (Exception e) { + SUBMISSION_EXCEPTION.set(e); + throw e; + } + } + + public static Exception getSubmissionException() { + return SUBMISSION_EXCEPTION.get(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java index b87682a80c242..8ee274ae97c76 100644 --- a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java @@ -29,6 +29,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.Function; @@ -55,6 +56,10 @@ public static boolean isNullOrEmpty(Map map) { return map == null || map.isEmpty(); } + public static Set ofNullable(@Nullable T obj) { + return obj == null ? Collections.emptySet() : Collections.singleton(obj); + } + public static Stream mapWithIndex( Collection input, final BiFunction mapper) { final AtomicInteger count = new AtomicInteger(0); diff --git a/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java b/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java index 4b069a09d1683..abeec238879b6 100644 --- a/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java @@ -18,14 +18,17 @@ package org.apache.flink.util; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; import java.util.List; +import static org.assertj.core.api.Assertions.assertThat; + /** Tests for java collection utilities. */ +@ExtendWith(TestLoggerExtension.class) public class CollectionUtilTest { @Test @@ -33,12 +36,20 @@ public void testPartition() { List list = Arrays.asList(1, 2, 3, 4); Collection> partitioned = CollectionUtil.partition(list, 4); - Assert.assertEquals( - "List partitioned into the an incorrect number of partitions", - 4, - partitioned.size()); - for (List partition : partitioned) { - Assert.assertEquals("Unexpected number of elements in partition", 1, partition.size()); - } + assertThat(partitioned) + .as("List partitioned into the an incorrect number of partitions") + .hasSize(4); + assertThat(partitioned).allSatisfy(partition -> assertThat(partition).hasSize(1)); + } + + @Test + public void testOfNullableWithNull() { + assertThat(CollectionUtil.ofNullable(null)).isEmpty(); + } + + @Test + public void testFromNullableWithObject() { + final Object element = new Object(); + assertThat(CollectionUtil.ofNullable(element)).singleElement().isEqualTo(element); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 6eb12adedccc4..3d5e2eebdbafb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -90,6 +90,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; @@ -157,11 +158,13 @@ public Dispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobs, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { super(rpcService, RpcServiceUtils.createRandomName(DISPATCHER_NAME), fencingToken); checkNotNull(dispatcherServices); + assertRecoveredJobsAndDirtyJobResults(recoveredJobs, recoveredDirtyJobs); this.configuration = dispatcherServices.getConfiguration(); this.highAvailabilityServices = dispatcherServices.getHighAvailabilityServices(); @@ -171,6 +174,7 @@ public Dispatcher( this.blobServer = dispatcherServices.getBlobServer(); this.fatalErrorHandler = dispatcherServices.getFatalErrorHandler(); this.jobGraphWriter = dispatcherServices.getJobGraphWriter(); + this.jobResultStore = dispatcherServices.getJobResultStore(); this.jobManagerMetricGroup = dispatcherServices.getJobManagerMetricGroup(); this.metricServiceQueryAddress = dispatcherServices.getMetricQueryServiceAddress(); this.ioExecutor = dispatcherServices.getIoExecutor(); @@ -179,8 +183,6 @@ public Dispatcher( JobManagerSharedServices.fromConfiguration( configuration, blobServer, fatalErrorHandler); - this.jobResultStore = highAvailabilityServices.getJobResultStore(); - runningJobs = new HashMap<>(16); this.historyServerArchivist = dispatcherServices.getHistoryServerArchivist(); @@ -196,6 +198,7 @@ public Dispatcher( this.dispatcherBootstrapFactory = checkNotNull(dispatcherBootstrapFactory); this.recoveredJobs = new HashSet<>(recoveredJobs); + this.blobServer.retainJobs( recoveredJobs.stream().map(JobGraph::getJobID).collect(Collectors.toSet())); @@ -246,6 +249,25 @@ private void startDispatcherServices() throws Exception { } } + private static void assertRecoveredJobsAndDirtyJobResults( + Collection recoveredJobs, Collection recoveredDirtyJobResults) { + final Set jobIdsOfFinishedJobs = + recoveredDirtyJobResults.stream() + .map(JobResult::getJobId) + .collect(Collectors.toSet()); + + final boolean noRecoveredJobGraphHasDirtyJobResult = + recoveredJobs.stream() + .noneMatch( + recoveredJobGraph -> + jobIdsOfFinishedJobs.contains( + recoveredJobGraph.getJobID())); + + Preconditions.checkArgument( + noRecoveredJobGraphHasDirtyJobResult, + "There should be no overlap between the recovered JobGraphs and the passed dirty JobResults based on their job ID."); + } + private void startRecoveredJobs() { for (JobGraph recoveredJob : recoveredJobs) { runRecoveredJob(recoveredJob); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java index 9fa086e36d407..547bda32de2c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import java.util.Collection; @@ -31,6 +32,7 @@ Dispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, PartialDispatcherServicesWithJobPersistenceComponents partialDispatcherServicesWithJobPersistenceComponents) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java index 60fda8219b9a1..5e3e32982f551 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; @@ -60,6 +61,8 @@ public class DispatcherServices { private final JobGraphWriter jobGraphWriter; + private final JobResultStore jobResultStore; + private final JobManagerRunnerFactory jobManagerRunnerFactory; private final Executor ioExecutor; @@ -77,6 +80,7 @@ public class DispatcherServices { DispatcherOperationCaches operationCaches, JobManagerMetricGroup jobManagerMetricGroup, JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore, JobManagerRunnerFactory jobManagerRunnerFactory, Executor ioExecutor) { this.configuration = Preconditions.checkNotNull(configuration, "Configuration"); @@ -97,6 +101,7 @@ public class DispatcherServices { this.jobManagerMetricGroup = Preconditions.checkNotNull(jobManagerMetricGroup, "JobManagerMetricGroup"); this.jobGraphWriter = Preconditions.checkNotNull(jobGraphWriter, "JobGraphWriter"); + this.jobResultStore = Preconditions.checkNotNull(jobResultStore, "JobResultStore"); this.jobManagerRunnerFactory = Preconditions.checkNotNull(jobManagerRunnerFactory, "JobManagerRunnerFactory"); this.ioExecutor = Preconditions.checkNotNull(ioExecutor, "IOExecutor"); @@ -151,6 +156,10 @@ public JobGraphWriter getJobGraphWriter() { return jobGraphWriter; } + public JobResultStore getJobResultStore() { + return jobResultStore; + } + JobManagerRunnerFactory getJobManagerRunnerFactory() { return jobManagerRunnerFactory; } @@ -181,6 +190,7 @@ public static DispatcherServices from( .getJobManagerMetricGroupFactory() .create(), partialDispatcherServicesWithJobPersistenceComponents.getJobGraphWriter(), + partialDispatcherServicesWithJobPersistenceComponents.getJobResultStore(), jobManagerRunnerFactory, partialDispatcherServicesWithJobPersistenceComponents.getIoExecutor()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java index 223093c088b51..20dc32ba4e32b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java @@ -21,7 +21,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; @@ -38,11 +40,18 @@ public MiniDispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, PartialDispatcherServicesWithJobPersistenceComponents partialDispatcherServicesWithJobPersistenceComponents) throws Exception { - final JobGraph jobGraph = Iterables.getOnlyElement(recoveredJobs); + final JobGraph recoveredJobGraph = Iterables.getOnlyElement(recoveredJobs, null); + final JobResult recoveredDirtyJob = + Iterables.getOnlyElement(recoveredDirtyJobResults, null); + + Preconditions.checkArgument( + recoveredJobGraph == null ^ recoveredDirtyJob == null, + "Either the JobGraph or the recovered JobResult needs to be specified."); final Configuration configuration = partialDispatcherServicesWithJobPersistenceComponents.getConfiguration(); @@ -56,7 +65,8 @@ public MiniDispatcher createDispatcher( DispatcherServices.from( partialDispatcherServicesWithJobPersistenceComponents, JobMasterServiceLeadershipRunnerFactory.INSTANCE), - jobGraph, + recoveredJobGraph, + recoveredDirtyJob, dispatcherBootstrapFactory, executionMode); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java index 83f173db8c750..bc5776e1948e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -29,9 +29,11 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.FlinkException; -import java.util.Collections; +import javax.annotation.Nullable; + import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -55,14 +57,16 @@ public MiniDispatcher( RpcService rpcService, DispatcherId fencingToken, DispatcherServices dispatcherServices, - JobGraph jobGraph, + @Nullable JobGraph jobGraph, + @Nullable JobResult recoveredDirtyJob, DispatcherBootstrapFactory dispatcherBootstrapFactory, JobClusterEntrypoint.ExecutionMode executionMode) throws Exception { super( rpcService, fencingToken, - Collections.singleton(jobGraph), + CollectionUtil.ofNullable(jobGraph), + CollectionUtil.ofNullable(recoveredDirtyJob), dispatcherBootstrapFactory, dispatcherServices); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java index 07b87a9637232..c6fe7b7eaa579 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -36,6 +37,7 @@ public class PartialDispatcherServicesWithJobPersistenceComponents extends PartialDispatcherServices { private final JobGraphWriter jobGraphWriter; + private final JobResultStore jobResultStore; private PartialDispatcherServicesWithJobPersistenceComponents( Configuration configuration, @@ -50,7 +52,8 @@ private PartialDispatcherServicesWithJobPersistenceComponents( @Nullable String metricQueryServiceAddress, Executor ioExecutor, DispatcherOperationCaches operationCaches, - JobGraphWriter jobGraphWriter) { + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore) { super( configuration, highAvailabilityServices, @@ -65,14 +68,21 @@ private PartialDispatcherServicesWithJobPersistenceComponents( ioExecutor, operationCaches); this.jobGraphWriter = jobGraphWriter; + this.jobResultStore = jobResultStore; } public JobGraphWriter getJobGraphWriter() { return jobGraphWriter; } + public JobResultStore getJobResultStore() { + return jobResultStore; + } + public static PartialDispatcherServicesWithJobPersistenceComponents from( - PartialDispatcherServices partialDispatcherServices, JobGraphWriter jobGraphWriter) { + PartialDispatcherServices partialDispatcherServices, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore) { return new PartialDispatcherServicesWithJobPersistenceComponents( partialDispatcherServices.getConfiguration(), partialDispatcherServices.getHighAvailabilityServices(), @@ -86,6 +96,7 @@ public static PartialDispatcherServicesWithJobPersistenceComponents from( partialDispatcherServices.getMetricQueryServiceAddress(), partialDispatcherServices.getIoExecutor(), partialDispatcherServices.getOperationCaches(), - jobGraphWriter); + jobGraphWriter, + jobResultStore); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java index 88261ae6ed78f..bd7bbad33f91c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import java.util.Collection; @@ -32,6 +33,7 @@ public StandaloneDispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, PartialDispatcherServicesWithJobPersistenceComponents partialDispatcherServicesWithJobPersistenceComponents) @@ -41,6 +43,7 @@ public StandaloneDispatcher createDispatcher( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, dispatcherBootstrapFactory, DispatcherServices.from( partialDispatcherServicesWithJobPersistenceComponents, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java index 1763bb75fc494..3bb10734ed8ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import java.util.Collection; @@ -34,6 +35,7 @@ public StandaloneDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { @@ -41,6 +43,7 @@ public StandaloneDispatcher( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, dispatcherBootstrapFactory, dispatcherServices); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java index b5bf60f19a7bb..faf161e1ba126 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java @@ -24,8 +24,10 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.util.AutoCloseableAsync; @@ -176,16 +178,14 @@ private void handleUnexpectedDispatcherServiceTermination( createdDispatcherService .getTerminationFuture() .whenComplete( - (ignored, throwable) -> { - runIfStateIs( - State.RUNNING, - () -> { - handleError( - new FlinkException( - "Unexpected termination of DispatcherService.", - throwable)); - }); - }); + (ignored, throwable) -> + runIfStateIs( + State.RUNNING, + () -> + handleError( + new FlinkException( + "Unexpected termination of DispatcherService.", + throwable)))); } final Optional supplyUnsynchronizedIfRunning(Supplier supplier) { @@ -257,9 +257,11 @@ protected enum State { /** Factory for {@link DispatcherGatewayService}. */ public interface DispatcherGatewayServiceFactory { DispatcherGatewayService create( - DispatcherId fencingToken, + DispatcherId dispatcherId, Collection recoveredJobs, - JobGraphWriter jobGraphWriter); + Collection recoveredDirtyJobResults, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore); } /** An accessor of the {@link DispatcherGateway}. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java index 181b60785f4ef..40498fee7df53 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java @@ -24,8 +24,10 @@ import org.apache.flink.runtime.dispatcher.NoOpDispatcherBootstrap; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkRuntimeException; @@ -54,7 +56,9 @@ class DefaultDispatcherGatewayServiceFactory public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, - JobGraphWriter jobGraphWriter) { + Collection recoveredDirtyJobResults, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore) { final Dispatcher dispatcher; try { @@ -63,10 +67,11 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, (dispatcherGateway, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), PartialDispatcherServicesWithJobPersistenceComponents.from( - partialDispatcherServices, jobGraphWriter)); + partialDispatcherServices, jobGraphWriter, jobResultStore)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java index f50b396ba0a36..6f56e2cf7db48 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java @@ -19,11 +19,16 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.ThrowingJobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; -import java.util.Collections; import java.util.UUID; /** {@link DispatcherLeaderProcess} implementation for the per-job mode. */ @@ -31,16 +36,23 @@ public class JobDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess private final DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; - private final JobGraph jobGraph; + @Nullable private final JobGraph jobGraph; + @Nullable private final JobResult recoveredDirtyJobResult; + + private final JobResultStore jobResultStore; JobDispatcherLeaderProcess( UUID leaderSessionId, DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, - JobGraph jobGraph, + @Nullable JobGraph jobGraph, + @Nullable JobResult recoveredDirtyJobResult, + JobResultStore jobResultStore, FatalErrorHandler fatalErrorHandler) { super(leaderSessionId, fatalErrorHandler); - this.jobGraph = jobGraph; this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; + this.jobGraph = jobGraph; + this.recoveredDirtyJobResult = recoveredDirtyJobResult; + this.jobResultStore = Preconditions.checkNotNull(jobResultStore); } @Override @@ -48,8 +60,10 @@ protected void onStart() { final DispatcherGatewayService dispatcherService = dispatcherGatewayServiceFactory.create( DispatcherId.fromUuid(getLeaderSessionId()), - Collections.singleton(jobGraph), - ThrowingJobGraphWriter.INSTANCE); + CollectionUtil.ofNullable(jobGraph), + CollectionUtil.ofNullable(recoveredDirtyJobResult), + ThrowingJobGraphWriter.INSTANCE, + jobResultStore); completeDispatcherSetup(dispatcherService); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java index 2081f60c69ea5..1716a1e961a19 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java @@ -18,33 +18,61 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; import java.util.UUID; /** Factory for the {@link JobDispatcherLeaderProcess}. */ public class JobDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { + private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; - private final JobGraph jobGraph; + @Nullable private final JobGraph jobGraph; + @Nullable private final JobResult recoveredDirtyJobResult; + + private final JobResultStore jobResultStore; private final FatalErrorHandler fatalErrorHandler; JobDispatcherLeaderProcessFactory( AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, - JobGraph jobGraph, + @Nullable JobGraph jobGraph, + @Nullable JobResult recoveredDirtyJobResult, + JobResultStore jobResultStore, FatalErrorHandler fatalErrorHandler) { this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; this.jobGraph = jobGraph; + this.recoveredDirtyJobResult = recoveredDirtyJobResult; + this.jobResultStore = Preconditions.checkNotNull(jobResultStore); this.fatalErrorHandler = fatalErrorHandler; } @Override public DispatcherLeaderProcess create(UUID leaderSessionID) { return new JobDispatcherLeaderProcess( - leaderSessionID, dispatcherGatewayServiceFactory, jobGraph, fatalErrorHandler); + leaderSessionID, + dispatcherGatewayServiceFactory, + jobGraph, + recoveredDirtyJobResult, + jobResultStore, + fatalErrorHandler); + } + + @Nullable + JobGraph getJobGraph() { + return this.jobGraph; + } + + @Nullable + JobResult getRecoveredDirtyJobResult() { + return this.recoveredDirtyJobResult; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java index 424774c3b4b70..2ebc3bce8fcbd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java @@ -18,30 +18,47 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.dispatcher.JobDispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.Executor; +import java.util.stream.Collectors; /** Factory for the {@link JobDispatcherLeaderProcessFactory}. */ public class JobDispatcherLeaderProcessFactoryFactory implements DispatcherLeaderProcessFactoryFactory { + private static final Logger LOG = + LoggerFactory.getLogger(JobDispatcherLeaderProcessFactoryFactory.class); + private final JobGraphRetriever jobGraphRetriever; - private JobDispatcherLeaderProcessFactoryFactory(JobGraphRetriever jobGraphRetriever) { + @VisibleForTesting + JobDispatcherLeaderProcessFactoryFactory(JobGraphRetriever jobGraphRetriever) { this.jobGraphRetriever = jobGraphRetriever; } @Override - public DispatcherLeaderProcessFactory createFactory( + public JobDispatcherLeaderProcessFactory createFactory( JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, @@ -52,22 +69,70 @@ public DispatcherLeaderProcessFactory createFactory( try { jobGraph = - jobGraphRetriever.retrieveJobGraph( - partialDispatcherServices.getConfiguration()); + Preconditions.checkNotNull( + jobGraphRetriever.retrieveJobGraph( + partialDispatcherServices.getConfiguration())); } catch (FlinkException e) { throw new FlinkRuntimeException("Could not retrieve the JobGraph.", e); } + final JobResultStore jobResultStore = jobPersistenceComponentFactory.createJobResultStore(); + final Collection recoveredDirtyJobResults = getDirtyJobResults(jobResultStore); + + final Optional maybeRecoveredDirtyJobResult = + extractDirtyJobResult(recoveredDirtyJobResults, jobGraph); + final Optional maybeJobGraph = + getJobGraphBasedOnDirtyJobResults(jobGraph, recoveredDirtyJobResults); + final DefaultDispatcherGatewayServiceFactory defaultDispatcherServiceFactory = new DefaultDispatcherGatewayServiceFactory( JobDispatcherFactory.INSTANCE, rpcService, partialDispatcherServices); return new JobDispatcherLeaderProcessFactory( - defaultDispatcherServiceFactory, jobGraph, fatalErrorHandler); + defaultDispatcherServiceFactory, + maybeJobGraph.orElse(null), + maybeRecoveredDirtyJobResult.orElse(null), + jobResultStore, + fatalErrorHandler); } public static JobDispatcherLeaderProcessFactoryFactory create( JobGraphRetriever jobGraphRetriever) { return new JobDispatcherLeaderProcessFactoryFactory(jobGraphRetriever); } + + private static Collection getDirtyJobResults(JobResultStore jobResultStore) { + try { + return jobResultStore.getDirtyResults(); + } catch (IOException e) { + throw new FlinkRuntimeException( + "Could not retrieve the JobResults of dirty jobs from the underlying JobResultStore.", + e); + } + } + + private static Optional extractDirtyJobResult( + Collection dirtyJobResults, JobGraph jobGraph) { + Optional actualDirtyJobResult = Optional.empty(); + for (JobResult dirtyJobResult : dirtyJobResults) { + if (dirtyJobResult.getJobId().equals(jobGraph.getJobID())) { + actualDirtyJobResult = Optional.of(dirtyJobResult); + } else { + LOG.warn( + "Unexpected dirty JobResultStore entry: Job '{}' is listed as dirty, isn't part of this single-job cluster, though.", + dirtyJobResult.getJobId()); + } + } + + return actualDirtyJobResult; + } + + private static Optional getJobGraphBasedOnDirtyJobResults( + JobGraph jobGraph, Collection dirtyJobResults) { + final Set jobIdsOfFinishedJobs = + dirtyJobResults.stream().map(JobResult::getJobId).collect(Collectors.toSet()); + return jobIdsOfFinishedJobs.contains(jobGraph.getJobID()) + ? Optional.empty() + : Optional.of(jobGraph); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java index efc6b2098a997..6f9d0bf2fd377 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java @@ -23,8 +23,10 @@ import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.util.ExceptionUtils; @@ -33,14 +35,17 @@ import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.function.FunctionUtils; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; +import java.util.stream.Collectors; /** * Process which encapsulates the job recovery logic and life cycle management of a {@link @@ -53,6 +58,8 @@ public class SessionDispatcherLeaderProcess extends AbstractDispatcherLeaderProc private final JobGraphStore jobGraphStore; + private final JobResultStore jobResultStore; + private final Executor ioExecutor; private CompletableFuture onGoingRecoveryOperation = FutureUtils.completedVoidFuture(); @@ -61,12 +68,14 @@ private SessionDispatcherLeaderProcess( UUID leaderSessionId, DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, JobGraphStore jobGraphStore, + JobResultStore jobResultStore, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { super(leaderSessionId, fatalErrorHandler); this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; this.jobGraphStore = jobGraphStore; + this.jobResultStore = jobResultStore; this.ioExecutor = ioExecutor; } @@ -75,9 +84,7 @@ protected void onStart() { startServices(); onGoingRecoveryOperation = - recoverJobsAsync() - .thenAccept(this::createDispatcherIfRunning) - .handle(this::onErrorIfRunning); + createDispatcherBasedOnRecoveredJobGraphsAndRecoveredDirtyJobResults(); } private void startServices() { @@ -92,34 +99,56 @@ private void startServices() { } } - private void createDispatcherIfRunning(Collection jobGraphs) { - runIfStateIs(State.RUNNING, () -> createDispatcher(jobGraphs)); + private void createDispatcherIfRunning( + Collection jobGraphs, Collection recoveredDirtyJobResults) { + runIfStateIs(State.RUNNING, () -> createDispatcher(jobGraphs, recoveredDirtyJobResults)); } - private void createDispatcher(Collection jobGraphs) { + private void createDispatcher( + Collection jobGraphs, Collection recoveredDirtyJobResults) { final DispatcherGatewayService dispatcherService = dispatcherGatewayServiceFactory.create( - DispatcherId.fromUuid(getLeaderSessionId()), jobGraphs, jobGraphStore); + DispatcherId.fromUuid(getLeaderSessionId()), + jobGraphs, + recoveredDirtyJobResults, + jobGraphStore, + jobResultStore); completeDispatcherSetup(dispatcherService); } - private CompletableFuture> recoverJobsAsync() { - return CompletableFuture.supplyAsync(this::recoverJobsIfRunning, ioExecutor); + private CompletableFuture + createDispatcherBasedOnRecoveredJobGraphsAndRecoveredDirtyJobResults() { + final CompletableFuture> dirtyJobsFuture = + CompletableFuture.supplyAsync(this::getDirtyJobResultsIfRunning, ioExecutor); + + return dirtyJobsFuture + .thenApplyAsync( + dirtyJobs -> + this.recoverJobsIfRunning( + dirtyJobs.stream() + .map(JobResult::getJobId) + .collect(Collectors.toSet())), + ioExecutor) + .thenAcceptBoth(dirtyJobsFuture, this::createDispatcherIfRunning) + .handle(this::onErrorIfRunning); } - private Collection recoverJobsIfRunning() { - return supplyUnsynchronizedIfRunning(this::recoverJobs).orElse(Collections.emptyList()); + private Collection recoverJobsIfRunning(Set recoveredDirtyJobResults) { + return supplyUnsynchronizedIfRunning(() -> recoverJobs(recoveredDirtyJobResults)) + .orElse(Collections.emptyList()); } - private Collection recoverJobs() { - log.info("Recover all persisted job graphs."); + private Collection recoverJobs(Set recoveredDirtyJobResults) { + log.info("Recover all persisted job graphs that are not finished, yet."); final Collection jobIds = getJobIds(); final Collection recoveredJobGraphs = new ArrayList<>(); for (JobID jobId : jobIds) { - recoveredJobGraphs.add(recoverJob(jobId)); + if (!recoveredDirtyJobResults.contains(jobId)) { + recoveredJobGraphs.add(recoverJob(jobId)); + } } log.info("Successfully recovered {} persisted job graphs.", recoveredJobGraphs.size()); @@ -145,6 +174,21 @@ private JobGraph recoverJob(JobID jobId) { } } + private Collection getDirtyJobResultsIfRunning() { + return supplyUnsynchronizedIfRunning(this::getDirtyJobResults) + .orElse(Collections.emptyList()); + } + + private Collection getDirtyJobResults() { + try { + return jobResultStore.getDirtyResults(); + } catch (IOException e) { + throw new FlinkRuntimeException( + "Could not retrieve JobResults of globally-terminated jobs from JobResultStore", + e); + } + } + @Override protected CompletableFuture onClose() { return CompletableFuture.runAsync(this::stopServices, ioExecutor); @@ -261,9 +305,15 @@ public static SessionDispatcherLeaderProcess create( UUID leaderSessionId, DispatcherGatewayServiceFactory dispatcherFactory, JobGraphStore jobGraphStore, + JobResultStore jobResultStore, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { return new SessionDispatcherLeaderProcess( - leaderSessionId, dispatcherFactory, jobGraphStore, ioExecutor, fatalErrorHandler); + leaderSessionId, + dispatcherFactory, + jobGraphStore, + jobResultStore, + ioExecutor, + fatalErrorHandler); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java index 18a6111194d01..8c08bb0d0b39c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java @@ -53,6 +53,7 @@ public DispatcherLeaderProcess create(UUID leaderSessionID) { leaderSessionID, dispatcherGatewayServiceFactory, jobPersistenceComponentFactory.createJobGraphStore(), + jobPersistenceComponentFactory.createJobResultStore(), ioExecutor, fatalErrorHandler); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java index a59a83eee14d7..05dce66c3aac4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java @@ -19,7 +19,9 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.function.SupplierWithException; /** * {@link JobPersistenceComponentFactory} implementation which creates a {@link JobGraphStore} using @@ -35,13 +37,22 @@ public HaServicesJobPersistenceComponentFactory( @Override public JobGraphStore createJobGraphStore() { + return create(highAvailabilityServices::getJobGraphStore, JobGraphStore.class); + } + + @Override + public JobResultStore createJobResultStore() { + return create(highAvailabilityServices::getJobResultStore, JobResultStore.class); + } + + private T create(SupplierWithException supplier, Class clazz) { try { - return highAvailabilityServices.getJobGraphStore(); + return supplier.get(); } catch (Exception e) { throw new FlinkRuntimeException( String.format( "Could not create %s from %s.", - JobGraphStore.class.getSimpleName(), + clazz.getSimpleName(), highAvailabilityServices.getClass().getSimpleName()), e); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java index b4ce131ade655..e49c12a1e65a2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java @@ -18,13 +18,22 @@ package org.apache.flink.runtime.jobmanager; -/** Factory for {@link JobGraphStore}. */ +import org.apache.flink.runtime.highavailability.JobResultStore; + +/** Factory for components that are responsible for persisting a job for recovery. */ public interface JobPersistenceComponentFactory { /** * Creates a {@link JobGraphStore}. * - * @return a {@link JobGraphStore} instance + * @return a {@code JobGraphStore} instance */ JobGraphStore createJobGraphStore(); + + /** + * Creates {@link JobResultStore} instances. + * + * @return a {@code JobResultStore} instance. + */ + JobResultStore createJobResultStore(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java index 67ec96e566eea..f277e2b2e1269 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java @@ -27,11 +27,13 @@ import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; @@ -133,7 +135,9 @@ public class TestingDispatcherBuilder { private Collection initialJobGraphs = Collections.emptyList(); - private final DispatcherBootstrapFactory dispatcherBootstrapFactory = + private Collection dirtyJobResults = Collections.emptyList(); + + private DispatcherBootstrapFactory dispatcherBootstrapFactory = (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(); private HeartbeatServices heartbeatServices = AbstractDispatcherTest.this.heartbeatServices; @@ -145,6 +149,8 @@ public class TestingDispatcherBuilder { private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; + private JobResultStore jobResultStore = new EmbeddedJobResultStore(); + private FatalErrorHandler fatalErrorHandler = testingFatalErrorHandlerResource.getFatalErrorHandler(); @@ -165,6 +171,17 @@ TestingDispatcherBuilder setInitialJobGraphs(Collection initialJobGrap return this; } + TestingDispatcherBuilder setDirtyJobResults(Collection dirtyJobResults) { + this.dirtyJobResults = dirtyJobResults; + return this; + } + + TestingDispatcherBuilder setDispatcherBootstrapFactory( + DispatcherBootstrapFactory dispatcherBootstrapFactory) { + this.dispatcherBootstrapFactory = dispatcherBootstrapFactory; + return this; + } + TestingDispatcherBuilder setJobManagerRunnerFactory( JobManagerRunnerFactory jobManagerRunnerFactory) { this.jobManagerRunnerFactory = jobManagerRunnerFactory; @@ -176,6 +193,11 @@ TestingDispatcherBuilder setJobGraphWriter(JobGraphWriter jobGraphWriter) { return this; } + TestingDispatcherBuilder setJobResultStore(JobResultStore jobResultStore) { + this.jobResultStore = jobResultStore; + return this; + } + public TestingDispatcherBuilder setFatalErrorHandler(FatalErrorHandler fatalErrorHandler) { this.fatalErrorHandler = fatalErrorHandler; return this; @@ -198,6 +220,7 @@ TestingDispatcher build() throws Exception { rpcService, DispatcherId.generate(), initialJobGraphs, + dirtyJobResults, dispatcherBootstrapFactory, new DispatcherServices( configuration, @@ -212,6 +235,7 @@ TestingDispatcher build() throws Exception { new DispatcherOperationCaches(), UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), jobGraphWriter, + jobResultStore, jobManagerRunnerFactory, ForkJoinPool.commonPool())); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java index 535709d26e9a4..ef6be6139c163 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java @@ -231,7 +231,9 @@ private TestingDispatcher createRecoveredDispatcher( .setJobManagerRunnerFactory( JobMasterServiceLeadershipRunnerFactory.INSTANCE) .setJobGraphWriter(haServices.getJobGraphStore()) + .setJobResultStore(haServices.getJobResultStore()) .setInitialJobGraphs(jobGraphs) + .setDirtyJobResults(haServices.getJobResultStore().getDirtyResults()) .setFatalErrorHandler( fatalErrorHandler == null ? testingFatalErrorHandlerResource.getFatalErrorHandler() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index ed96fae91e83b..5b84a63dcc47f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -203,6 +203,7 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th rpcService, DispatcherId.generate(), Collections.emptyList(), + Collections.emptyList(), (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), new DispatcherServices( @@ -218,6 +219,7 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th new DispatcherOperationCaches(), UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), jobGraphWriter, + jobResultStore, jobManagerRunnerFactory, ForkJoinPool.commonPool())); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index c327cbc2fe55f..a131d97f6e045 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -181,6 +181,7 @@ private TestingDispatcher createAndStartDispatcher( .setHeartbeatServices(heartbeatServices) .setJobManagerRunnerFactory(jobManagerRunnerFactory) .setJobGraphWriter(haServices.getJobGraphStore()) + .setJobResultStore(haServices.getJobResultStore()) .build(); dispatcher.start(); return dispatcher; @@ -708,6 +709,18 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { fatalErrorHandler.clearError(); } + @Test(expected = IllegalArgumentException.class) + public void testThatDirtilyFinishedJobsNotBeingRetriggered() throws Exception { + final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + final JobResult jobResult = + TestingJobResultStore.createSuccessfulJobResult(jobGraph.getJobID()); + dispatcher = + new TestingDispatcherBuilder() + .setInitialJobGraphs(Collections.singleton(jobGraph)) + .setDirtyJobResults(Collections.singleton(jobResult)) + .build(); + } + /** Tests that a failing {@link JobManagerRunner} will be properly cleaned up. */ @Test public void testFailingJobManagerRunnerCleanup() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index d76e4bbaa2dbc..af40e71c2c8ec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -18,9 +18,11 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.clusterframework.ApplicationStatus; @@ -38,6 +40,7 @@ import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.util.TestLogger; @@ -49,7 +52,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -149,6 +152,30 @@ public void testSingleJobRecovery() throws Exception { } } + /** Tests that the {@link MiniDispatcher} recovers the single job with which it was started. */ + @Test + public void testDirtyJobResultCleanup() throws Exception { + final OneShotLatch dispatcherBootstrapLatch = new OneShotLatch(); + final MiniDispatcher miniDispatcher = + createMiniDispatcher( + ClusterEntrypoint.ExecutionMode.DETACHED, + null, + TestingJobResultStore.createSuccessfulJobResult(new JobID()), + (dispatcher, scheduledExecutor, errorHandler) -> { + dispatcherBootstrapLatch.trigger(); + return new NoOpDispatcherBootstrap(); + }); + + miniDispatcher.start(); + + try { + dispatcherBootstrapLatch.await(); + assertThat(testingJobManagerRunnerFactory.getQueueSize(), is(0)); + } finally { + RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); + } + } + /** * Tests that in detached mode, the {@link MiniDispatcher} will complete the future that signals * job termination. @@ -244,9 +271,21 @@ public void testShutdownIfJobCancelledInNormalMode() throws Exception { // Utilities // -------------------------------------------------------- - @Nonnull private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode executionMode) throws Exception { + return createMiniDispatcher( + executionMode, + jobGraph, + null, + (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap()); + } + + private MiniDispatcher createMiniDispatcher( + ClusterEntrypoint.ExecutionMode executionMode, + @Nullable JobGraph recoveredJobGraph, + @Nullable JobResult recoveredDirtyJob, + DispatcherBootstrapFactory dispatcherBootstrapFactory) + throws Exception { return new MiniDispatcher( rpcService, DispatcherId.generate(), @@ -263,10 +302,12 @@ private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode exec new DispatcherOperationCaches(), UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), highAvailabilityServices.getJobGraphStore(), + highAvailabilityServices.getJobResultStore(), testingJobManagerRunnerFactory, ForkJoinPool.commonPool()), - jobGraph, - (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), + recoveredJobGraph, + recoveredDirtyJob, + dispatcherBootstrapFactory, executionMode); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java index aab1760b794b6..ba5ecca79a0e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; @@ -40,6 +41,7 @@ class TestingDispatcher extends Dispatcher { RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { @@ -47,6 +49,7 @@ class TestingDispatcher extends Dispatcher { rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, dispatcherBootstrapFactory, dispatcherServices); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java index 384444d06193e..10d9c011b1ecb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java @@ -86,4 +86,8 @@ private TestingJobManagerRunner createTestingJobManagerRunner(JobGraph jobGraph) public TestingJobManagerRunner takeCreatedJobManagerRunner() throws InterruptedException { return createdJobManagerRunner.take(); } + + public int getQueueSize() { + return createdJobManagerRunner.size(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index f349cea10afc5..2ea326b64d2e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -35,9 +35,13 @@ import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.TestingPartialDispatcherServices; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.TestingJobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.rpc.RpcService; @@ -89,6 +93,8 @@ public class DefaultDispatcherRunnerITCase extends TestLogger { private JobGraphStore jobGraphStore; + private JobResultStore jobResultStore; + private PartialDispatcherServices partialDispatcherServices; private DefaultDispatcherRunnerFactory dispatcherRunnerFactory; @@ -102,6 +108,7 @@ public void setup() { dispatcherLeaderElectionService = new TestingLeaderElectionService(); fatalErrorHandler = new TestingFatalErrorHandler(); jobGraphStore = TestingJobGraphStore.newBuilder().build(); + jobResultStore = new EmbeddedJobResultStore(); partialDispatcherServices = TestingPartialDispatcherServices.builder() @@ -224,6 +231,7 @@ public Dispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, PartialDispatcherServicesWithJobPersistenceComponents partialDispatcherServicesWithJobPersistenceComponents) @@ -232,6 +240,7 @@ public Dispatcher createDispatcher( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, dispatcherBootstrapFactory, DispatcherServices.from( partialDispatcherServicesWithJobPersistenceComponents, @@ -247,7 +256,7 @@ private DispatcherRunner createDispatcherRunner() throws Exception { return dispatcherRunnerFactory.createDispatcherRunner( dispatcherLeaderElectionService, fatalErrorHandler, - () -> jobGraphStore, + new TestingJobPersistenceComponentFactory(jobGraphStore, jobResultStore), TestingUtils.defaultExecutor(), rpcServiceResource.getTestingRpcService(), partialDispatcherServices); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java new file mode 100644 index 0000000000000..257294e8c4b9f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java @@ -0,0 +1,134 @@ +/* + * 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.flink.runtime.dispatcher.runner; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.dispatcher.TestingPartialDispatcherServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; +import org.apache.flink.runtime.jobmanager.TestingJobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.testutils.TestingJobResultStore; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.nio.file.Path; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +@ExtendWith(TestLoggerExtension.class) +class JobDispatcherLeaderProcessFactoryFactoryTest { + + @TempDir private Path temporaryFolder; + + @Test + public void testJobGraphWithoutDirtyJobResult() throws IOException { + final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + + final JobDispatcherLeaderProcessFactory factory = + createDispatcherLeaderProcessFactoryFromTestInstance( + jobGraph, null, temporaryFolder); + + assertThat(factory.getJobGraph()).isEqualTo(jobGraph); + assertThat(factory.getRecoveredDirtyJobResult()).isNull(); + } + + @Test + public void testJobGraphWithMatchingDirtyJobResult() throws IOException { + final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + final JobResult jobResult = + TestingJobResultStore.createSuccessfulJobResult(jobGraph.getJobID()); + + final JobDispatcherLeaderProcessFactory factory = + createDispatcherLeaderProcessFactoryFromTestInstance( + jobGraph, jobResult, temporaryFolder); + + assertThat(factory.getJobGraph()).isNull(); + assertThat(factory.getRecoveredDirtyJobResult()).isEqualTo(jobResult); + } + + @Test + public void testJobGraphWithNotMatchingDirtyJobResult() throws IOException { + final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + final JobResult jobResult = TestingJobResultStore.createSuccessfulJobResult(new JobID()); + + final JobDispatcherLeaderProcessFactory factory = + createDispatcherLeaderProcessFactoryFromTestInstance( + jobGraph, jobResult, temporaryFolder); + + assertThat(factory.getJobGraph()).isEqualTo(jobGraph); + assertThat(factory.getRecoveredDirtyJobResult()).isNull(); + } + + @Test + public void testMissingJobGraph() throws IOException { + assertThatThrownBy( + () -> + createDispatcherLeaderProcessFactoryFromTestInstance( + null, + TestingJobResultStore.createSuccessfulJobResult( + new JobID()), + temporaryFolder)) + .isInstanceOf(NullPointerException.class); + } + + private static JobDispatcherLeaderProcessFactory + createDispatcherLeaderProcessFactoryFromTestInstance( + @Nullable JobGraph jobGraph, + @Nullable JobResult dirtyJobResult, + Path storageDir) + throws IOException { + final JobDispatcherLeaderProcessFactoryFactory testInstance = + new JobDispatcherLeaderProcessFactoryFactory(ignoredConfig -> jobGraph); + + final TestingJobResultStore jobResultStore = + TestingJobResultStore.builder() + .withGetDirtyResultsSupplier( + () -> CollectionUtil.ofNullable(dirtyJobResult)) + .build(); + final JobGraphStore jobGraphStore = new StandaloneJobGraphStore(); + return testInstance.createFactory( + new TestingJobPersistenceComponentFactory(jobGraphStore, jobResultStore), + Executors.directExecutor(), + new TestingRpcService(), + TestingPartialDispatcherServices.builder() + .withHighAvailabilityServices( + new TestingHighAvailabilityServicesBuilder() + .setJobGraphStore(jobGraphStore) + .setJobResultStore(jobResultStore) + .build()) + .build(storageDir.toFile(), new Configuration()), + NoOpFatalErrorHandler.INSTANCE); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index 12cdcaa5d1918..2dded1cfaef22 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -22,11 +22,14 @@ import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.testutils.TestingJobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; import org.apache.flink.util.ExceptionUtils; @@ -35,8 +38,8 @@ import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.function.ThrowingConsumer; -import org.apache.flink.util.function.TriFunctionWithException; +import com.google.common.collect.Iterables; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -44,20 +47,26 @@ import org.junit.Test; import java.time.Duration; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; import static org.apache.flink.core.testutils.FlinkMatchers.willNotComplete; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.collection.IsEmptyCollection.empty; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -73,8 +82,10 @@ public class SessionDispatcherLeaderProcessTest extends TestLogger { private TestingFatalErrorHandler fatalErrorHandler; private JobGraphStore jobGraphStore; + private JobResultStore jobResultStore; - private TestingDispatcherServiceFactory dispatcherServiceFactory; + private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory + dispatcherServiceFactory; @BeforeClass public static void setupClass() { @@ -85,7 +96,10 @@ public static void setupClass() { public void setup() { fatalErrorHandler = new TestingFatalErrorHandler(); jobGraphStore = TestingJobGraphStore.newBuilder().build(); - dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder().build(); + jobResultStore = TestingJobResultStore.builder().build(); + dispatcherServiceFactory = + createFactoryBasedOnGenericSupplier( + () -> TestingDispatcherGatewayService.newBuilder().build()); } @After @@ -117,33 +131,116 @@ public void start_afterClose_doesNotHaveAnEffect() throws Exception { } @Test - public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws Exception { + public void testStartTriggeringDispatcherServiceCreation() throws Exception { + dispatcherServiceFactory = + createFactoryBasedOnGenericSupplier( + () -> TestingDispatcherGatewayService.newBuilder().build()); + + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = + createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + assertThat( + dispatcherLeaderProcess.getState(), + is(SessionDispatcherLeaderProcess.State.RUNNING)); + } + } + + @Test + public void testRecoveryWithJobGraphButNoDirtyJobResult() throws Exception { + testJobRecovery( + Collections.singleton(JOB_GRAPH), + Collections.emptySet(), + actualRecoveredJobGraphs -> + assertThat( + Iterables.getOnlyElement(actualRecoveredJobGraphs), is(JOB_GRAPH)), + actualRecoveredDirtyJobResults -> + assertThat(actualRecoveredDirtyJobResults, empty())); + } + + @Test + public void testRecoveryWithJobGraphAndMatchingDirtyJobResult() throws Exception { + final JobResult matchingDirtyJobResult = + TestingJobResultStore.createSuccessfulJobResult(JOB_GRAPH.getJobID()); + + testJobRecovery( + Collections.singleton(JOB_GRAPH), + Collections.singleton(matchingDirtyJobResult), + actualRecoveredJobGraphs -> assertThat(actualRecoveredJobGraphs, empty()), + actualRecoveredDirtyJobResults -> + assertThat( + Iterables.getOnlyElement(actualRecoveredDirtyJobResults), + is(matchingDirtyJobResult))); + } + + @Test + public void testRecoveryWithMultipleJobGraphsAndOneMatchingDirtyJobResult() throws Exception { + final JobResult matchingDirtyJobResult = + TestingJobResultStore.createSuccessfulJobResult(JOB_GRAPH.getJobID()); + final JobGraph otherJobGraph = JobGraphTestUtils.emptyJobGraph(); + + testJobRecovery( + Arrays.asList(otherJobGraph, JOB_GRAPH), + Collections.singleton(matchingDirtyJobResult), + actualRecoveredJobGraphs -> + assertThat( + Iterables.getOnlyElement(actualRecoveredJobGraphs), + is(otherJobGraph)), + actualRecoveredDirtyJobResults -> + assertThat( + Iterables.getOnlyElement(actualRecoveredDirtyJobResults), + is(matchingDirtyJobResult))); + } + + @Test + public void testRecoveryWithoutJobGraphButDirtyJobResult() throws Exception { + final JobResult dirtyJobResult = + TestingJobResultStore.createSuccessfulJobResult(new JobID()); + + testJobRecovery( + Collections.emptyList(), + Collections.singleton(dirtyJobResult), + actualRecoveredJobGraphs -> assertThat(actualRecoveredJobGraphs, empty()), + actualRecoveredDirtyJobResults -> + assertThat( + Iterables.getOnlyElement(actualRecoveredDirtyJobResults), + is(dirtyJobResult))); + } + + private void testJobRecovery( + Collection jobGraphsToRecover, + Set dirtyJobResults, + Consumer> recoveredJobGraphAssertion, + Consumer> recoveredDirtyJobResultAssertion) + throws Exception { jobGraphStore = - TestingJobGraphStore.newBuilder() - .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) + TestingJobGraphStore.newBuilder().setInitialJobGraphs(jobGraphsToRecover).build(); + + jobResultStore = + TestingJobResultStore.builder() + .withGetDirtyResultsSupplier(() -> dirtyJobResults) .build(); final CompletableFuture> recoveredJobGraphsFuture = new CompletableFuture<>(); + final CompletableFuture> recoveredDirtyJobResultsFuture = + new CompletableFuture<>(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (fencingToken, recoveredJobGraphs, jobGraphStore) -> { - recoveredJobGraphsFuture.complete(recoveredJobGraphs); - return TestingDispatcherGatewayService.newBuilder().build(); - }) - .build(); + (ignoredDispatcherId, + recoveredJobs, + recoveredDirtyJobResults, + ignoredJobGraphWriter, + ignoredJobResultStore) -> { + recoveredJobGraphsFuture.complete(recoveredJobs); + recoveredDirtyJobResultsFuture.complete(recoveredDirtyJobResults); + return TestingDispatcherGatewayService.newBuilder().build(); + }; try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); - assertThat( - dispatcherLeaderProcess.getState(), - is(SessionDispatcherLeaderProcess.State.RUNNING)); - - final Collection recoveredJobGraphs = recoveredJobGraphsFuture.get(); - assertThat(recoveredJobGraphs, containsInAnyOrder(JOB_GRAPH)); + recoveredJobGraphAssertion.accept(recoveredJobGraphsFuture.get()); + recoveredDirtyJobResultAssertion.accept(recoveredDirtyJobResultsFuture.get()); } } @@ -158,15 +255,12 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { final CompletableFuture dispatcherServiceTerminationFuture = new CompletableFuture<>(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder() - .setTerminationFuture( - dispatcherServiceTerminationFuture) - .withManualTerminationFutureCompletion() - .build()) - .build(); + createFactoryBasedOnGenericSupplier( + () -> + TestingDispatcherGatewayService.newBuilder() + .setTerminationFuture(dispatcherServiceTerminationFuture) + .withManualTerminationFutureCompletion() + .build()); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -194,13 +288,12 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { public void unexpectedDispatcherServiceTerminationWhileRunning_callsFatalErrorHandler() { final CompletableFuture terminationFuture = new CompletableFuture<>(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder() - .setTerminationFuture(terminationFuture) - .build()) - .build(); + createFactoryBasedOnGenericSupplier( + () -> + TestingDispatcherGatewayService.newBuilder() + .setTerminationFuture(terminationFuture) + .build()); + final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess(); dispatcherLeaderProcess.start(); @@ -219,14 +312,12 @@ public void unexpectedDispatcherServiceTerminationWhileRunning_callsFatalErrorHa unexpectedDispatcherServiceTerminationWhileNotRunning_doesNotCallFatalErrorHandler() { final CompletableFuture terminationFuture = new CompletableFuture<>(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder() - .setTerminationFuture(terminationFuture) - .withManualTerminationFutureCompletion() - .build()) - .build(); + createFactoryBasedOnGenericSupplier( + () -> + TestingDispatcherGatewayService.newBuilder() + .setTerminationFuture(terminationFuture) + .withManualTerminationFutureCompletion() + .build()); final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess(); dispatcherLeaderProcess.start(); @@ -248,16 +339,17 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta TestingDispatcherGateway.newBuilder().setAddress(dispatcherAddress).build(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - TriFunctionWithException.unchecked( - (ignoredA, ignoredB, ignoredC) -> { - createDispatcherServiceLatch.await(); - return TestingDispatcherGatewayService.newBuilder() - .setDispatcherGateway(dispatcherGateway) - .build(); - })) - .build(); + createFactoryBasedOnGenericSupplier( + () -> { + try { + createDispatcherServiceLatch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return TestingDispatcherGatewayService.newBuilder() + .setDispatcherGateway(dispatcherGateway) + .build(); + }); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -291,13 +383,11 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr .build(); this.dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> { - createDispatcherServiceLatch.trigger(); - return TestingDispatcherGatewayService.newBuilder().build(); - }) - .build(); + createFactoryBasedOnGenericSupplier( + () -> { + createDispatcherServiceLatch.trigger(); + return TestingDispatcherGatewayService.newBuilder().build(); + }); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -335,11 +425,7 @@ public void onRemovedJobGraph_terminatesRunningJob() throws Exception { .build(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (dispatcherId, jobGraphs, jobGraphWriter) -> - testingDispatcherService) - .build(); + createFactoryBasedOnGenericSupplier(() -> testingDispatcherService); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -367,11 +453,7 @@ public void onRemovedJobGraph_failingRemovalCall_failsFatally() throws Exception .build(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (dispatcherId, jobGraphs, jobGraphWriter) -> - testingDispatcherService) - .build(); + createFactoryBasedOnGenericSupplier(() -> testingDispatcherService); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -405,7 +487,12 @@ public void onAddedJobGraph_submitsRecoveredJob() throws Exception { }) .build(); - dispatcherServiceFactory = createDispatcherServiceFactoryFor(testingDispatcherGateway); + dispatcherServiceFactory = + createFactoryBasedOnGenericSupplier( + () -> + TestingDispatcherGatewayService.newBuilder() + .setDispatcherGateway(testingDispatcherGateway) + .build()); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -586,16 +673,14 @@ private void runOnAddedJobGraphTest( .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) .build(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (dispatcherId, jobGraphs, jobGraphWriter) -> { - assertThat(jobGraphs, containsInAnyOrder(JOB_GRAPH)); - - return TestingDispatcherGatewayService.newBuilder() - .setDispatcherGateway(dispatcherGateway) - .build(); - }) - .build(); + createFactoryBasedOnJobGraphs( + jobGraphs -> { + assertThat(jobGraphs, containsInAnyOrder(JOB_GRAPH)); + + return TestingDispatcherGatewayService.newBuilder() + .setDispatcherGateway(dispatcherGateway) + .build(); + }); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -609,6 +694,29 @@ private void runOnAddedJobGraphTest( } } + private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory + createFactoryBasedOnJobGraphs( + Function< + Collection, + AbstractDispatcherLeaderProcess.DispatcherGatewayService> + createFunction) { + return (ignoredDispatcherId, + recoveredJobs, + ignoredRecoveredDirtyJobResults, + ignoredJobGraphWriter, + ignoredJobResultStore) -> createFunction.apply(recoveredJobs); + } + + private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory + createFactoryBasedOnGenericSupplier( + Supplier supplier) { + return (ignoredDispatcherId, + ignoredRecoveredJobs, + ignoredRecoveredDirtyJobResults, + ignoredJobGraphWriter, + ignoredJobResultStore) -> supplier.get(); + } + private void verifyOnAddedJobGraphResultDidNotFail(TestingFatalErrorHandler fatalErrorHandler) throws Exception { try { @@ -619,22 +727,12 @@ private void verifyOnAddedJobGraphResultDidNotFail(TestingFatalErrorHandler fata } } - private TestingDispatcherServiceFactory createDispatcherServiceFactoryFor( - TestingDispatcherGateway testingDispatcherGateway) { - return TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder() - .setDispatcherGateway(testingDispatcherGateway) - .build()) - .build(); - } - private SessionDispatcherLeaderProcess createDispatcherLeaderProcess() { return SessionDispatcherLeaderProcess.create( leaderSessionId, dispatcherServiceFactory, jobGraphStore, + jobResultStore, ioExecutor, fatalErrorHandler); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java deleted file mode 100644 index 5efacdafeab5a..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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.flink.runtime.dispatcher.runner; - -import org.apache.flink.runtime.dispatcher.DispatcherId; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphWriter; -import org.apache.flink.util.function.TriFunction; - -import java.util.Collection; - -class TestingDispatcherServiceFactory - implements AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory { - private final TriFunction< - DispatcherId, - Collection, - JobGraphWriter, - AbstractDispatcherLeaderProcess.DispatcherGatewayService> - createFunction; - - private TestingDispatcherServiceFactory( - TriFunction< - DispatcherId, - Collection, - JobGraphWriter, - AbstractDispatcherLeaderProcess.DispatcherGatewayService> - createFunction) { - this.createFunction = createFunction; - } - - @Override - public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( - DispatcherId fencingToken, - Collection recoveredJobs, - JobGraphWriter jobGraphWriter) { - return createFunction.apply(fencingToken, recoveredJobs, jobGraphWriter); - } - - public static Builder newBuilder() { - return new Builder(); - } - - public static class Builder { - private TriFunction< - DispatcherId, - Collection, - JobGraphWriter, - AbstractDispatcherLeaderProcess.DispatcherGatewayService> - createFunction = - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder().build(); - - private Builder() {} - - Builder setCreateFunction( - TriFunction< - DispatcherId, - Collection, - JobGraphWriter, - AbstractDispatcherLeaderProcess.DispatcherGatewayService> - createFunction) { - this.createFunction = createFunction; - return this; - } - - public TestingDispatcherServiceFactory build() { - return new TestingDispatcherServiceFactory(createFunction); - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index 484c41e60f3cc..ca45d4e90aa1c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -35,8 +35,10 @@ import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -178,7 +180,17 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { createDispatcherRunner( rpcService, dispatcherLeaderElectionService, - () -> createZooKeeperJobGraphStore(client), + new JobPersistenceComponentFactory() { + @Override + public JobGraphStore createJobGraphStore() { + return createZooKeeperJobGraphStore(client); + } + + @Override + public JobResultStore createJobResultStore() { + return new EmbeddedJobResultStore(); + } + }, partialDispatcherServices, defaultDispatcherRunnerFactory)) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java new file mode 100644 index 0000000000000..39fa27bac143d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java @@ -0,0 +1,48 @@ +/* + * 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.flink.runtime.jobmanager; + +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.util.Preconditions; + +/** + * {@code TestingJobPersistenceComponentFactory} implements {@link JobPersistenceComponentFactory} + * for a given {@link JobGraphStore} and {@link JobResultStore}. + */ +public class TestingJobPersistenceComponentFactory implements JobPersistenceComponentFactory { + + private final JobGraphStore jobGraphStore; + private final JobResultStore jobResultStore; + + public TestingJobPersistenceComponentFactory( + JobGraphStore jobGraphStore, JobResultStore jobResultStore) { + this.jobGraphStore = Preconditions.checkNotNull(jobGraphStore); + this.jobResultStore = Preconditions.checkNotNull(jobResultStore); + } + + @Override + public JobGraphStore createJobGraphStore() { + return jobGraphStore; + } + + @Override + public JobResultStore createJobResultStore() { + return jobResultStore; + } +} From 8c54900797b2a6e15318a1f7308d27ad962e0488 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 25 Jan 2022 17:19:23 +0100 Subject: [PATCH 22/49] [hotfix] Migrates SessionDispatcherLeaderProcessTest to JUnit5/AssertJ --- .../SessionDispatcherLeaderProcessTest.java | 173 +++++++++--------- 1 file changed, 82 insertions(+), 91 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index 2dded1cfaef22..57f79e69f1d90 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.api.common.JobID; +import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; @@ -32,21 +33,19 @@ import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; -import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.function.ThrowingConsumer; -import com.google.common.collect.Iterables; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; -import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -61,17 +60,14 @@ import java.util.function.Function; import java.util.function.Supplier; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; -import static org.apache.flink.core.testutils.FlinkMatchers.willNotComplete; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.collection.IsEmptyCollection.empty; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.apache.flink.core.testutils.FlinkAssertions.STREAM_THROWABLE; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link SessionDispatcherLeaderProcess}. */ -public class SessionDispatcherLeaderProcessTest extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +public class SessionDispatcherLeaderProcessTest { private static final JobGraph JOB_GRAPH = JobGraphTestUtils.emptyJobGraph(); @@ -87,12 +83,12 @@ public class SessionDispatcherLeaderProcessTest extends TestLogger { private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherServiceFactory; - @BeforeClass + @BeforeAll public static void setupClass() { ioExecutor = Executors.newSingleThreadExecutor(); } - @Before + @BeforeEach public void setup() { fatalErrorHandler = new TestingFatalErrorHandler(); jobGraphStore = TestingJobGraphStore.newBuilder().build(); @@ -102,7 +98,7 @@ public void setup() { () -> TestingDispatcherGatewayService.newBuilder().build()); } - @After + @AfterEach public void teardown() throws Exception { if (fatalErrorHandler != null) { fatalErrorHandler.rethrowError(); @@ -110,7 +106,7 @@ public void teardown() throws Exception { } } - @AfterClass + @AfterAll public static void teardownClass() { if (ioExecutor != null) { ExecutorUtils.gracefulShutdown(5L, TimeUnit.SECONDS, ioExecutor); @@ -125,9 +121,8 @@ public void start_afterClose_doesNotHaveAnEffect() throws Exception { dispatcherLeaderProcess.close(); dispatcherLeaderProcess.start(); - assertThat( - dispatcherLeaderProcess.getState(), - is(SessionDispatcherLeaderProcess.State.STOPPED)); + assertThat(dispatcherLeaderProcess.getState()) + .isEqualTo(SessionDispatcherLeaderProcess.State.STOPPED); } @Test @@ -139,9 +134,8 @@ public void testStartTriggeringDispatcherServiceCreation() throws Exception { try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); - assertThat( - dispatcherLeaderProcess.getState(), - is(SessionDispatcherLeaderProcess.State.RUNNING)); + assertThat(dispatcherLeaderProcess.getState()) + .isEqualTo(SessionDispatcherLeaderProcess.State.RUNNING); } } @@ -151,10 +145,9 @@ public void testRecoveryWithJobGraphButNoDirtyJobResult() throws Exception { Collections.singleton(JOB_GRAPH), Collections.emptySet(), actualRecoveredJobGraphs -> - assertThat( - Iterables.getOnlyElement(actualRecoveredJobGraphs), is(JOB_GRAPH)), + assertThat(actualRecoveredJobGraphs).singleElement().isEqualTo(JOB_GRAPH), actualRecoveredDirtyJobResults -> - assertThat(actualRecoveredDirtyJobResults, empty())); + assertThat(actualRecoveredDirtyJobResults).isEmpty()); } @Test @@ -165,11 +158,11 @@ public void testRecoveryWithJobGraphAndMatchingDirtyJobResult() throws Exception testJobRecovery( Collections.singleton(JOB_GRAPH), Collections.singleton(matchingDirtyJobResult), - actualRecoveredJobGraphs -> assertThat(actualRecoveredJobGraphs, empty()), + actualRecoveredJobGraphs -> assertThat(actualRecoveredJobGraphs).isEmpty(), actualRecoveredDirtyJobResults -> - assertThat( - Iterables.getOnlyElement(actualRecoveredDirtyJobResults), - is(matchingDirtyJobResult))); + assertThat(actualRecoveredDirtyJobResults) + .singleElement() + .isEqualTo(matchingDirtyJobResult)); } @Test @@ -182,13 +175,13 @@ public void testRecoveryWithMultipleJobGraphsAndOneMatchingDirtyJobResult() thro Arrays.asList(otherJobGraph, JOB_GRAPH), Collections.singleton(matchingDirtyJobResult), actualRecoveredJobGraphs -> - assertThat( - Iterables.getOnlyElement(actualRecoveredJobGraphs), - is(otherJobGraph)), + assertThat(actualRecoveredJobGraphs) + .singleElement() + .isEqualTo(otherJobGraph), actualRecoveredDirtyJobResults -> - assertThat( - Iterables.getOnlyElement(actualRecoveredDirtyJobResults), - is(matchingDirtyJobResult))); + assertThat(actualRecoveredDirtyJobResults) + .singleElement() + .isEqualTo(matchingDirtyJobResult)); } @Test @@ -199,11 +192,11 @@ public void testRecoveryWithoutJobGraphButDirtyJobResult() throws Exception { testJobRecovery( Collections.emptyList(), Collections.singleton(dirtyJobResult), - actualRecoveredJobGraphs -> assertThat(actualRecoveredJobGraphs, empty()), + actualRecoveredJobGraphs -> assertThat(actualRecoveredJobGraphs).isEmpty(), actualRecoveredDirtyJobResults -> - assertThat( - Iterables.getOnlyElement(actualRecoveredDirtyJobResults), - is(dirtyJobResult))); + assertThat(actualRecoveredDirtyJobResults) + .singleElement() + .isEqualTo(dirtyJobResult)); } private void testJobRecovery( @@ -271,8 +264,8 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { final CompletableFuture terminationFuture = dispatcherLeaderProcess.closeAsync(); - assertThat(jobGraphStopFuture.isDone(), is(false)); - assertThat(terminationFuture.isDone(), is(false)); + assertThat(jobGraphStopFuture).isNotDone(); + assertThat(terminationFuture).isNotDone(); dispatcherServiceTerminationFuture.complete(null); @@ -302,7 +295,7 @@ public void unexpectedDispatcherServiceTerminationWhileRunning_callsFatalErrorHa terminationFuture.completeExceptionally(expectedFailure); final Throwable error = fatalErrorHandler.getErrorFuture().join(); - assertThat(error, containsCause(expectedFailure)); + assertThat(error).getRootCause().isEqualTo(expectedFailure); fatalErrorHandler.clearError(); } @@ -327,7 +320,8 @@ public void unexpectedDispatcherServiceTerminationWhileRunning_callsFatalErrorHa final FlinkException expectedFailure = new FlinkException("Expected test failure."); terminationFuture.completeExceptionally(expectedFailure); - assertThat(fatalErrorHandler.getErrorFuture(), willNotComplete(Duration.ofMillis(10))); + assertThatThrownBy(() -> fatalErrorHandler.getErrorFuture().get(10, TimeUnit.MILLISECONDS)) + .isInstanceOf(TimeoutException.class); } @Test @@ -358,11 +352,13 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta dispatcherLeaderProcess.start(); - assertThat(confirmLeaderSessionFuture.isDone(), is(false)); + assertThat(confirmLeaderSessionFuture).isNotDone(); createDispatcherServiceLatch.trigger(); - assertThat(confirmLeaderSessionFuture.get(), is(dispatcherAddress)); + assertThat(confirmLeaderSessionFuture) + .succeedsWithin(100, TimeUnit.MILLISECONDS) + .isEqualTo(dispatcherAddress); } } @@ -399,11 +395,10 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr completeJobRecoveryLatch.trigger(); - try { - createDispatcherServiceLatch.await(10L, TimeUnit.MILLISECONDS); - fail("No dispatcher service should be created after the process has been stopped."); - } catch (TimeoutException expected) { - } + assertThatThrownBy( + () -> createDispatcherServiceLatch.await(10L, TimeUnit.MILLISECONDS), + "No dispatcher service should be created after the process has been stopped.") + .isInstanceOf(TimeoutException.class); } } @@ -438,7 +433,7 @@ public void onRemovedJobGraph_terminatesRunningJob() throws Exception { jobGraphStore.removeJobGraph(JOB_GRAPH.getJobID()); dispatcherLeaderProcess.onRemovedJobGraph(JOB_GRAPH.getJobID()); - assertThat(terminateJobFuture.get(), is(JOB_GRAPH.getJobID())); + assertThat(terminateJobFuture).isCompletedWithValue(JOB_GRAPH.getJobID()); } } @@ -467,9 +462,7 @@ public void onRemovedJobGraph_failingRemovalCall_failsFatally() throws Exception final Throwable fatalError = fatalErrorHandler.getErrorFuture().join(); - assertTrue( - ExceptionUtils.findThrowable(fatalError, cause -> cause.equals(testException)) - .isPresent()); + assertThat(fatalError).hasCause(testException); fatalErrorHandler.clearError(); } @@ -506,7 +499,7 @@ public void onAddedJobGraph_submitsRecoveredJob() throws Exception { final JobGraph submittedJobGraph = submittedJobFuture.get(); - assertThat(submittedJobGraph.getJobID(), is(JOB_GRAPH.getJobID())); + assertThat(submittedJobGraph.getJobID()).isEqualTo(JOB_GRAPH.getJobID()); } } @@ -536,11 +529,10 @@ public void onAddedJobGraph_ifNotRunning_isBeingIgnored() throws Exception { dispatcherLeaderProcess.onAddedJobGraph(JOB_GRAPH.getJobID()); - try { - recoveredJobFuture.get(10L, TimeUnit.MILLISECONDS); - fail("onAddedJobGraph should be ignored if the leader process is not running."); - } catch (TimeoutException expected) { - } + assertThatThrownBy( + () -> recoveredJobFuture.get(10L, TimeUnit.MILLISECONDS), + "onAddedJobGraph should be ignored if the leader process is not running.") + .isInstanceOf(TimeoutException.class); } } @@ -565,15 +557,13 @@ public void onAddedJobGraph_failingRecovery_propagatesTheFailure() throws Except jobGraphStore.putJobGraph(JOB_GRAPH); dispatcherLeaderProcess.onAddedJobGraph(JOB_GRAPH.getJobID()); - final CompletableFuture errorFuture = fatalErrorHandler.getErrorFuture(); - final Throwable throwable = errorFuture.get(); - assertThat( - ExceptionUtils.findThrowable(throwable, expectedFailure::equals).isPresent(), - is(true)); + assertThat(fatalErrorHandler.getErrorFuture()) + .succeedsWithin(100, TimeUnit.MILLISECONDS) + .extracting(FlinkAssertions::chainOfCauses, STREAM_THROWABLE) + .contains(expectedFailure); - assertThat( - dispatcherLeaderProcess.getState(), - is(SessionDispatcherLeaderProcess.State.STOPPED)); + assertThat(dispatcherLeaderProcess.getState()) + .isEqualTo(SessionDispatcherLeaderProcess.State.STOPPED); fatalErrorHandler.clearError(); } @@ -614,11 +604,15 @@ private void runJobRecoveryFailureTest(FlinkException testException) throws Exce dispatcherLeaderProcess.start(); // we expect that a fatal error occurred - final Throwable error = fatalErrorHandler.getErrorFuture().get(); - assertThat( - ExceptionUtils.findThrowableWithMessage(error, testException.getMessage()) - .isPresent(), - is(true)); + assertThat(fatalErrorHandler.getErrorFuture()) + .succeedsWithin(100, TimeUnit.MILLISECONDS) + .satisfies( + error -> + assertThat(error) + .satisfies( + anyCauseMatches( + testException.getClass(), + testException.getMessage()))); fatalErrorHandler.clearError(); } @@ -642,9 +636,9 @@ private void verifyOnAddedJobGraphResultFailsFatally( TestingFatalErrorHandler fatalErrorHandler) { final Throwable actualCause = fatalErrorHandler.getErrorFuture().join(); - assertTrue( - ExceptionUtils.findThrowable(actualCause, JobSubmissionException.class) - .isPresent()); + assertThat(actualCause) + .extracting(FlinkAssertions::chainOfCauses, FlinkAssertions.STREAM_THROWABLE) + .hasAtLeastOneElementOfType(JobSubmissionException.class); fatalErrorHandler.clearError(); } @@ -675,7 +669,7 @@ private void runOnAddedJobGraphTest( dispatcherServiceFactory = createFactoryBasedOnJobGraphs( jobGraphs -> { - assertThat(jobGraphs, containsInAnyOrder(JOB_GRAPH)); + assertThat(jobGraphs).containsExactlyInAnyOrder(JOB_GRAPH); return TestingDispatcherGatewayService.newBuilder() .setDispatcherGateway(dispatcherGateway) @@ -717,14 +711,11 @@ private void runOnAddedJobGraphTest( ignoredJobResultStore) -> supplier.get(); } - private void verifyOnAddedJobGraphResultDidNotFail(TestingFatalErrorHandler fatalErrorHandler) - throws Exception { - try { - fatalErrorHandler.getErrorFuture().get(10L, TimeUnit.MILLISECONDS); - fail( - "Expected that duplicate job submissions due to false job recoveries are ignored."); - } catch (TimeoutException expected) { - } + private void verifyOnAddedJobGraphResultDidNotFail(TestingFatalErrorHandler fatalErrorHandler) { + assertThatThrownBy( + () -> fatalErrorHandler.getErrorFuture().get(10L, TimeUnit.MILLISECONDS), + "Expected that duplicate job submissions due to false job recoveries are ignored.") + .isInstanceOf(TimeoutException.class); } private SessionDispatcherLeaderProcess createDispatcherLeaderProcess() { From 68948aec1a989546e89371dc6e30695134e3fbfe Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 20 Jan 2022 10:50:38 +0100 Subject: [PATCH 23/49] [hotfix] Migrates ApplicationDispatcherBootstrapITCase to JUnit5/AssertJ --- .../ApplicationDispatcherBootstrapITCase.java | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java index 8caceb1db5fad..2b49f0c87512f 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java @@ -57,9 +57,10 @@ import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.UUID; @@ -68,11 +69,10 @@ import java.util.function.Supplier; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; /** Integration tests related to {@link ApplicationDispatcherBootstrap}. */ -public class ApplicationDispatcherBootstrapITCase extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +public class ApplicationDispatcherBootstrapITCase { private static final Duration TIMEOUT = Duration.ofMinutes(10); @@ -134,7 +134,9 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc cluster.requestJobResult(ApplicationDispatcherBootstrap.ZERO_JOB_ID); haServices.revokeDispatcherLeadership(); // make sure the leadership is revoked to avoid race conditions - assertEquals(ApplicationStatus.UNKNOWN, firstJobResult.get().getApplicationStatus()); + assertThat(firstJobResult.get()) + .extracting(JobResult::getApplicationStatus) + .isEqualTo(ApplicationStatus.UNKNOWN); haServices.grantDispatcherLeadership(); // job is suspended, wait until it's running @@ -148,10 +150,11 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc BlockingJob.unblock(blockId); // and wait for it to actually finish - final CompletableFuture secondJobResult = - cluster.requestJobResult(ApplicationDispatcherBootstrap.ZERO_JOB_ID); - assertTrue(secondJobResult.get().isSuccess()); - assertEquals(ApplicationStatus.SUCCEEDED, secondJobResult.get().getApplicationStatus()); + final JobResult secondJobResult = + cluster.requestJobResult(ApplicationDispatcherBootstrap.ZERO_JOB_ID).get(); + assertThat(secondJobResult.isSuccess()).isTrue(); + assertThat(secondJobResult.getApplicationStatus()) + .isEqualTo(ApplicationStatus.SUCCEEDED); // the cluster should shut down automatically once the application completes awaitClusterStopped(cluster, deadline); From 4d982301c7a698f0c52132814bfff3d86241a956 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Sat, 22 Jan 2022 16:59:59 +0100 Subject: [PATCH 24/49] [hotfix] Makes FileSystemBlobStore.(delete|deleteAll) comply to the BlobStore interface --- .../runtime/blob/FileSystemBlobStore.java | 8 +- .../runtime/blob/FileSystemBlobStoreTest.java | 229 ++++++++++++++++++ 2 files changed, 236 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java index d03cc745d0687..e8b53cc9170b4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java @@ -158,7 +158,13 @@ private boolean delete(String blobPath) { Path path = new Path(blobPath); - boolean result = fileSystem.delete(path, true); + boolean result = true; + if (fileSystem.exists(path)) { + result = fileSystem.delete(path, true); + } else { + LOG.debug( + "The given path {} is not present anymore. No deletion is required.", path); + } // send a call to delete the directory containing the file. This will // fail (and be ignored) when some files still exist. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/FileSystemBlobStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/FileSystemBlobStoreTest.java index 5b72270d63d78..479f58475fcd7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/FileSystemBlobStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/FileSystemBlobStoreTest.java @@ -22,26 +22,255 @@ import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.local.LocalDataOutputStream; import org.apache.flink.runtime.state.filesystem.TestFs; +import org.apache.flink.testutils.TestFileSystem; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.function.FunctionWithException; import org.apache.commons.io.FileUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +import java.io.BufferedWriter; import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileWriter; import java.io.IOException; +import java.io.InputStream; import java.nio.charset.StandardCharsets; +import java.nio.file.Files; import java.nio.file.Path; +import java.security.MessageDigest; import java.util.concurrent.atomic.AtomicReference; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link FileSystemBlobStore}. */ @ExtendWith(TestLoggerExtension.class) class FileSystemBlobStoreTest { + private FileSystemBlobStore testInstance; + private Path storagePath; + + @BeforeEach + public void createTestInstance(@TempDir Path storagePath) throws IOException { + this.testInstance = new FileSystemBlobStore(new TestFileSystem(), storagePath.toString()); + this.storagePath = storagePath; + } + + @AfterEach + public void finalizeTestInstance() throws IOException { + testInstance.close(); + } + + @Test + public void testSuccessfulPut() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("put"); + + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + assertThat(getBlobDirectoryPath()).isEmptyDirectory(); + + final boolean successfullyWritten = + testInstance.put(temporaryFile.toFile(), jobId, blobKey); + assertThat(successfullyWritten).isTrue(); + + assertThat(getPath(jobId)).isDirectory().exists(); + assertThat(getPath(jobId, blobKey)).isNotEmptyFile().hasSameTextualContentAs(temporaryFile); + } + + @Test + public void testMissingFilePut() throws IOException { + assertThatThrownBy( + () -> + testInstance.put( + new File("/not/existing/file"), + new JobID(), + new PermanentBlobKey())) + .isInstanceOf(FileNotFoundException.class); + } + + @Test + public void testSuccessfulGet() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("get"); + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, blobKey)).isTrue(); + + final Path targetFile = Files.createTempFile("filesystemblobstoretest-get-target-", ""); + assertThat(targetFile).isEmptyFile(); + final boolean successfullyGet = testInstance.get(jobId, blobKey, targetFile.toFile()); + assertThat(successfullyGet).isTrue(); + + assertThat(targetFile).hasSameTextualContentAs(temporaryFile); + } + + @Test + public void testGetWithWrongJobId() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("get"); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + + assertThat(testInstance.put(temporaryFile.toFile(), new JobID(), blobKey)).isTrue(); + + assertThatThrownBy( + () -> + testInstance.get( + new JobID(), + blobKey, + Files.createTempFile( + "filesystemblobstoretest-get-with-wrong-jobid-", + "") + .toFile())) + .isInstanceOf(FileNotFoundException.class); + } + + @Test + public void testGetWithWrongBlobKey() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("get"); + + final JobID jobId = new JobID(); + assertThat(testInstance.put(temporaryFile.toFile(), jobId, new PermanentBlobKey())) + .isTrue(); + + assertThatThrownBy( + () -> + testInstance.get( + jobId, + new PermanentBlobKey(), + Files.createTempFile( + "filesystemblobstoretest-get-with-wrong-blobkey-", + "") + .toFile())) + .isInstanceOf(FileNotFoundException.class); + } + + @Test + public void testSuccessfulDeleteOnlyBlob() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("delete"); + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, blobKey)).isTrue(); + + assertThat(getPath(jobId)).isDirectory().exists(); + assertThat(getPath(jobId, blobKey)).isNotEmptyFile(); + + final boolean successfullyDeleted = testInstance.delete(jobId, blobKey); + + assertThat(successfullyDeleted).isTrue(); + assertThat(getPath(jobId)).doesNotExist(); + } + + @Test + public void testSuccessfulDeleteBlob() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("delete"); + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + final BlobKey otherBlobKey = new PermanentBlobKey(); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, blobKey)).isTrue(); + // create another artifact to omit deleting the directory + assertThat(testInstance.put(temporaryFile.toFile(), jobId, otherBlobKey)).isTrue(); + + assertThat(getPath(jobId)).isDirectory().exists(); + assertThat(getPath(jobId, blobKey)).isNotEmptyFile(); + assertThat(getPath(jobId, otherBlobKey)).isNotEmptyFile(); + + final boolean successfullyDeleted = testInstance.delete(jobId, blobKey); + + assertThat(successfullyDeleted).isTrue(); + assertThat(getPath(jobId, otherBlobKey)).exists(); + } + + @Test + public void testDeleteWithNotExistingJobId() { + assertThat(testInstance.delete(new JobID(), new PermanentBlobKey())).isTrue(); + } + + @Test + public void testDeleteWithNotExistingBlobKey() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("delete"); + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, blobKey)).isTrue(); + assertThat(testInstance.delete(jobId, new PermanentBlobKey())).isTrue(); + assertThat(getPath(jobId, blobKey)).exists(); + } + + @Test + public void testDeleteAll() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("delete"); + final JobID jobId = new JobID(); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, new PermanentBlobKey())) + .isTrue(); + assertThat(testInstance.put(temporaryFile.toFile(), jobId, new PermanentBlobKey())) + .isTrue(); + + assertThat(getPath(jobId)).isDirectory().exists(); + assertThat(getPath(jobId).toFile().listFiles()).hasSize(2); + + assertThat(testInstance.deleteAll(jobId)).isTrue(); + assertThat(getPath(jobId)).doesNotExist(); + } + + @Test + public void testDeleteAllWithNotExistingJobId() { + final JobID jobId = new JobID(); + assertThat(testInstance.deleteAll(jobId)).isTrue(); + assertThat(getPath(jobId)).doesNotExist(); + } + + private Path createTemporaryFileWithContent(String operationLabel) throws IOException { + final String actualContent = + String.format("Content for testing the %s operation", operationLabel); + final Path temporaryFile = + Files.createTempFile( + String.format("filesystemblobstoretest-%s-", operationLabel), ""); + try (BufferedWriter writer = + new BufferedWriter(new FileWriter(temporaryFile.toAbsolutePath().toString()))) { + writer.write(actualContent); + } + + return temporaryFile; + } + + private Path getBlobDirectoryPath() { + return storagePath.resolve(FileSystemBlobStore.BLOB_PATH_NAME); + } + + private Path getPath(JobID jobId) { + return getBlobDirectoryPath().resolve(String.format("job_%s", jobId)); + } + + private Path getPath(JobID jobId, BlobKey blobKey) { + return getPath(jobId).resolve(String.format("blob_%s", blobKey)); + } + + private BlobKey createPermanentBlobKeyFromFile(Path path) throws IOException { + Preconditions.checkArgument(!Files.isDirectory(path)); + Preconditions.checkArgument(Files.exists(path)); + + MessageDigest md = BlobUtils.createMessageDigest(); + try (InputStream is = new FileInputStream(path.toFile())) { + final byte[] buf = new byte[1024]; + int bytesRead = is.read(buf); + while (bytesRead >= 0) { + md.update(buf, 0, bytesRead); + bytesRead = is.read(buf); + } + + return BlobKey.createKey(BlobKey.BlobType.PERMANENT_BLOB, md.digest()); + } + } + @Test public void fileSystemBlobStoreCallsSyncOnPut(@TempDir Path storageDirectory) throws IOException { From 2514d979cffb3ec1f1548a489beecf5b97463203 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 25 Nov 2021 12:54:06 +0100 Subject: [PATCH 25/49] [hotfix] Removes unused classloader parameter from CheckpointRecoveryFactory.createRecoveredCompletedCheckpointStore --- .../KubernetesCheckpointRecoveryFactory.java | 1 - .../runtime/checkpoint/CheckpointRecoveryFactory.java | 2 -- .../checkpoint/PerJobCheckpointRecoveryFactory.java | 1 - .../StandaloneCheckpointRecoveryFactory.java | 1 - .../checkpoint/ZooKeeperCheckpointRecoveryFactory.java | 1 - .../flink/runtime/scheduler/DefaultScheduler.java | 1 - .../apache/flink/runtime/scheduler/SchedulerBase.java | 2 -- .../apache/flink/runtime/scheduler/SchedulerUtils.java | 10 +--------- .../runtime/scheduler/adaptive/AdaptiveScheduler.java | 7 +------ .../checkpoint/PerJobCheckpointRecoveryTest.java | 6 ------ .../checkpoint/TestingCheckpointRecoveryFactory.java | 1 - .../flink/runtime/scheduler/SchedulerUtilsTest.java | 3 --- 12 files changed, 2 insertions(+), 34 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java index 2c318645a46ea..ea78ecbbd2829 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java @@ -80,7 +80,6 @@ private KubernetesCheckpointRecoveryFactory( public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobID, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java index 81d70097471a4..ab0b5ef8506ed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java @@ -34,7 +34,6 @@ public interface CheckpointRecoveryFactory { * * @param jobId Job ID to recover checkpoints for * @param maxNumberOfCheckpointsToRetain Maximum number of checkpoints to retain - * @param userClassLoader User code class loader of the job * @param sharedStateRegistryFactory Simple factory to produce {@link SharedStateRegistry} * objects. * @param ioExecutor Executor used to run (async) deletes. @@ -43,7 +42,6 @@ public interface CheckpointRecoveryFactory { CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) throws Exception; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java index 83ce07732e108..bc18c453969d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java @@ -74,7 +74,6 @@ public PerJobCheckpointRecoveryFactory( public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) { return store.compute( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java index a77c6fb8cb372..95f9da72406ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java @@ -31,7 +31,6 @@ public class StandaloneCheckpointRecoveryFactory implements CheckpointRecoveryFa public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java index db4cfeceeb24f..354259bd81953 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java @@ -50,7 +50,6 @@ public ZooKeeperCheckpointRecoveryFactory( public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 1a2fa36a38cfc..d4eacb1dad671 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -142,7 +142,6 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio jobGraph, ioExecutor, jobMasterConfiguration, - userCodeLoader, checkpointsCleaner, checkpointRecoveryFactory, jobManagerJobMetricGroup, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 70d4ab2ac38de..c975085d2d6c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -167,7 +167,6 @@ public SchedulerBase( final JobGraph jobGraph, final Executor ioExecutor, final Configuration jobMasterConfiguration, - final ClassLoader userCodeLoader, final CheckpointsCleaner checkpointsCleaner, final CheckpointRecoveryFactory checkpointRecoveryFactory, final JobManagerJobMetricGroup jobManagerJobMetricGroup, @@ -192,7 +191,6 @@ public SchedulerBase( SchedulerUtils.createCompletedCheckpointStoreIfCheckpointingIsEnabled( jobGraph, jobMasterConfiguration, - userCodeLoader, checkNotNull(checkpointRecoveryFactory), ioExecutor, log); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java index 5f6130492262b..ab807ecba375e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java @@ -47,7 +47,6 @@ private SchedulerUtils() { public static CompletedCheckpointStore createCompletedCheckpointStoreIfCheckpointingIsEnabled( JobGraph jobGraph, Configuration configuration, - ClassLoader userCodeLoader, CheckpointRecoveryFactory checkpointRecoveryFactory, Executor ioExecutor, Logger log) @@ -56,12 +55,7 @@ public static CompletedCheckpointStore createCompletedCheckpointStoreIfCheckpoin if (DefaultExecutionGraphBuilder.isCheckpointingEnabled(jobGraph)) { try { return createCompletedCheckpointStore( - configuration, - userCodeLoader, - checkpointRecoveryFactory, - ioExecutor, - log, - jobId); + configuration, checkpointRecoveryFactory, ioExecutor, log, jobId); } catch (Exception e) { throw new JobExecutionException( jobId, @@ -76,7 +70,6 @@ public static CompletedCheckpointStore createCompletedCheckpointStoreIfCheckpoin @VisibleForTesting static CompletedCheckpointStore createCompletedCheckpointStore( Configuration jobManagerConfig, - ClassLoader classLoader, CheckpointRecoveryFactory recoveryFactory, Executor ioExecutor, Logger log, @@ -101,7 +94,6 @@ static CompletedCheckpointStore createCompletedCheckpointStore( return recoveryFactory.createRecoveredCompletedCheckpointStore( jobId, maxNumberOfCheckpointsToRetain, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, ioExecutor); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java index db6c2a17892ba..d289f7742827c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java @@ -245,12 +245,7 @@ public AdaptiveScheduler( this.checkpointsCleaner = checkpointsCleaner; this.completedCheckpointStore = SchedulerUtils.createCompletedCheckpointStoreIfCheckpointingIsEnabled( - jobGraph, - configuration, - userCodeClassLoader, - checkpointRecoveryFactory, - ioExecutor, - LOG); + jobGraph, configuration, checkpointRecoveryFactory, ioExecutor, LOG); this.checkpointIdCounter = SchedulerUtils.createCheckpointIDCounterIfCheckpointingIsEnabled( jobGraph, checkpointRecoveryFactory); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java index acd9d809f4d28..2fe17366351c9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java @@ -40,15 +40,12 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { final CheckpointRecoveryFactory factory = PerJobCheckpointRecoveryFactory.withoutCheckpointStoreRecovery( maxCheckpoints -> store); - final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - final JobID firstJobId = new JobID(); assertSame( store, factory.createRecoveredCompletedCheckpointStore( firstJobId, 1, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, Executors.directExecutor())); assertThrows( @@ -57,7 +54,6 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { factory.createRecoveredCompletedCheckpointStore( firstJobId, 1, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, Executors.directExecutor())); @@ -67,7 +63,6 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { factory.createRecoveredCompletedCheckpointStore( secondJobId, 1, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, Executors.directExecutor())); assertThrows( @@ -76,7 +71,6 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { factory.createRecoveredCompletedCheckpointStore( secondJobId, 1, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, Executors.directExecutor())); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java index c1e851478b130..e164543d0ab81 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java @@ -38,7 +38,6 @@ public TestingCheckpointRecoveryFactory( public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) { return store; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java index 8f5bffcf594ad..641a9181bc8ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java @@ -75,7 +75,6 @@ public void testSettingMaxNumberOfCheckpointsToRetain() throws Exception { final CompletedCheckpointStore completedCheckpointStore = SchedulerUtils.createCompletedCheckpointStore( jobManagerConfig, - getClass().getClassLoader(), new StandaloneCheckpointRecoveryFactory(), Executors.directExecutor(), log, @@ -102,7 +101,6 @@ public void testSharedStateRegistration() throws Exception { CompletedCheckpointStore checkpointStore = SchedulerUtils.createCompletedCheckpointStore( new Configuration(), - getClass().getClassLoader(), recoveryFactory, Executors.directExecutor(), log, @@ -123,7 +121,6 @@ private CheckpointRecoveryFactory buildRecoveryFactory(CompletedCheckpoint check public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) { List checkpoints = singletonList(checkpoint); From f4221355158444b81386d3e9a2d9f8b7d20b0f3a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 8 Dec 2021 12:30:41 +0100 Subject: [PATCH 26/49] [hotfix] Refactors nested if statements --- .../flink/runtime/dispatcher/Dispatcher.java | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 3d5e2eebdbafb..0622c666c0f08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -496,17 +496,13 @@ private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Excep private CleanupJobState handleJobManagerRunnerResult( JobManagerRunnerResult jobManagerRunnerResult, ExecutionType executionType) { - if (jobManagerRunnerResult.isInitializationFailure()) { - if (executionType == ExecutionType.RECOVERY) { - return jobManagerRunnerFailed( - jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), - jobManagerRunnerResult.getInitializationFailure()); - } else { - return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); - } - } else { - return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); + if (jobManagerRunnerResult.isInitializationFailure() + && executionType == ExecutionType.RECOVERY) { + return jobManagerRunnerFailed( + jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), + jobManagerRunnerResult.getInitializationFailure()); } + return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); } enum CleanupJobState { From 99c371c27f29ec312ea1a33abca77e741074097a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 12:26:39 +0100 Subject: [PATCH 27/49] [FLINK-25432] Adds generic interfaces for cleaning up Job-related data --- .../cleanup/GloballyCleanableResource.java | 49 +++++++++++ .../cleanup/LocallyCleanableResource.java | 49 +++++++++++ .../GloballyCleanableResourceTest.java | 49 +++++++++++ .../cleanup/LocallyCleanableResourceTest.java | 49 +++++++++++ .../cleanup/TestingCleanableResource.java | 83 +++++++++++++++++++ 5 files changed, 279 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResource.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResource.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResourceTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResourceTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanableResource.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResource.java new file mode 100644 index 0000000000000..57ba065f84ace --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResource.java @@ -0,0 +1,49 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * {@code GloballyCleanableResource} is supposed to be used by any class that provides artifacts for + * a given job that can be cleaned up globally. + * + * @see LocallyCleanableResource + */ +@FunctionalInterface +public interface GloballyCleanableResource { + + void globalCleanup(JobID jobId) throws Throwable; + + default CompletableFuture globalCleanupAsync(JobID jobId, Executor cleanupExecutor) { + return CompletableFuture.runAsync( + () -> { + try { + globalCleanup(jobId); + } catch (Throwable t) { + throw new CompletionException("Asynchronous global cleanup failed", t); + } + }, + cleanupExecutor); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResource.java new file mode 100644 index 0000000000000..fd2b98caed62b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResource.java @@ -0,0 +1,49 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * {@code LocallyCleanableResource} is supposed to be used by any class that provides artifacts for + * a given job that can be cleaned up locally. + * + * @see GloballyCleanableResource + */ +@FunctionalInterface +public interface LocallyCleanableResource { + + void localCleanup(JobID jobId) throws Throwable; + + default CompletableFuture localCleanupAsync(JobID jobId, Executor cleanupExecutor) { + return CompletableFuture.runAsync( + () -> { + try { + localCleanup(jobId); + } catch (Throwable t) { + throw new CompletionException("Asynchronous local cleanup failed", t); + } + }, + cleanupExecutor); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResourceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResourceTest.java new file mode 100644 index 0000000000000..00fe5641c4500 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResourceTest.java @@ -0,0 +1,49 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CompletableFuture; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +/** + * {@code GloballyCleanableResourceTest} tests the default implementation of {@link + * GloballyCleanableResource}. + */ +public class GloballyCleanableResourceTest { + + @Test + public void testGlobalCleanupAsync() { + final CompletableFuture globalCleanupTriggered = new CompletableFuture<>(); + final GloballyCleanableResource testInstance = + TestingCleanableResource.builder() + .withGlobalCleanupConsumer(globalCleanupTriggered::complete) + .build(); + + final JobID jobId = new JobID(); + testInstance.globalCleanupAsync(jobId, Executors.directExecutor()); + + assertThat(globalCleanupTriggered).isCompletedWithValue(jobId); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResourceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResourceTest.java new file mode 100644 index 0000000000000..8db2841fccfdc --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResourceTest.java @@ -0,0 +1,49 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CompletableFuture; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +/** + * {@code LocallyCleanableResourceTest} tests the default implementation of {@link + * LocallyCleanableResource}. + */ +public class LocallyCleanableResourceTest { + + @Test + public void testLocalCleanupAsync() { + final CompletableFuture localCleanupTriggered = new CompletableFuture<>(); + final LocallyCleanableResource testInstance = + TestingCleanableResource.builder() + .withLocalCleanupConsumer(localCleanupTriggered::complete) + .build(); + + final JobID jobId = new JobID(); + testInstance.localCleanupAsync(jobId, Executors.directExecutor()); + + assertThat(localCleanupTriggered).isCompletedWithValue(jobId); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanableResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanableResource.java new file mode 100644 index 0000000000000..0380e11c4208d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanableResource.java @@ -0,0 +1,83 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.function.ThrowingConsumer; + +/** + * {@code TestingCleanableResource} provides methods for testing the callbacks of {@link + * LocallyCleanableResource}. + */ +class TestingCleanableResource implements LocallyCleanableResource, GloballyCleanableResource { + + private final ThrowingConsumer localCleanupConsumer; + private final ThrowingConsumer globalCleanupConsumer; + + private TestingCleanableResource( + ThrowingConsumer localCleanupConsumer, + ThrowingConsumer globalCleanupConsumer) { + this.localCleanupConsumer = localCleanupConsumer; + this.globalCleanupConsumer = globalCleanupConsumer; + } + + @Override + public void localCleanup(JobID jobId) throws Exception { + localCleanupConsumer.accept(jobId); + } + + @Override + public void globalCleanup(JobID jobId) throws Exception { + globalCleanupConsumer.accept(jobId); + } + + public static TestingCleanableResource.Builder builder() { + return new Builder(); + } + + static class Builder { + + private ThrowingConsumer localCleanupConsumer = + jobId -> { + throw new UnsupportedOperationException("Local cleanup is not supported."); + }; + private ThrowingConsumer globalCleanupConsumer = + jobId -> { + throw new UnsupportedOperationException("Global cleanup is not supported."); + }; + + private Builder() {} + + public Builder withLocalCleanupConsumer( + ThrowingConsumer localCleanupConsumer) { + this.localCleanupConsumer = localCleanupConsumer; + return this; + } + + public Builder withGlobalCleanupConsumer( + ThrowingConsumer globalCleanupConsumer) { + this.globalCleanupConsumer = globalCleanupConsumer; + return this; + } + + public TestingCleanableResource build() { + return new TestingCleanableResource(localCleanupConsumer, globalCleanupConsumer); + } + } +} From 57d31773e16e44d60923e18f004f8e9dcca8055f Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 12:31:31 +0100 Subject: [PATCH 28/49] [FLINK-25432] Refactors JobGraphWriter interface to implement LocallyCleanableResource and GloballyCleanableResource --- .../flink/runtime/dispatcher/Dispatcher.java | 4 +- .../dispatcher/SingleJobJobGraphStore.java | 10 ----- .../jobmanager/DefaultJobGraphStore.java | 4 +- .../runtime/jobmanager/JobGraphWriter.java | 13 +++--- .../jobmanager/StandaloneJobGraphStore.java | 10 ----- .../jobmanager/ThrowingJobGraphWriter.java | 7 ---- .../dispatcher/DispatcherFailoverITCase.java | 2 +- .../DispatcherResourceCleanupTest.java | 2 +- .../runtime/dispatcher/DispatcherTest.java | 12 +++--- .../dispatcher/NoOpJobGraphWriter.java | 7 ---- .../SessionDispatcherLeaderProcessTest.java | 2 +- .../jobmanager/DefaultJobGraphStoreTest.java | 6 +-- .../StandaloneJobGraphStoreTest.java | 4 +- .../ZooKeeperJobGraphsStoreITCase.java | 10 ++--- .../testutils/TestingJobGraphStore.java | 41 +++++++++---------- 15 files changed, 51 insertions(+), 83 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 0622c666c0f08..0080ee1281955 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -868,7 +868,7 @@ private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJo private boolean cleanUpJobGraph(JobID jobId, boolean cleanupHA) { if (cleanupHA) { try { - jobGraphWriter.removeJobGraph(jobId); + jobGraphWriter.globalCleanup(jobId); return true; } catch (Exception e) { log.warn( @@ -879,7 +879,7 @@ private boolean cleanUpJobGraph(JobID jobId, boolean cleanupHA) { } } try { - jobGraphWriter.releaseJobGraph(jobId); + jobGraphWriter.localCleanup(jobId); } catch (Exception e) { log.warn("Could not properly release job {} from submitted job graph store.", jobId, e); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobJobGraphStore.java index f2c450abf868d..b063b4bcad505 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobJobGraphStore.java @@ -64,16 +64,6 @@ public void putJobGraph(JobGraph jobGraph) throws Exception { } } - @Override - public void removeJobGraph(JobID jobId) { - // ignore - } - - @Override - public void releaseJobGraph(JobID jobId) { - // ignore - } - @Override public Collection getJobIds() { return Collections.singleton(jobGraph.getJobID()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java index 832f9c92baf97..5416bdae4261e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java @@ -238,7 +238,7 @@ public void putJobGraph(JobGraph jobGraph) throws Exception { } @Override - public void removeJobGraph(JobID jobId) throws Exception { + public void globalCleanup(JobID jobId) throws Exception { checkNotNull(jobId, "Job ID"); String name = jobGraphStoreUtil.jobIDToName(jobId); @@ -262,7 +262,7 @@ public void removeJobGraph(JobID jobId) throws Exception { } @Override - public void releaseJobGraph(JobID jobId) throws Exception { + public void localCleanup(JobID jobId) throws Exception { checkNotNull(jobId, "Job ID"); LOG.debug("Releasing job graph {} from {}.", jobId, jobGraphStateHandleStore); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java index 23542a330b923..63d644a813c07 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java @@ -19,10 +19,12 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; import org.apache.flink.runtime.jobgraph.JobGraph; /** Allows to store and remove job graphs. */ -public interface JobGraphWriter { +public interface JobGraphWriter extends LocallyCleanableResource, GloballyCleanableResource { /** * Adds the {@link JobGraph} instance. * @@ -30,9 +32,6 @@ public interface JobGraphWriter { */ void putJobGraph(JobGraph jobGraph) throws Exception; - /** Removes the {@link JobGraph} with the given {@link JobID} if it exists. */ - void removeJobGraph(JobID jobId) throws Exception; - /** * Releases the locks on the specified {@link JobGraph}. * @@ -42,5 +41,9 @@ public interface JobGraphWriter { * @param jobId specifying the job to release the locks for * @throws Exception if the locks cannot be released */ - void releaseJobGraph(JobID jobId) throws Exception; + @Override + default void localCleanup(JobID jobId) throws Exception {} + + @Override + default void globalCleanup(JobID jobId) throws Exception {} } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStore.java index 656df2ffd3607..f7d8135136aa4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStore.java @@ -47,16 +47,6 @@ public void putJobGraph(JobGraph jobGraph) { // Nothing to do } - @Override - public void removeJobGraph(JobID jobId) { - // Nothing to do - } - - @Override - public void releaseJobGraph(JobID jobId) { - // nothing to do - } - @Override public Collection getJobIds() { return Collections.emptyList(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java index 224b7fbd224f5..8bf9aa0c45d2f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.jobmanager; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.JobGraph; /** {@link JobGraphWriter} implementation which does not allow to store {@link JobGraph}. */ @@ -29,10 +28,4 @@ public enum ThrowingJobGraphWriter implements JobGraphWriter { public void putJobGraph(JobGraph jobGraph) { throw new UnsupportedOperationException("Cannot store job graphs."); } - - @Override - public void removeJobGraph(JobID jobId) {} - - @Override - public void releaseJobGraph(JobID jobId) {} } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java index ef6be6139c163..e75051d7f75fb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java @@ -112,7 +112,7 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() final Error jobGraphRemovalError = new Error("Unable to remove job graph."); final TestingJobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() - .setRemoveJobGraphConsumer( + .setGlobalCleanupConsumer( graph -> { throw jobGraphRemovalError; }) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 5b84a63dcc47f..4992c76ddc9b2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -620,7 +620,7 @@ public Set getDirtyResults() throws IOException { public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception { jobGraphWriter = TestingJobGraphStore.newBuilder() - .setRemoveJobGraphConsumer( + .setGlobalCleanupConsumer( ignored -> { throw new Exception("Failed to Remove future"); }) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index a131d97f6e045..bec6936a66932 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -746,9 +746,9 @@ public void testFailingJobManagerRunnerCleanup() throws Exception { // Track cleanup - job-graph final TestingJobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() - .setReleaseJobGraphConsumer( + .setLocalCleanupConsumer( jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_RELEASE)) - .setRemoveJobGraphConsumer( + .setGlobalCleanupConsumer( jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_REMOVE)) .build(); jobGraphStore.start(null); @@ -904,8 +904,8 @@ public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception { final TestingJobGraphStore testingJobGraphStore = TestingJobGraphStore.newBuilder() - .setRemoveJobGraphConsumer(removeJobGraphFuture::complete) - .setReleaseJobGraphConsumer(releaseJobGraphFuture::complete) + .setGlobalCleanupConsumer(removeJobGraphFuture::complete) + .setLocalCleanupConsumer(releaseJobGraphFuture::complete) .build(); testingJobGraphStore.start(null); @@ -1142,9 +1142,9 @@ private void testJobDataAreCleanedUpInCorrectOrder(JobStatus jobStatus) throws E // Track cleanup - job-graph final TestingJobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() - .setReleaseJobGraphConsumer( + .setLocalCleanupConsumer( jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_RELEASE)) - .setRemoveJobGraphConsumer( + .setGlobalCleanupConsumer( jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_REMOVE)) .build(); jobGraphStore.start(null); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java index 7a2d156e1945c..bc17d94c145be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; @@ -28,10 +27,4 @@ public enum NoOpJobGraphWriter implements JobGraphWriter { @Override public void putJobGraph(JobGraph jobGraph) throws Exception {} - - @Override - public void removeJobGraph(JobID jobId) throws Exception {} - - @Override - public void releaseJobGraph(JobID jobId) throws Exception {} } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index 57f79e69f1d90..7dd2446ce9f0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -430,7 +430,7 @@ public void onRemovedJobGraph_terminatesRunningJob() throws Exception { dispatcherLeaderProcess.getDispatcherGateway().get(); // now remove the Job from the JobGraphStore and notify the dispatcher service - jobGraphStore.removeJobGraph(JOB_GRAPH.getJobID()); + jobGraphStore.globalCleanup(JOB_GRAPH.getJobID()); dispatcherLeaderProcess.onRemovedJobGraph(JOB_GRAPH.getJobID()); assertThat(terminateJobFuture).isCompletedWithValue(JOB_GRAPH.getJobID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java index 568e937716ae8..ccabb3cf69390 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java @@ -200,7 +200,7 @@ public void testRemoveJobGraph() throws Exception { final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); jobGraphStore.putJobGraph(testingJobGraph); - jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + jobGraphStore.globalCleanup(testingJobGraph.getJobID()); final JobID actual = removeFuture.get(timeout, TimeUnit.MILLISECONDS); assertThat(actual, is(testingJobGraph.getJobID())); } @@ -213,7 +213,7 @@ public void testRemoveJobGraphWithNonExistName() throws Exception { .build(); final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); - jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + jobGraphStore.globalCleanup(testingJobGraph.getJobID()); try { removeFuture.get(timeout, TimeUnit.MILLISECONDS); @@ -346,7 +346,7 @@ public void testReleasingJobGraphShouldReleaseHandle() throws Exception { builder.setReleaseConsumer(releaseFuture::complete).build(); final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); jobGraphStore.putJobGraph(testingJobGraph); - jobGraphStore.releaseJobGraph(testingJobGraph.getJobID()); + jobGraphStore.localCleanup(testingJobGraph.getJobID()); final String actual = releaseFuture.get(); assertThat(actual, is(testingJobGraph.getJobID().toString())); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStoreTest.java index 9f89d9f5669f0..d6c85a0c617dc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStoreTest.java @@ -31,7 +31,7 @@ public class StandaloneJobGraphStoreTest { /** Tests that all operations work and don't change the state. */ @Test - public void testNoOps() { + public void testNoOps() throws Exception { StandaloneJobGraphStore jobGraphs = new StandaloneJobGraphStore(); JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); @@ -41,7 +41,7 @@ public void testNoOps() { jobGraphs.putJobGraph(jobGraph); assertEquals(0, jobGraphs.getJobIds().size()); - jobGraphs.removeJobGraph(jobGraph.getJobID()); + jobGraphs.globalCleanup(jobGraph.getJobID()); assertEquals(0, jobGraphs.getJobIds().size()); assertNull(jobGraphs.recoverJobGraph(new JobID())); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphsStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphsStoreITCase.java index ad5bcf81829d3..0773c404803a3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphsStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphsStoreITCase.java @@ -130,7 +130,7 @@ public void testPutAndRemoveJobGraph() throws Exception { verifyJobGraphs(jobGraph, jobGraphs.recoverJobGraph(jobId)); // Remove - jobGraphs.removeJobGraph(jobGraph.getJobID()); + jobGraphs.globalCleanup(jobGraph.getJobID()); // Empty state assertEquals(0, jobGraphs.getJobIds().size()); @@ -140,7 +140,7 @@ public void testPutAndRemoveJobGraph() throws Exception { verify(listener, never()).onRemovedJobGraph(any(JobID.class)); // Don't fail if called again - jobGraphs.removeJobGraph(jobGraph.getJobID()); + jobGraphs.globalCleanup(jobGraph.getJobID()); } finally { jobGraphs.stop(); } @@ -193,7 +193,7 @@ public void testRecoverJobGraphs() throws Exception { verifyJobGraphs(expected.get(jobGraph.getJobID()), jobGraph); - jobGraphs.removeJobGraph(jobGraph.getJobID()); + jobGraphs.globalCleanup(jobGraph.getJobID()); } // Empty state @@ -313,7 +313,7 @@ public void testJobGraphRemovalFailureAndLockRelease() throws Exception { assertThat(recoveredJobGraph, is(notNullValue())); try { - otherSubmittedJobGraphStore.removeJobGraph(recoveredJobGraph.getJobID()); + otherSubmittedJobGraphStore.globalCleanup(recoveredJobGraph.getJobID()); fail( "It should not be possible to remove the JobGraph since the first store still has a lock on it."); } catch (Exception ignored) { @@ -323,7 +323,7 @@ public void testJobGraphRemovalFailureAndLockRelease() throws Exception { submittedJobGraphStore.stop(); // now we should be able to delete the job graph - otherSubmittedJobGraphStore.removeJobGraph(recoveredJobGraph.getJobID()); + otherSubmittedJobGraphStore.globalCleanup(recoveredJobGraph.getJobID()); assertThat( otherSubmittedJobGraphStore.recoverJobGraph(recoveredJobGraph.getJobID()), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java index e9a797f0f47f2..9fa121a31d39f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java @@ -54,9 +54,9 @@ public class TestingJobGraphStore implements JobGraphStore { private final ThrowingConsumer putJobGraphConsumer; - private final ThrowingConsumer removeJobGraphConsumer; + private final ThrowingConsumer globalCleanupConsumer; - private final ThrowingConsumer releaseJobGraphConsumer; + private final ThrowingConsumer localCleanupConsumer; private boolean started; @@ -68,16 +68,16 @@ private TestingJobGraphStore( BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction, ThrowingConsumer putJobGraphConsumer, - ThrowingConsumer removeJobGraphConsumer, - ThrowingConsumer releaseJobGraphConsumer, + ThrowingConsumer globalCleanupConsumer, + ThrowingConsumer localCleanupConsumer, Collection initialJobGraphs) { this.startConsumer = startConsumer; this.stopRunnable = stopRunnable; this.jobIdsFunction = jobIdsFunction; this.recoverJobGraphFunction = recoverJobGraphFunction; this.putJobGraphConsumer = putJobGraphConsumer; - this.removeJobGraphConsumer = removeJobGraphConsumer; - this.releaseJobGraphConsumer = releaseJobGraphConsumer; + this.globalCleanupConsumer = globalCleanupConsumer; + this.localCleanupConsumer = localCleanupConsumer; for (JobGraph initialJobGraph : initialJobGraphs) { storedJobs.put(initialJobGraph.getJobID(), initialJobGraph); @@ -110,16 +110,16 @@ public synchronized void putJobGraph(JobGraph jobGraph) throws Exception { } @Override - public synchronized void removeJobGraph(JobID jobId) throws Exception { + public synchronized void globalCleanup(JobID jobId) throws Exception { verifyIsStarted(); - removeJobGraphConsumer.accept(jobId); + globalCleanupConsumer.accept(jobId); storedJobs.remove(jobId); } @Override - public synchronized void releaseJobGraph(JobID jobId) throws Exception { + public synchronized void localCleanup(JobID jobId) throws Exception { verifyIsStarted(); - releaseJobGraphConsumer.accept(jobId); + localCleanupConsumer.accept(jobId); } @Override @@ -156,10 +156,9 @@ public static class Builder { private ThrowingConsumer putJobGraphConsumer = ignored -> {}; - private ThrowingConsumer removeJobGraphConsumer = ignored -> {}; + private ThrowingConsumer globalCleanupConsumer = ignored -> {}; - private ThrowingConsumer releaseJobGraphConsumer = - ignored -> {}; + private ThrowingConsumer localCleanupConsumer = ignored -> {}; private Collection initialJobGraphs = Collections.emptyList(); @@ -198,15 +197,15 @@ public Builder setPutJobGraphConsumer( return this; } - public Builder setRemoveJobGraphConsumer( - ThrowingConsumer removeJobGraphConsumer) { - this.removeJobGraphConsumer = removeJobGraphConsumer; + public Builder setGlobalCleanupConsumer( + ThrowingConsumer globalCleanupConsumer) { + this.globalCleanupConsumer = globalCleanupConsumer; return this; } - public Builder setReleaseJobGraphConsumer( - ThrowingConsumer releaseJobGraphConsumer) { - this.releaseJobGraphConsumer = releaseJobGraphConsumer; + public Builder setLocalCleanupConsumer( + ThrowingConsumer localCleanupConsumer) { + this.localCleanupConsumer = localCleanupConsumer; return this; } @@ -228,8 +227,8 @@ public TestingJobGraphStore build() { jobIdsFunction, recoverJobGraphFunction, putJobGraphConsumer, - removeJobGraphConsumer, - releaseJobGraphConsumer, + globalCleanupConsumer, + localCleanupConsumer, initialJobGraphs); if (startJobGraphStore) { From 7a895e86b41ffcb6b5fd020afbba7cba21cb5a66 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 12:37:05 +0100 Subject: [PATCH 29/49] [FLINK-25432] Makes BlobServer implement LocallyCleanableResource and GloballyCleanableResource --- .../apache/flink/runtime/blob/BlobServer.java | 76 ++++++++++++------- .../flink/runtime/dispatcher/Dispatcher.java | 19 ++++- .../runtime/blob/BlobCacheCleanupTest.java | 12 +-- .../runtime/blob/BlobServerCleanupTest.java | 2 +- .../runtime/blob/BlobServerDeleteTest.java | 15 ++-- .../runtime/blob/BlobServerRecoveryTest.java | 6 +- .../DispatcherResourceCleanupTest.java | 72 ++++++++++++------ .../BlobLibraryCacheRecoveryITCase.java | 2 +- 8 files changed, 132 insertions(+), 72 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java index 944031eabdec0..0a5628f1de897 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java @@ -25,9 +25,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; import org.apache.flink.runtime.net.SSLUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Reference; import org.apache.flink.util.ShutdownHookUtil; @@ -74,7 +77,12 @@ * the directory structure to store the BLOBs or temporarily cache them. */ public class BlobServer extends Thread - implements BlobService, BlobWriter, PermanentBlobService, TransientBlobService { + implements BlobService, + BlobWriter, + PermanentBlobService, + TransientBlobService, + LocallyCleanableResource, + GloballyCleanableResource { /** The log object used for debugging. */ private static final Logger LOG = LoggerFactory.getLogger(BlobServer.class); @@ -861,15 +869,14 @@ public boolean deletePermanent(JobID jobId, PermanentBlobKey key) { } /** - * Removes all BLOBs from local and HA store belonging to the given job ID. + * Deletes locally stored artifacts for the job represented by the given {@link JobID}. This + * doesn't touch the job's entry in the {@link BlobStore} to enable recovering. * - * @param jobId ID of the job this blob belongs to - * @param cleanupBlobStoreFiles True if the corresponding blob store files shall be cleaned up - * as well. Otherwise false. - * @return true if the job directory is successfully deleted or non-existing; - * false otherwise + * @param jobId The {@code JobID} of the job that is subject to cleanup. + * @throws IOException if the cleanup failed. */ - public boolean cleanupJob(JobID jobId, boolean cleanupBlobStoreFiles) { + @Override + public void localCleanup(JobID jobId) throws IOException { checkNotNull(jobId); final File jobDir = @@ -877,45 +884,58 @@ public boolean cleanupJob(JobID jobId, boolean cleanupBlobStoreFiles) { BlobUtils.getStorageLocationPath( storageDir.deref().getAbsolutePath(), jobId)); + FileUtils.deleteDirectory(jobDir); + + // NOTE on why blobExpiryTimes are not cleaned up: + // Instead of going through blobExpiryTimes, keep lingering entries - they + // will be cleaned up by the timer task which tolerates non-existing files + // If inserted again with the same IDs (via put()), the TTL will be updated + // again. + } + + /** + * Removes all BLOBs from local and HA store belonging to the given {@link JobID}. + * + * @param jobId ID of the job this blob belongs to + * @throws Exception if the cleanup fails. + */ + @Override + public void globalCleanup(JobID jobId) throws Exception { + checkNotNull(jobId); + readWriteLock.writeLock().lock(); try { - // delete locally - boolean deletedLocally = false; - try { - FileUtils.deleteDirectory(jobDir); - - // NOTE on why blobExpiryTimes are not cleaned up: - // Instead of going through blobExpiryTimes, keep lingering entries - they - // will be cleaned up by the timer task which tolerates non-existing files - // If inserted again with the same IDs (via put()), the TTL will be updated - // again. + Exception exception = null; - deletedLocally = true; + try { + localCleanup(jobId); } catch (IOException e) { - LOG.warn( - "Failed to locally delete BLOB storage directory at " - + jobDir.getAbsolutePath(), - e); + exception = e; } - // delete in HA blob store files - final boolean deletedHA = !cleanupBlobStoreFiles || blobStore.deleteAll(jobId); + if (!blobStore.deleteAll(jobId)) { + exception = + ExceptionUtils.firstOrSuppressed( + new FlinkException( + "Error while cleaning up the BlobStore for job " + jobId), + exception); + } - return deletedLocally && deletedHA; + ExceptionUtils.tryRethrowException(exception); } finally { readWriteLock.writeLock().unlock(); } } - public void retainJobs(Collection jobsToRetain) throws IOException { + public void retainJobs(Collection jobsToRetain) throws Exception { if (storageDir.deref().exists()) { final Set jobsToRemove = BlobUtils.listExistingJobs(storageDir.deref().toPath()); jobsToRemove.removeAll(jobsToRetain); for (JobID jobToRemove : jobsToRemove) { - cleanupJob(jobToRemove, true); + globalCleanup(jobToRemove); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 0080ee1281955..b7f0f54f717cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -895,8 +895,25 @@ private void cleanUpRemainingJobData(JobID jobId, boolean jobGraphRemoved) { log.warn( "Could not properly clean data for job {} stored by ha services", jobId, e); } + + try { + blobServer.globalCleanup(jobId); + } catch (Exception e) { + log.warn( + "Could not properly global clean data for job {} stored in the BlobServer.", + jobId, + e); + } + } else { + try { + blobServer.localCleanup(jobId); + } catch (IOException e) { + log.warn( + "Could not properly clean local data for job {} stored in the BlobServer.", + jobId, + e); + } } - blobServer.cleanupJob(jobId, jobGraphRemoved); } private void markJobAsClean(JobID jobId) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java index 42fa2353dd573..d06dd9712b0f7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java @@ -44,7 +44,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -317,14 +316,12 @@ public void testPermanentBlobDeferredCleanup() throws IOException, InterruptedEx } @Test - public void testTransientBlobNoJobCleanup() - throws IOException, InterruptedException, ExecutionException { + public void testTransientBlobNoJobCleanup() throws Exception { testTransientBlobCleanup(null); } @Test - public void testTransientBlobForJobCleanup() - throws IOException, InterruptedException, ExecutionException { + public void testTransientBlobForJobCleanup() throws Exception { testTransientBlobCleanup(new JobID()); } @@ -332,8 +329,7 @@ public void testTransientBlobForJobCleanup() * Tests that {@link TransientBlobCache} cleans up after a default TTL and keeps files which are * constantly accessed. */ - private void testTransientBlobCleanup(@Nullable final JobID jobId) - throws IOException, InterruptedException, ExecutionException { + private void testTransientBlobCleanup(@Nullable final JobID jobId) throws Exception { // 1s should be a safe-enough buffer to still check for existence after a BLOB's last access long cleanupInterval = 1L; // in seconds @@ -390,7 +386,7 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) // files are cached now for the given TTL - remove from server so that they are not // re-downloaded if (jobId != null) { - server.cleanupJob(jobId, true); + server.globalCleanup(jobId); } else { server.deleteFromCache(key1); server.deleteFromCache(key2); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java index f4beefd510e8c..be0ba5d72d3af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java @@ -220,7 +220,7 @@ private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) } @Test - public void testBlobServerRetainsJobs() throws IOException { + public void testBlobServerRetainsJobs() throws Exception { final File storageDirectory = temporaryFolder.newFolder(); final JobID jobId1 = new JobID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java index a87a2b1c4cd06..f6acef3f38ec0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java @@ -266,21 +266,22 @@ private void testDeleteBlobFails(@Nullable final JobID jobId, BlobKey.BlobType b } @Test - public void testJobCleanup() throws IOException { + public void testJobCleanup() throws Exception { testJobCleanup(TRANSIENT_BLOB); } @Test - public void testJobCleanupHa() throws IOException { + public void testJobCleanupHa() throws Exception { testJobCleanup(PERMANENT_BLOB); } /** - * Tests that {@link BlobServer} cleans up after calling {@link BlobServer#cleanupJob}. + * Tests that {@link BlobServer} cleans up after calling {@link + * BlobServer#globalCleanup(JobID)}. * * @param blobType whether the BLOB should become permanent or transient */ - private void testJobCleanup(BlobKey.BlobType blobType) throws IOException { + private void testJobCleanup(BlobKey.BlobType blobType) throws Exception { JobID jobId1 = new JobID(); JobID jobId2 = new JobID(); @@ -308,7 +309,7 @@ private void testJobCleanup(BlobKey.BlobType blobType) throws IOException { verifyContents(server, jobId2, key2, data); checkFileCountForJob(1, jobId2, server); - server.cleanupJob(jobId1, true); + server.globalCleanup(jobId1); verifyDeleted(server, jobId1, key1a); verifyDeleted(server, jobId1, key1b); @@ -316,14 +317,14 @@ private void testJobCleanup(BlobKey.BlobType blobType) throws IOException { verifyContents(server, jobId2, key2, data); checkFileCountForJob(1, jobId2, server); - server.cleanupJob(jobId2, true); + server.globalCleanup(jobId2); checkFileCountForJob(0, jobId1, server); verifyDeleted(server, jobId2, key2); checkFileCountForJob(0, jobId2, server); // calling a second time should not fail - server.cleanupJob(jobId2, true); + server.globalCleanup(jobId2); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java index dfa279930638e..655d64adb50a7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java @@ -89,7 +89,7 @@ public void testBlobServerRecovery() throws Exception { */ public static void testBlobServerRecovery( final Configuration config, final BlobStore blobStore, final File blobStorage) - throws IOException { + throws Exception { final String clusterId = config.getString(HighAvailabilityOptions.HA_CLUSTER_ID); String storagePath = config.getString(HighAvailabilityOptions.HA_STORAGE_PATH) + "/" + clusterId; @@ -141,8 +141,8 @@ public static void testBlobServerRecovery( verifyDeleted(cache1, jobId[0], nonHAKey); // Remove again - server1.cleanupJob(jobId[0], true); - server1.cleanupJob(jobId[1], true); + server1.globalCleanup(jobId[0]); + server1.globalCleanup(jobId[1]); // Verify everything is clean assertTrue("HA storage directory does not exist", fs.exists(new Path(storagePath))); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 4992c76ddc9b2..68dcf7a1494d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -63,6 +63,7 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.function.ThrowingConsumer; import org.hamcrest.core.IsInstanceOf; import org.junit.After; @@ -136,7 +137,8 @@ public class DispatcherResourceCleanupTest extends TestLogger { private CompletableFuture storedHABlobFuture; private CompletableFuture deleteAllHABlobsFuture; - private CompletableFuture cleanupJobFuture; + private CompletableFuture localCleanupFuture; + private CompletableFuture globalCleanupFuture; private CompletableFuture cleanupJobHADataFuture; private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; @@ -169,14 +171,16 @@ public void setup() throws Exception { .setDeleteAllFunction(deleteAllHABlobsFuture::complete) .createTestingBlobStore(); - cleanupJobFuture = new CompletableFuture<>(); + globalCleanupFuture = new CompletableFuture<>(); + localCleanupFuture = new CompletableFuture<>(); blobServer = new TestingBlobServer( configuration, temporaryFolder.newFolder(), testingBlobStore, - cleanupJobFuture); + jobId -> globalCleanupFuture.complete(jobId), + jobId -> localCleanupFuture.complete(jobId)); } private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob() throws Exception { @@ -254,8 +258,8 @@ public void testBlobServerCleanupWhenJobFinished() throws Exception { } private void assertThatHABlobsHaveBeenRemoved() - throws InterruptedException, ExecutionException { - assertThat(cleanupJobFuture.get(), equalTo(jobId)); + throws InterruptedException, ExecutionException, TimeoutException { + assertGlobalCleanupTriggered(jobId); // verify that we also cleared the BlobStore assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); @@ -294,8 +298,7 @@ public void testBlobServerCleanupWhenJobNotFinished() throws Exception { jobManagerRunnerFactory.takeCreatedJobManagerRunner(); suspendJob(testingJobManagerRunner); - assertThat(cleanupJobFuture.get(), equalTo(jobId)); - + assertLocalCleanupTriggered(jobId); assertThat(blobFile.exists(), is(false)); // verify that we did not clear the BlobStore @@ -331,8 +334,7 @@ public void testBlobServerCleanupWhenClosingDispatcher() throws Exception { dispatcher.closeAsync().get(); - assertThat(cleanupJobFuture.get(), equalTo(jobId)); - + assertLocalCleanupTriggered(jobId); assertThat(blobFile.exists(), is(false)); // verify that we did not clear the BlobStore @@ -375,7 +377,7 @@ public void testHACleanupWhenJobFinishedWhileClosingDispatcher() throws Exceptio // check that no exceptions have been thrown dispatcherTerminationFuture.get(); - assertThat(cleanupJobFuture.get(), is(jobId)); + assertGlobalCleanupTriggered(jobId); assertThat(deleteAllHABlobsFuture.get(), is(jobId)); } @@ -468,7 +470,7 @@ public void testDuplicateJobSubmissionDoesNotDeleteJobMetaData() throws Exceptio is(true)); } - assertThatHABlobsHaveNotBeenRemoved(); + assertThatNoCleanupWasTriggered(); } finally { finishJob(testingJobManagerRunnerFactoryNG.takeCreatedJobManagerRunner()); } @@ -519,8 +521,9 @@ private void terminateJobWithState( .build())); } - private void assertThatHABlobsHaveNotBeenRemoved() { - assertThat(cleanupJobFuture.isDone(), is(false)); + private void assertThatNoCleanupWasTriggered() { + assertThat(globalCleanupFuture.isDone(), is(false)); + assertThat(localCleanupFuture.isDone(), is(false)); assertThat(deleteAllHABlobsFuture.isDone(), is(false)); assertThat(blobFile.exists(), is(true)); } @@ -640,7 +643,7 @@ public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception jobManagerRunnerFactory.takeCreatedJobManagerRunner(); testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); - assertThat(cleanupJobFuture.get(), equalTo(jobId)); + assertLocalCleanupTriggered(jobId); assertThat(deleteAllHABlobsFuture.isDone(), is(false)); } @@ -659,10 +662,22 @@ public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception jobManagerRunnerFactory.takeCreatedJobManagerRunner(); testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); - assertThat(cleanupJobFuture.get(), equalTo(jobId)); + assertGlobalCleanupTriggered(jobId); assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); } + private void assertLocalCleanupTriggered(JobID jobId) + throws ExecutionException, InterruptedException, TimeoutException { + assertThat(localCleanupFuture.get(100, TimeUnit.MILLISECONDS), equalTo(jobId)); + assertThat(globalCleanupFuture.isDone(), is(false)); + } + + private void assertGlobalCleanupTriggered(JobID jobId) + throws ExecutionException, InterruptedException, TimeoutException { + assertThat(localCleanupFuture.get(100, TimeUnit.MILLISECONDS), equalTo(jobId)); + assertThat(globalCleanupFuture.get(100, TimeUnit.MILLISECONDS), equalTo(jobId)); + } + @Test public void testFatalErrorIfJobCannotBeMarkedDirtyInJobResultStore() throws Exception { jobResultStore = @@ -738,14 +753,18 @@ public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws private static final class TestingBlobServer extends BlobServer { - private final CompletableFuture cleanupJobFuture; + private final ThrowingConsumer globalCleanupConsumer; + private final ThrowingConsumer localCleanupConsumer; /** * Instantiates a new BLOB server and binds it to a free network port. * * @param config Configuration to be used to instantiate the BlobServer * @param blobStore BlobStore to store blobs persistently - * @param cleanupJobFuture + * @param globalCleanupConsumer The consumer called along the actual {@link + * #globalCleanup(JobID)} call. + * @param localCleanupConsumer The consumer called along the actual {@link + * #localCleanup(JobID)} call. * @throws IOException thrown if the BLOB server cannot bind to a free network port or if * the (local or distributed) file storage cannot be created or is not usable */ @@ -753,17 +772,24 @@ public TestingBlobServer( Configuration config, File storageDirectory, BlobStore blobStore, - CompletableFuture cleanupJobFuture) + ThrowingConsumer globalCleanupConsumer, + ThrowingConsumer localCleanupConsumer) throws IOException { super(config, storageDirectory, blobStore); - this.cleanupJobFuture = cleanupJobFuture; + this.globalCleanupConsumer = globalCleanupConsumer; + this.localCleanupConsumer = localCleanupConsumer; + } + + @Override + public void globalCleanup(JobID jobId) throws Exception { + super.globalCleanup(jobId); + globalCleanupConsumer.accept(jobId); } @Override - public boolean cleanupJob(JobID jobId, boolean cleanupBlobStoreFiles) { - final boolean result = super.cleanupJob(jobId, cleanupBlobStoreFiles); - cleanupJobFuture.complete(jobId); - return result; + public void localCleanup(JobID jobId) throws IOException { + super.localCleanup(jobId); + localCleanupConsumer.accept(jobId); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java index b2e1b3d6d4b66..f849d65b37435 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java @@ -160,7 +160,7 @@ public void testRecoveryRegisterAndDownload() throws Exception { } // Remove blobs again - server[1].cleanupJob(jobId, true); + server[1].globalCleanup(jobId); // Verify everything is clean below recoveryDir/ final String clusterId = config.getString(HighAvailabilityOptions.HA_CLUSTER_ID); From d05c9a96cbdf7a9065cf1ffabd1d1a2bd932777d Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 12:40:09 +0100 Subject: [PATCH 30/49] [FLINK-25432] Makes HighAvailabilityServices implement LocallyCleanableResource and GloballyCleanableResource --- .../flink/runtime/dispatcher/Dispatcher.java | 2 +- .../highavailability/AbstractHaServices.java | 2 +- .../HighAvailabilityServices.java | 19 ++++++++++-------- .../DispatcherResourceCleanupTest.java | 2 +- .../runtime/dispatcher/DispatcherTest.java | 4 ++-- .../AbstractHaServicesTest.java | 8 ++++---- .../TestingHighAvailabilityServices.java | 20 ++++++++++++++----- .../zookeeper/ZooKeeperHaServicesTest.java | 2 +- 8 files changed, 36 insertions(+), 23 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index b7f0f54f717cb..8ef362a13f0f2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -890,7 +890,7 @@ private void cleanUpRemainingJobData(JobID jobId, boolean jobGraphRemoved) { jobManagerMetricGroup.removeJob(jobId); if (jobGraphRemoved) { try { - highAvailabilityServices.cleanupJobData(jobId); + highAvailabilityServices.globalCleanup(jobId); } catch (Exception e) { log.warn( "Could not properly clean data for job {} stored by ha services", jobId, e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java index d6c0418f37995..d1c1926a6d1ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java @@ -206,7 +206,7 @@ public void closeAndCleanupAllData() throws Exception { } @Override - public void cleanupJobData(JobID jobID) throws Exception { + public void globalCleanup(JobID jobID) throws Exception { logger.info("Clean up the high availability data for job {}.", jobID); internalCleanupJobData(jobID); logger.info("Finished cleaning up the high availability data for job {}.", jobID); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index 8df9227ce22b6..672973239ca72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -21,6 +21,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.blob.BlobStore; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -43,7 +45,10 @@ *

  • Naming of RPC endpoints * */ -public interface HighAvailabilityServices extends ClientHighAvailabilityServices { +public interface HighAvailabilityServices + extends ClientHighAvailabilityServices, + LocallyCleanableResource, + GloballyCleanableResource { // ------------------------------------------------------------------------ // Constants @@ -239,11 +244,9 @@ default LeaderRetrievalService getClusterRestEndpointLeaderRetriever() { */ void closeAndCleanupAllData() throws Exception; - /** - * Deletes all data for specified job stored by these services in external stores. - * - * @param jobID The identifier of the job to cleanup. - * @throws Exception Thrown, if an exception occurred while cleaning data stored by them. - */ - default void cleanupJobData(JobID jobID) throws Exception {} + @Override + default void globalCleanup(JobID jobId) throws Exception {} + + @Override + default void localCleanup(JobID jobId) throws Exception {} } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 68dcf7a1494d2..f395791de8a3c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -159,7 +159,7 @@ public void setup() throws Exception { jobResultStore = new SingleJobResultStore(jobId, clearedJobLatch); highAvailabilityServices.setJobResultStore(jobResultStore); cleanupJobHADataFuture = new CompletableFuture<>(); - highAvailabilityServices.setCleanupJobDataFuture(cleanupJobHADataFuture); + highAvailabilityServices.setGlobalCleanupFuture(cleanupJobHADataFuture); storedHABlobFuture = new CompletableFuture<>(); deleteAllHABlobsFuture = new CompletableFuture<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index bec6936a66932..b9213ee44e361 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -740,7 +740,7 @@ public void testFailingJobManagerRunnerCleanup() throws Exception { // Track cleanup - ha-services final CompletableFuture cleanupJobData = new CompletableFuture<>(); - haServices.setCleanupJobDataFuture(cleanupJobData); + haServices.setGlobalCleanupFuture(cleanupJobData); cleanupJobData.thenAccept(jobId -> cleanUpEvents.add(CLEANUP_HA_SERVICES)); // Track cleanup - job-graph @@ -1136,7 +1136,7 @@ private void testJobDataAreCleanedUpInCorrectOrder(JobStatus jobStatus) throws E // Track cleanup - ha-services final CompletableFuture cleanupJobData = new CompletableFuture<>(); - haServices.setCleanupJobDataFuture(cleanupJobData); + haServices.setGlobalCleanupFuture(cleanupJobData); cleanupJobData.thenAccept(jobId -> cleanUpEvents.add(CLEANUP_HA_SERVICES)); // Track cleanup - job-graph diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java index a105cf8bf842d..433ed3aa7cb55 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java @@ -31,6 +31,7 @@ import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; import org.apache.flink.util.function.RunnableWithException; +import org.apache.flink.util.function.ThrowingConsumer; import org.junit.Test; @@ -40,7 +41,6 @@ import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import java.util.function.Consumer; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.is; @@ -130,7 +130,7 @@ public void testCleanupJobData() throws Exception { () -> {}, jobCleanupFuture::complete); - haServices.cleanupJobData(jobID); + haServices.globalCleanup(jobID); JobID jobIDCleaned = jobCleanupFuture.get(); assertThat(jobIDCleaned, is(jobID)); } @@ -185,7 +185,7 @@ private static final class TestingHaServices extends AbstractHaServices { private final Queue closeOperations; private final RunnableWithException internalCleanupRunnable; - private final Consumer internalJobCleanupConsumer; + private final ThrowingConsumer internalJobCleanupConsumer; private TestingHaServices( Configuration config, @@ -193,7 +193,7 @@ private TestingHaServices( BlobStoreService blobStoreService, Queue closeOperations, RunnableWithException internalCleanupRunnable, - Consumer internalJobCleanupConsumer) { + ThrowingConsumer internalJobCleanupConsumer) { super( config, ioExecutor, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java index d98150a5fb5c6..e4c573f21f4cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -73,7 +73,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private CompletableFuture closeAndCleanupAllDataFuture = new CompletableFuture<>(); - private volatile CompletableFuture jobCleanupFuture; + private volatile CompletableFuture globalCleanupFuture; + private volatile CompletableFuture localCleanupFuture; // ------------------------------------------------------------------------ // Setters for mock / testing implementations @@ -148,8 +149,12 @@ public void setCloseAndCleanupAllDataFuture( this.closeAndCleanupAllDataFuture = closeAndCleanupAllDataFuture; } - public void setCleanupJobDataFuture(CompletableFuture jobCleanupFuture) { - this.jobCleanupFuture = jobCleanupFuture; + public void setGlobalCleanupFuture(CompletableFuture globalCleanupFuture) { + this.globalCleanupFuture = globalCleanupFuture; + } + + public void setLocalCleanupFuture(CompletableFuture localCleanupFuture) { + this.localCleanupFuture = localCleanupFuture; } // ------------------------------------------------------------------------ @@ -286,7 +291,12 @@ public void closeAndCleanupAllData() throws Exception { } @Override - public void cleanupJobData(JobID jobID) { - Optional.ofNullable(jobCleanupFuture).ifPresent(f -> f.complete(jobID)); + public void globalCleanup(JobID jobID) throws Exception { + Optional.ofNullable(globalCleanupFuture).ifPresent(f -> f.complete(jobID)); + } + + @Override + public void localCleanup(JobID jobID) throws Exception { + Optional.ofNullable(localCleanupFuture).ifPresent(f -> f.complete(jobID)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java index b95fb74f020da..04556da7b2f62 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java @@ -173,7 +173,7 @@ public void testCleanupJobData() throws Exception { haServices -> { final List childrenBefore = client.getChildren().forPath(path); - haServices.cleanupJobData(jobID); + haServices.globalCleanup(jobID); final List childrenAfter = client.getChildren().forPath(path); From a011193483860d3f5d97d6fc5ccb5cf1e6da7d52 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 12:42:22 +0100 Subject: [PATCH 31/49] [FLINK-25432] Makes JobManagerMetricGroup implement LocallyCleanableResource and GloballyCleanableResource --- .../flink/runtime/dispatcher/Dispatcher.java | 10 +++++++++- .../metrics/groups/JobManagerMetricGroup.java | 17 +++++++++++++++-- .../metrics/groups/JobManagerGroupTest.java | 4 ++-- 3 files changed, 26 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 8ef362a13f0f2..2429609a4ede6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -887,7 +887,15 @@ private boolean cleanUpJobGraph(JobID jobId, boolean cleanupHA) { } private void cleanUpRemainingJobData(JobID jobId, boolean jobGraphRemoved) { - jobManagerMetricGroup.removeJob(jobId); + try { + jobManagerMetricGroup.globalCleanup(jobId); + } catch (Exception e) { + log.warn( + "Could not properly clean data for job {} stored in JobManager metric group", + jobId, + e); + } + if (jobGraphRemoved) { try { highAvailabilityServices.globalCleanup(jobId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/JobManagerMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/JobManagerMetricGroup.java index 431dab46bcce2..0ac161402ce1d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/JobManagerMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/JobManagerMetricGroup.java @@ -20,6 +20,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.scope.ScopeFormat; @@ -33,7 +35,8 @@ *

    Contains extra logic for adding jobs with tasks, and removing jobs when they do not contain * tasks any more */ -public class JobManagerMetricGroup extends ComponentMetricGroup { +public class JobManagerMetricGroup extends ComponentMetricGroup + implements LocallyCleanableResource, GloballyCleanableResource { private final Map jobs = new HashMap<>(); @@ -84,7 +87,17 @@ public JobManagerJobMetricGroup addJob(JobID jobId, String jobName) { } } - public void removeJob(JobID jobId) { + @Override + public void globalCleanup(JobID jobId) { + cleanup(jobId); + } + + @Override + public void localCleanup(JobID jobId) { + cleanup(jobId); + } + + private void cleanup(JobID jobId) { if (jobId == null) { return; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java index 958c01a4a189b..ed74b2e363019 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java @@ -62,12 +62,12 @@ public void addAndRemoveJobs() throws Exception { assertEquals(2, group.numRegisteredJobMetricGroups()); - group.removeJob(jid1); + group.globalCleanup(jid1); assertTrue(jmJobGroup11.isClosed()); assertEquals(1, group.numRegisteredJobMetricGroups()); - group.removeJob(jid2); + group.globalCleanup(jid2); assertTrue(jmJobGroup21.isClosed()); assertEquals(0, group.numRegisteredJobMetricGroups()); From febcf75336ce303389066ee995159820f7e2e34f Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 14:54:27 +0100 Subject: [PATCH 32/49] [FLINK-25432] Adds JobManagerRunnerRegistry and integrates it into the Dispatcher --- .../flink/runtime/dispatcher/Dispatcher.java | 64 +++-- .../dispatcher/JobManagerRunnerRegistry.java | 129 +++++++++ .../JobManagerRunnerRegistryTest.java | 265 ++++++++++++++++++ .../jobmaster/TestingJobManagerRunner.java | 6 +- 4 files changed, 433 insertions(+), 31 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistry.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistryTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 2429609a4ede6..4ac09e18e0488 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -122,7 +122,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint runningJobs; + private final JobManagerRunnerRegistry jobManagerRunnerRegistry; private final Collection recoveredJobs; @@ -183,7 +183,7 @@ public Dispatcher( JobManagerSharedServices.fromConfiguration( configuration, blobServer, fatalErrorHandler); - runningJobs = new HashMap<>(16); + jobManagerRunnerRegistry = new JobManagerRunnerRegistry(16); this.historyServerArchivist = dispatcherServices.getHistoryServerArchivist(); @@ -383,7 +383,7 @@ public CompletableFuture submitFailedJob( * @throws FlinkException if the job scheduling status cannot be retrieved */ private boolean isDuplicateJob(JobID jobId) throws FlinkException { - return isInGloballyTerminalState(jobId) || runningJobs.containsKey(jobId); + return isInGloballyTerminalState(jobId) || jobManagerRunnerRegistry.isRegistered(jobId); } /** @@ -456,12 +456,12 @@ private void persistAndRunJob(JobGraph jobGraph) throws Exception { } private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Exception { - Preconditions.checkState(!runningJobs.containsKey(jobGraph.getJobID())); + Preconditions.checkState(!jobManagerRunnerRegistry.isRegistered(jobGraph.getJobID())); long initializationTimestamp = System.currentTimeMillis(); JobManagerRunner jobManagerRunner = createJobManagerRunner(jobGraph, initializationTimestamp); - runningJobs.put(jobGraph.getJobID(), jobManagerRunner); + jobManagerRunnerRegistry.register(jobManagerRunner); final JobID jobId = jobGraph.getJobID(); @@ -471,7 +471,9 @@ private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Excep .handleAsync( (jobManagerRunnerResult, throwable) -> { Preconditions.checkState( - runningJobs.get(jobId) == jobManagerRunner, + jobManagerRunnerRegistry.isRegistered(jobId) + && jobManagerRunnerRegistry.get(jobId) + == jobManagerRunner, "The job entry in runningJobs must be bound to the lifetime of the JobManagerRunner."); if (jobManagerRunnerResult != null) { @@ -543,7 +545,7 @@ JobManagerRunner createJobManagerRunner(JobGraph jobGraph, long initializationTi @Override public CompletableFuture> listJobs(Time timeout) { return CompletableFuture.completedFuture( - Collections.unmodifiableSet(new HashSet<>(runningJobs.keySet()))); + Collections.unmodifiableSet(jobManagerRunnerRegistry.getRunningJobIds())); } @Override @@ -688,9 +690,7 @@ public CompletableFuture requestExecutionGraphInfo( @Override public CompletableFuture requestJobResult(JobID jobId, Time timeout) { - JobManagerRunner job = runningJobs.get(jobId); - - if (job == null) { + if (!jobManagerRunnerRegistry.isRegistered(jobId)) { final ExecutionGraphInfo executionGraphInfo = executionGraphInfoStore.get(jobId); if (executionGraphInfo == null) { @@ -699,15 +699,17 @@ public CompletableFuture requestJobResult(JobID jobId, Time timeout) return CompletableFuture.completedFuture( JobResult.createFrom(executionGraphInfo.getArchivedExecutionGraph())); } - } else { - return job.getResultFuture() - .thenApply( - jobManagerRunnerResult -> - JobResult.createFrom( - jobManagerRunnerResult - .getExecutionGraphInfo() - .getArchivedExecutionGraph())); } + + final JobManagerRunner jobManagerRunner = jobManagerRunnerRegistry.get(jobId); + return jobManagerRunner + .getResultFuture() + .thenApply( + jobManagerRunnerResult -> + JobResult.createFrom( + jobManagerRunnerResult + .getExecutionGraphInfo() + .getArchivedExecutionGraph())); } @Override @@ -841,7 +843,7 @@ private void registerJobManagerRunnerTerminationFuture( } private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJobState) { - final JobManagerRunner job = checkNotNull(runningJobs.remove(jobId)); + final JobManagerRunner job = checkNotNull(jobManagerRunnerRegistry.unregister(jobId)); return CompletableFuture.supplyAsync( () -> cleanUpJobGraph(jobId, cleanupJobState.cleanupHAData), ioExecutor) .thenCompose( @@ -943,7 +945,7 @@ private void cleanUpHighAvailabilityJobData(JobID jobId) { private void terminateRunningJobs() { log.info("Stopping all currently running jobs of dispatcher {}.", getAddress()); - final HashSet jobsToRemove = new HashSet<>(runningJobs.keySet()); + final Set jobsToRemove = jobManagerRunnerRegistry.getRunningJobIds(); for (JobID jobId : jobsToRemove) { terminateJob(jobId); @@ -951,9 +953,8 @@ private void terminateRunningJobs() { } private void terminateJob(JobID jobId) { - final JobManagerRunner jobManagerRunner = runningJobs.get(jobId); - - if (jobManagerRunner != null) { + if (jobManagerRunnerRegistry.isRegistered(jobId)) { + final JobManagerRunner jobManagerRunner = jobManagerRunnerRegistry.get(jobId); jobManagerRunner.closeAsync(); } } @@ -1068,11 +1069,11 @@ private void jobMasterFailed(JobID jobId, Throwable cause) { /** Ensures that the JobMasterGateway is available. */ private CompletableFuture getJobMasterGateway(JobID jobId) { - JobManagerRunner job = runningJobs.get(jobId); - if (job == null) { + if (!jobManagerRunnerRegistry.isRegistered(jobId)) { return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); } + final JobManagerRunner job = jobManagerRunnerRegistry.get(jobId); if (!job.isInitialized()) { return FutureUtils.completedExceptionally( new UnavailableDispatcherOperationException( @@ -1092,7 +1093,9 @@ private CompletableFuture getResourceManagerGateway() { } private Optional getJobManagerRunner(JobID jobId) { - return Optional.ofNullable(runningJobs.get(jobId)); + return jobManagerRunnerRegistry.isRegistered(jobId) + ? Optional.of(jobManagerRunnerRegistry.get(jobId)) + : Optional.empty(); } private CompletableFuture runResourceManagerCommand( @@ -1114,9 +1117,9 @@ private List>> queryJobMastersForInformation( Function> queryFunction) { List>> optionalJobInformation = - new ArrayList<>(runningJobs.size()); + new ArrayList<>(jobManagerRunnerRegistry.size()); - for (JobManagerRunner job : runningJobs.values()) { + for (JobManagerRunner job : jobManagerRunnerRegistry.getJobManagerRunners()) { final CompletableFuture> queryResult = queryFunction .apply(job) @@ -1150,7 +1153,7 @@ private CompletableFuture waitForTerminatingJob( } CompletableFuture getJobTerminationFuture(JobID jobId) { - if (runningJobs.containsKey(jobId)) { + if (jobManagerRunnerRegistry.isRegistered(jobId)) { return FutureUtils.completedExceptionally( new DispatcherException( String.format("Job with job id %s is still running.", jobId))); @@ -1161,7 +1164,8 @@ CompletableFuture getJobTerminationFuture(JobID jobId) { } private void registerDispatcherMetrics(MetricGroup jobManagerMetricGroup) { - jobManagerMetricGroup.gauge(MetricNames.NUM_RUNNING_JOBS, () -> (long) runningJobs.size()); + jobManagerMetricGroup.gauge( + MetricNames.NUM_RUNNING_JOBS, () -> (long) jobManagerRunnerRegistry.size()); } public CompletableFuture onRemovedJobGraph(JobID jobId) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistry.java new file mode 100644 index 0000000000000..dc9f2f4b6e4e8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistry.java @@ -0,0 +1,129 @@ +/* + * 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.flink.runtime.dispatcher; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +/** {@code JobManagerRunner} collects running jobs represented by {@link JobManagerRunner}. */ +public class JobManagerRunnerRegistry + implements LocallyCleanableResource, GloballyCleanableResource { + + @VisibleForTesting final Map jobManagerRunners; + + public JobManagerRunnerRegistry(int initialCapacity) { + Preconditions.checkArgument(initialCapacity > 0); + jobManagerRunners = new HashMap<>(initialCapacity); + } + + public boolean isRegistered(JobID jobId) { + return jobManagerRunners.containsKey(jobId); + } + + public void register(JobManagerRunner jobManagerRunner) { + Preconditions.checkArgument( + !isRegistered(jobManagerRunner.getJobID()), + "A job with the ID %s is already registered.", + jobManagerRunner.getJobID()); + this.jobManagerRunners.put(jobManagerRunner.getJobID(), jobManagerRunner); + } + + /** + * Returns the {@link JobManagerRunner} for the given {@code JobID}. + * + * @throws NoSuchElementException if the passed {@code JobID} does not belong to a registered + * {@code JobManagerRunner}. + * @see #isRegistered(JobID) + */ + public JobManagerRunner get(JobID jobId) { + assertJobRegistered(jobId); + return this.jobManagerRunners.get(jobId); + } + + public int size() { + return this.jobManagerRunners.size(); + } + + public Set getRunningJobIds() { + return this.jobManagerRunners.keySet(); + } + + public Collection getJobManagerRunners() { + return this.jobManagerRunners.values(); + } + + @Override + public void globalCleanup(JobID jobId) throws Exception { + cleanup(jobId); + } + + @Override + public CompletableFuture globalCleanupAsync(JobID jobId, Executor unusedExecutor) { + return cleanupAsync(jobId); + } + + @Override + public void localCleanup(JobID jobId) throws Exception { + cleanup(jobId); + } + + @Override + public CompletableFuture localCleanupAsync(JobID jobId, Executor unusedExecutor) { + return cleanupAsync(jobId); + } + + private void cleanup(JobID jobId) throws Exception { + if (isRegistered(jobId)) { + unregister(jobId).close(); + } + } + + private CompletableFuture cleanupAsync(JobID jobId) { + if (!isRegistered(jobId)) { + return FutureUtils.completedVoidFuture(); + } + + return unregister(jobId).closeAsync(); + } + + public JobManagerRunner unregister(JobID jobId) { + assertJobRegistered(jobId); + return this.jobManagerRunners.remove(jobId); + } + + private void assertJobRegistered(JobID jobId) { + if (!isRegistered(jobId)) { + throw new NoSuchElementException( + "There is no running job registered for the job ID " + jobId); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistryTest.java new file mode 100644 index 0000000000000..314b038627de8 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistryTest.java @@ -0,0 +1,265 @@ +/* + * 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.flink.runtime.dispatcher; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.concurrent.Executors; +import org.apache.flink.util.function.BiFunctionWithException; + +import org.assertj.core.api.ThrowingConsumer; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * {@code JobManagerRunnerRegistryTest} tests the functionality of {@link JobManagerRunnerRegistry}. + */ +public class JobManagerRunnerRegistryTest { + + private JobManagerRunnerRegistry testInstance; + + @BeforeEach + public void setup() { + testInstance = new JobManagerRunnerRegistry(4); + } + + @Test + public void testIsRegistered() { + final JobID jobId = new JobID(); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId).build()); + assertThat(testInstance.isRegistered(jobId)).isTrue(); + } + + @Test + public void testIsNotRegistered() { + assertThat(testInstance.isRegistered(new JobID())).isFalse(); + } + + @Test + public void testRegister() { + final JobID jobId = new JobID(); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId).build()); + assertThat(testInstance.isRegistered(jobId)).isTrue(); + } + + @Test + public void testRegisteringTwiceCausesFailure() { + final JobID jobId = new JobID(); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId).build()); + assertThat(testInstance.isRegistered(jobId)).isTrue(); + + assertThatThrownBy( + () -> + testInstance.register( + TestingJobManagerRunner.newBuilder() + .setJobId(jobId) + .build())) + .isInstanceOf(IllegalArgumentException.class); + } + + @Test + public void testGet() { + final JobID jobId = new JobID(); + final JobManagerRunner jobManagerRunner = + TestingJobManagerRunner.newBuilder().setJobId(jobId).build(); + testInstance.register(jobManagerRunner); + + assertThat(testInstance.get(jobId)).isEqualTo(jobManagerRunner); + } + + @Test + public void testGetOnNonExistingJobManagerRunner() { + assertThatThrownBy(() -> testInstance.get(new JobID())) + .isInstanceOf(NoSuchElementException.class); + } + + @Test + public void size() { + assertThat(testInstance.size()).isEqualTo(0); + testInstance.register(TestingJobManagerRunner.newBuilder().build()); + assertThat(testInstance.size()).isEqualTo(1); + testInstance.register(TestingJobManagerRunner.newBuilder().build()); + assertThat(testInstance.size()).isEqualTo(2); + } + + @Test + public void testGetRunningJobIds() { + assertThat(testInstance.getRunningJobIds()).isEmpty(); + + final JobID jobId0 = new JobID(); + final JobID jobId1 = new JobID(); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId0).build()); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId1).build()); + + assertThat(testInstance.getRunningJobIds()).containsExactlyInAnyOrder(jobId0, jobId1); + } + + @Test + public void testGetJobManagerRunners() { + assertThat(testInstance.getJobManagerRunners()).isEmpty(); + + final JobManagerRunner jobManagerRunner0 = TestingJobManagerRunner.newBuilder().build(); + final JobManagerRunner jobManagerRunner1 = TestingJobManagerRunner.newBuilder().build(); + testInstance.register(jobManagerRunner0); + testInstance.register(jobManagerRunner1); + + assertThat(testInstance.getJobManagerRunners()) + .containsExactlyInAnyOrder(jobManagerRunner0, jobManagerRunner1); + } + + @Test + public void testSuccessfulGlobalCleanup() throws Throwable { + testSuccessfulSynchronousCleanup(testInstance::globalCleanup); + } + + @Test + public void testSuccessfulLocalCleanup() throws Throwable { + testSuccessfulSynchronousCleanup(testInstance::localCleanup); + } + + private void testSuccessfulSynchronousCleanup(ThrowingConsumer callback) + throws Throwable { + final TestingJobManagerRunner jobManagerRunner = registerTestingJobManagerRunner(); + + callback.acceptThrows(jobManagerRunner.getJobID()); + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isFalse(); + assertThat(jobManagerRunner.getTerminationFuture()).isCompleted(); + } + + @Test + public void testFailingGlobalCleanup() { + testFailingSynchronousCleanup(testInstance::globalCleanup); + } + + @Test + public void testFailingLocalCleanup() { + testFailingSynchronousCleanup(testInstance::localCleanup); + } + + private void testFailingSynchronousCleanup(ThrowingConsumer callback) { + final TestingJobManagerRunner jobManagerRunner = registerTestingJobManagerRunner(); + + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isTrue(); + assertThat(jobManagerRunner.getTerminationFuture()).isNotDone(); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + jobManagerRunner.completeTerminationFutureExceptionally(expectedException); + + assertThatThrownBy(() -> callback.acceptThrows(jobManagerRunner.getJobID())) + .isInstanceOf(FlinkException.class) + .hasCause(expectedException); + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isFalse(); + } + + @Test + public void testSuccessfulGlobalCleanupAsync() throws Exception { + testSuccessfulCleanupAsync(testInstance::globalCleanupAsync); + } + + @Test + public void testSuccessfulLocalCleanupAsync() throws Exception { + testSuccessfulCleanupAsync(testInstance::localCleanupAsync); + } + + private void testSuccessfulCleanupAsync( + BiFunctionWithException, Exception> callback) + throws Exception { + final TestingJobManagerRunner jobManagerRunner = registerTestingJobManagerRunner(); + + final CompletableFuture cleanupResult = + callback.apply(jobManagerRunner.getJobID(), Executors.directExecutor()); + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isFalse(); + assertThat(cleanupResult).isCompleted(); + } + + @Test + public void testFailingGlobalCleanupAsync() throws Exception { + testFailingCleanupAsync(testInstance::globalCleanupAsync); + } + + @Test + public void testFailingLocalCleanupAsync() throws Exception { + testFailingCleanupAsync(testInstance::localCleanupAsync); + } + + private void testFailingCleanupAsync( + BiFunctionWithException, Exception> callback) + throws Exception { + final TestingJobManagerRunner jobManagerRunner = registerTestingJobManagerRunner(); + + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isTrue(); + assertThat(jobManagerRunner.getTerminationFuture()).isNotDone(); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + jobManagerRunner.completeTerminationFutureExceptionally(expectedException); + + final CompletableFuture cleanupResult = + callback.apply(jobManagerRunner.getJobID(), Executors.directExecutor()); + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isFalse(); + assertThat(cleanupResult) + .isCompletedExceptionally() + .failsWithin(Duration.ZERO) + .withThrowableOfType(ExecutionException.class) + .withCause(expectedException); + } + + private TestingJobManagerRunner registerTestingJobManagerRunner() { + final TestingJobManagerRunner jobManagerRunner = + TestingJobManagerRunner.newBuilder().build(); + testInstance.register(jobManagerRunner); + + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isTrue(); + assertThat(jobManagerRunner.getTerminationFuture()).isNotDone(); + + return jobManagerRunner; + } + + @Test + public void testGlobalCleanupAsyncOnUnknownJobId() { + assertThat(testInstance.globalCleanupAsync(new JobID(), Executors.directExecutor())) + .isCompleted(); + } + + @Test + public void testLocalCleanupAsyncOnUnknownJobId() { + assertThat(testInstance.localCleanupAsync(new JobID(), Executors.directExecutor())) + .isCompleted(); + } + + @Test + public void testGlobalCleanupOnUnknownJobId() throws Exception { + testInstance.globalCleanup(new JobID()); + } + + @Test + public void testLocalCleanupOnUnknownJobId() throws Exception { + testInstance.localCleanup(new JobID()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java index 0c0994d990dd8..7b6adf544ab65 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java @@ -155,6 +155,10 @@ public void completeTerminationFuture() { terminationFuture.complete(null); } + public void completeTerminationFutureExceptionally(Throwable expectedException) { + terminationFuture.completeExceptionally(expectedException); + } + public CompletableFuture getTerminationFuture() { return terminationFuture; } @@ -166,7 +170,7 @@ public void completeJobMasterGatewayFuture(JobMasterGateway testingJobMasterGate /** {@code Builder} for instantiating {@link TestingJobManagerRunner} instances. */ public static class Builder { - private JobID jobId = null; + private JobID jobId = new JobID(); private boolean blockingTermination = false; private CompletableFuture jobMasterGatewayFuture = new CompletableFuture<>(); From c769cc2a4d1ef8d194383f840abde8f70a9043ac Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 15:54:14 +0100 Subject: [PATCH 33/49] [FLINK-25432] Refactors Dispatcher constructor signature Moves JobManagerRunnerRegistry instantiation into separate constructor. The purpose of this change is to improve the testability of the code. --- .../flink/runtime/dispatcher/Dispatcher.java | 100 ++++++++++++++---- 1 file changed, 82 insertions(+), 18 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 4ac09e18e0488..bd8e9ffb9c99c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -162,34 +162,98 @@ public Dispatcher( DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { + this( + rpcService, + fencingToken, + recoveredJobs, + recoveredDirtyJobs, + dispatcherBootstrapFactory, + dispatcherServices, + new JobManagerRunnerRegistry(16)); + } + + private Dispatcher( + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + Collection globallyTerminatedJobs, + DispatcherBootstrapFactory dispatcherBootstrapFactory, + DispatcherServices dispatcherServices, + JobManagerRunnerRegistry jobManagerRunnerRegistry) + throws Exception { + this( + rpcService, + fencingToken, + recoveredJobs, + globallyTerminatedJobs, + dispatcherServices.getConfiguration(), + dispatcherServices.getHighAvailabilityServices(), + dispatcherServices.getResourceManagerGatewayRetriever(), + dispatcherServices.getHeartbeatServices(), + dispatcherServices.getBlobServer(), + dispatcherServices.getFatalErrorHandler(), + dispatcherServices.getJobGraphWriter(), + dispatcherServices.getJobResultStore(), + dispatcherServices.getJobManagerMetricGroup(), + dispatcherServices.getMetricQueryServiceAddress(), + dispatcherServices.getIoExecutor(), + dispatcherServices.getHistoryServerArchivist(), + dispatcherServices.getArchivedExecutionGraphStore(), + dispatcherServices.getJobManagerRunnerFactory(), + dispatcherBootstrapFactory, + dispatcherServices.getOperationCaches(), + jobManagerRunnerRegistry); + } + + private Dispatcher( + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + Collection recoveredDirtyJobs, + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + HeartbeatServices heartbeatServices, + BlobServer blobServer, + FatalErrorHandler fatalErrorHandler, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricServiceQueryAddress, + Executor ioExecutor, + HistoryServerArchivist historyServerArchivist, + ExecutionGraphInfoStore executionGraphInfoStore, + JobManagerRunnerFactory jobManagerRunnerFactory, + DispatcherBootstrapFactory dispatcherBootstrapFactory, + DispatcherOperationCaches dispatcherOperationCaches, + JobManagerRunnerRegistry jobManagerRunnerRegistry) + throws Exception { super(rpcService, RpcServiceUtils.createRandomName(DISPATCHER_NAME), fencingToken); - checkNotNull(dispatcherServices); assertRecoveredJobsAndDirtyJobResults(recoveredJobs, recoveredDirtyJobs); - this.configuration = dispatcherServices.getConfiguration(); - this.highAvailabilityServices = dispatcherServices.getHighAvailabilityServices(); - this.resourceManagerGatewayRetriever = - dispatcherServices.getResourceManagerGatewayRetriever(); - this.heartbeatServices = dispatcherServices.getHeartbeatServices(); - this.blobServer = dispatcherServices.getBlobServer(); - this.fatalErrorHandler = dispatcherServices.getFatalErrorHandler(); - this.jobGraphWriter = dispatcherServices.getJobGraphWriter(); - this.jobResultStore = dispatcherServices.getJobResultStore(); - this.jobManagerMetricGroup = dispatcherServices.getJobManagerMetricGroup(); - this.metricServiceQueryAddress = dispatcherServices.getMetricQueryServiceAddress(); - this.ioExecutor = dispatcherServices.getIoExecutor(); + this.configuration = checkNotNull(configuration); + this.highAvailabilityServices = checkNotNull(highAvailabilityServices); + this.resourceManagerGatewayRetriever = checkNotNull(resourceManagerGatewayRetriever); + this.heartbeatServices = checkNotNull(heartbeatServices); + this.blobServer = checkNotNull(blobServer); + this.fatalErrorHandler = checkNotNull(fatalErrorHandler); + this.jobGraphWriter = checkNotNull(jobGraphWriter); + this.jobResultStore = checkNotNull(jobResultStore); + this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup); + this.metricServiceQueryAddress = metricServiceQueryAddress; + this.ioExecutor = checkNotNull(ioExecutor); this.jobManagerSharedServices = JobManagerSharedServices.fromConfiguration( configuration, blobServer, fatalErrorHandler); - jobManagerRunnerRegistry = new JobManagerRunnerRegistry(16); + this.jobManagerRunnerRegistry = checkNotNull(jobManagerRunnerRegistry); - this.historyServerArchivist = dispatcherServices.getHistoryServerArchivist(); + this.historyServerArchivist = checkNotNull(historyServerArchivist); - this.executionGraphInfoStore = dispatcherServices.getArchivedExecutionGraphStore(); + this.executionGraphInfoStore = checkNotNull(executionGraphInfoStore); - this.jobManagerRunnerFactory = dispatcherServices.getJobManagerRunnerFactory(); + this.jobManagerRunnerFactory = checkNotNull(jobManagerRunnerFactory); this.jobManagerRunnerTerminationFutures = new HashMap<>(2); @@ -204,7 +268,7 @@ public Dispatcher( this.dispatcherCachedOperationsHandler = new DispatcherCachedOperationsHandler( - dispatcherServices.getOperationCaches(), + checkNotNull(dispatcherOperationCaches), this::triggerSavepointAndGetLocation, this::stopWithSavepointAndGetLocation); } From 6c5e8781c20bcee5fb18d0f8524dd970c5ff2d9d Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 13:34:20 +0100 Subject: [PATCH 34/49] [FLINK-25432] Adds ResourceCleaner (+ default implementation) and integrates it into Dispatcher Additionally, testHABlobsAreNotRemovedIfHAJobGraphRemovalFails is deleted because this case doesn't apply anymore. The job graph deletion and the HA file deletion happen independently from each other now: The JobResult is already added to the JobResultStore as a dirty entry and all data can be deleted. There is a dependency between cleaning up the JobManagerRegistry (i.e. closing the JobMaster) and cleaning up the HighAvailabilityServices for the job. This issue is solved by a JobManager-wide leader election (FLINK-24038). But we have to consider closing the JobMaster before cleaning the HA job data until the legacy leader election functionality is removed entirely. --- .../flink/runtime/dispatcher/Dispatcher.java | 130 ++---- .../cleanup/DefaultResourceCleaner.java | 74 +++ .../DispatcherResourceCleanerFactory.java | 109 +++++ .../dispatcher/cleanup/ResourceCleaner.java | 30 ++ .../cleanup/ResourceCleanerFactory.java | 45 ++ .../dispatcher/DispatcherFailoverITCase.java | 76 ++-- .../DispatcherResourceCleanupTest.java | 29 -- .../runtime/dispatcher/DispatcherTest.java | 6 +- .../cleanup/DefaultResourceCleanerTest.java | 227 ++++++++++ .../DispatcherResourceCleanerFactoryTest.java | 420 ++++++++++++++++++ .../TestingResourceCleanerFactory.java | 89 ++++ 11 files changed, 1074 insertions(+), 161 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleaner.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleaner.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleanerFactory.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleanerTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactoryTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index bd8e9ffb9c99c..f03c2692d5545 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -32,6 +32,9 @@ import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleaner; +import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; import org.apache.flink.runtime.entrypoint.ClusterEntryPointExceptionUtils; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; @@ -148,6 +151,9 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint internalSubmitJob(JobGraph jobGraph) { return persistAndRunFuture.handleAsync( (acknowledge, throwable) -> { if (throwable != null) { - cleanUpHighAvailabilityJobData(jobGraph.getJobID()); + globalResourceCleaner + .cleanupAsync(jobGraph.getJobID()) + .exceptionally( + t -> { + log.warn( + "Cleanup didn't succeed after job submission failed for job " + + jobGraph.getJobID(), + t); + return null; + }); ClusterEntryPointExceptionUtils.tryEnrichClusterEntryPointError(throwable); final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable); @@ -572,14 +592,8 @@ private CleanupJobState handleJobManagerRunnerResult( } enum CleanupJobState { - LOCAL(false), - GLOBAL(true); - - final boolean cleanupHAData; - - CleanupJobState(boolean cleanupHAData) { - this.cleanupHAData = cleanupHAData; - } + LOCAL, + GLOBAL } private CleanupJobState jobManagerRunnerFailed(JobID jobId, Throwable throwable) { @@ -907,86 +921,15 @@ private void registerJobManagerRunnerTerminationFuture( } private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJobState) { - final JobManagerRunner job = checkNotNull(jobManagerRunnerRegistry.unregister(jobId)); - return CompletableFuture.supplyAsync( - () -> cleanUpJobGraph(jobId, cleanupJobState.cleanupHAData), ioExecutor) - .thenCompose( - jobGraphRemoved -> job.closeAsync().thenApply(ignored -> jobGraphRemoved)) - .thenAcceptAsync( - jobGraphRemoved -> { - cleanUpRemainingJobData(jobId, jobGraphRemoved); - if (jobGraphRemoved) { - markJobAsClean(jobId); - } - }, - ioExecutor); - } - - /** - * Clean up job graph from {@link org.apache.flink.runtime.jobmanager.JobGraphStore}. - * - * @param jobId Reference to the job that we want to clean. - * @param cleanupHA Flag signalling whether we should remove (we're done with the job) or just - * release the job graph. - * @return True if we have removed the job graph. This means we can clean other HA-related - * services as well. - */ - private boolean cleanUpJobGraph(JobID jobId, boolean cleanupHA) { - if (cleanupHA) { - try { - jobGraphWriter.globalCleanup(jobId); - return true; - } catch (Exception e) { - log.warn( - "Could not properly remove job {} from submitted job graph store.", - jobId, - e); - return false; - } - } - try { - jobGraphWriter.localCleanup(jobId); - } catch (Exception e) { - log.warn("Could not properly release job {} from submitted job graph store.", jobId, e); - } - return false; - } - - private void cleanUpRemainingJobData(JobID jobId, boolean jobGraphRemoved) { - try { - jobManagerMetricGroup.globalCleanup(jobId); - } catch (Exception e) { - log.warn( - "Could not properly clean data for job {} stored in JobManager metric group", - jobId, - e); - } - - if (jobGraphRemoved) { - try { - highAvailabilityServices.globalCleanup(jobId); - } catch (Exception e) { - log.warn( - "Could not properly clean data for job {} stored by ha services", jobId, e); - } - - try { - blobServer.globalCleanup(jobId); - } catch (Exception e) { - log.warn( - "Could not properly global clean data for job {} stored in the BlobServer.", - jobId, - e); - } - } else { - try { - blobServer.localCleanup(jobId); - } catch (IOException e) { - log.warn( - "Could not properly clean local data for job {} stored in the BlobServer.", - jobId, - e); - } + switch (cleanupJobState) { + case LOCAL: + return localResourceCleaner.cleanupAsync(jobId); + case GLOBAL: + return globalResourceCleaner + .cleanupAsync(jobId) + .thenRun(() -> markJobAsClean(jobId)); + default: + throw new IllegalStateException("Invalid cleanup state: " + cleanupJobState); } } @@ -1000,11 +943,6 @@ private void markJobAsClean(JobID jobId) { } } - private void cleanUpHighAvailabilityJobData(JobID jobId) { - final boolean jobGraphRemoved = cleanUpJobGraph(jobId, true); - cleanUpRemainingJobData(jobId, jobGraphRemoved); - } - /** Terminate all currently running {@link JobManagerRunner}s. */ private void terminateRunningJobs() { log.info("Stopping all currently running jobs of dispatcher {}.", getAddress()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleaner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleaner.java new file mode 100644 index 0000000000000..42bfeba66e892 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleaner.java @@ -0,0 +1,74 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +/** {@code DefaultResourceCleaner} is the default implementation of {@link ResourceCleaner}. */ +public class DefaultResourceCleaner implements ResourceCleaner { + + private final Collection>> + prioritizedOrderedJobRelatedCleanups = new ArrayList<>(); + private final Collection>> + jobRelatedCleanups = new ArrayList<>(); + private final Executor cleanupExecutor; + + DefaultResourceCleaner(Executor cleanupExecutor) { + this.cleanupExecutor = cleanupExecutor; + } + + DefaultResourceCleaner withPriorityCleanupOf( + BiFunction> cleanupCallback) { + prioritizedOrderedJobRelatedCleanups.add(cleanupCallback); + return this; + } + + DefaultResourceCleaner withCleanupOf( + BiFunction> cleanupCallback) { + jobRelatedCleanups.add(cleanupCallback); + return this; + } + + @Override + public CompletableFuture cleanupAsync(JobID jobId) { + CompletableFuture prioritizedOrderedCleanupFuture = FutureUtils.completedVoidFuture(); + for (BiFunction> prioritizedCleanupFunction : + prioritizedOrderedJobRelatedCleanups) { + prioritizedOrderedCleanupFuture = + prioritizedOrderedCleanupFuture.thenCompose( + ignoredValue -> + prioritizedCleanupFunction.apply(jobId, cleanupExecutor)); + } + + return prioritizedOrderedCleanupFuture.thenCompose( + ignoredValue -> + FutureUtils.completeAll( + jobRelatedCleanups.stream() + .map(c -> c.apply(jobId, cleanupExecutor)) + .collect(Collectors.toList()))); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java new file mode 100644 index 0000000000000..3cfc6ea56b0c2 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java @@ -0,0 +1,109 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.dispatcher.DispatcherServices; +import org.apache.flink.runtime.dispatcher.JobManagerRunnerRegistry; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.util.Preconditions; + +import java.util.Arrays; +import java.util.concurrent.Executor; + +/** + * {@code DispatcherResourceCleanerFactory} instantiates {@link ResourceCleaner} instances that + * clean cleanable resources from the {@link org.apache.flink.runtime.dispatcher.Dispatcher}. + * + *

    We need to handle the {@link JobManagerRunnerRegistry} differently due to a dependency between + * closing the {@link org.apache.flink.runtime.jobmaster.JobManagerRunner} and the {@link + * HighAvailabilityServices}. This is fixed in {@code FLINK-24038} using a feature flag to + * enable/disable single leader election for all the {@code JobManager} components. We can remove + * the priority cleanup logic after removing the per-component leader election. + */ +public class DispatcherResourceCleanerFactory implements ResourceCleanerFactory { + + private final Executor cleanupExecutor; + private final JobManagerRunnerRegistry jobManagerRunnerRegistry; + private final JobGraphWriter jobGraphWriter; + private final BlobServer blobServer; + private final HighAvailabilityServices highAvailabilityServices; + private final JobManagerMetricGroup jobManagerMetricGroup; + + public DispatcherResourceCleanerFactory( + JobManagerRunnerRegistry jobManagerRunnerRegistry, + DispatcherServices dispatcherServices) { + this( + dispatcherServices.getIoExecutor(), + jobManagerRunnerRegistry, + dispatcherServices.getJobGraphWriter(), + dispatcherServices.getBlobServer(), + dispatcherServices.getHighAvailabilityServices(), + dispatcherServices.getJobManagerMetricGroup()); + } + + @VisibleForTesting + DispatcherResourceCleanerFactory( + Executor cleanupExecutor, + JobManagerRunnerRegistry jobManagerRunnerRegistry, + JobGraphWriter jobGraphWriter, + BlobServer blobServer, + HighAvailabilityServices highAvailabilityServices, + JobManagerMetricGroup jobManagerMetricGroup) { + this.cleanupExecutor = Preconditions.checkNotNull(cleanupExecutor); + this.jobManagerRunnerRegistry = Preconditions.checkNotNull(jobManagerRunnerRegistry); + this.jobGraphWriter = Preconditions.checkNotNull(jobGraphWriter); + this.blobServer = Preconditions.checkNotNull(blobServer); + this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices); + this.jobManagerMetricGroup = Preconditions.checkNotNull(jobManagerMetricGroup); + } + + public ResourceCleaner createLocalResourceCleaner() { + final DefaultResourceCleaner resourceCleaner = new DefaultResourceCleaner(cleanupExecutor); + resourceCleaner.withPriorityCleanupOf(jobManagerRunnerRegistry::localCleanupAsync); + for (LocallyCleanableResource locallyCleanableResource : + Arrays.asList( + jobGraphWriter, + blobServer, + highAvailabilityServices, + jobManagerMetricGroup)) { + resourceCleaner.withCleanupOf(locallyCleanableResource::localCleanupAsync); + } + + return resourceCleaner; + } + + public ResourceCleaner createGlobalResourceCleaner() { + final DefaultResourceCleaner resourceCleaner = new DefaultResourceCleaner(cleanupExecutor); + resourceCleaner.withPriorityCleanupOf(jobManagerRunnerRegistry::globalCleanupAsync); + for (GloballyCleanableResource globallyCleanableResource : + Arrays.asList( + jobGraphWriter, + blobServer, + highAvailabilityServices, + jobManagerMetricGroup)) { + resourceCleaner.withCleanupOf(globallyCleanableResource::globalCleanupAsync); + } + + return resourceCleaner; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleaner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleaner.java new file mode 100644 index 0000000000000..9a43dd59a7e16 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleaner.java @@ -0,0 +1,30 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; + +import java.util.concurrent.CompletableFuture; + +/** {@code ResourceCleaner} executes instances on the given {@code JobID}. */ +@FunctionalInterface +public interface ResourceCleaner { + + CompletableFuture cleanupAsync(JobID jobId); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleanerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleanerFactory.java new file mode 100644 index 0000000000000..91584dc0a2e31 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleanerFactory.java @@ -0,0 +1,45 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; + +import java.util.concurrent.Executor; + +/** + * {@code ResourceCleanerFactory} provides methods to create {@link ResourceCleaner} for local and + * global cleanup. + * + * @see GloballyCleanableResource + * @see LocallyCleanableResource + */ +public interface ResourceCleanerFactory { + + /** + * Creates {@link ResourceCleaner} that initiates {@link + * LocallyCleanableResource#localCleanupAsync(JobID, Executor)} calls. + */ + ResourceCleaner createLocalResourceCleaner(); + + /** + * Creates {@link ResourceCleaner} that initiates {@link + * GloballyCleanableResource#globalCleanupAsync(JobID, Executor)} calls. + */ + ResourceCleaner createGlobalResourceCleaner(); +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java index e75051d7f75fb..1875c59bd2e1b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java @@ -20,11 +20,9 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.checkpoint.EmbeddedCompletedCheckpointStore; -import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.checkpoint.PerJobCheckpointRecoveryFactory; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphBuilder; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -32,6 +30,7 @@ import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; @@ -40,6 +39,7 @@ import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.util.ExceptionUtils; +import org.hamcrest.CoreMatchers; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -49,15 +49,19 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; -import static org.junit.Assert.assertFalse; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; /** An integration test for various fail-over scenarios of the {@link Dispatcher} component. */ @@ -72,10 +76,11 @@ public void setUp() throws Exception { new PerJobCheckpointRecoveryFactory( (maxCheckpoints, previous, sharedStateRegistryFactory, ioExecutor) -> { if (previous != null) { - // First job attempt failed before cleaning up the checkpoint - // store. - assertFalse(previous.getShutdownStatus().isPresent()); - assertFalse(previous.getAllCheckpoints().isEmpty()); + // First job cleanup still succeeded for the + // CompletedCheckpointStore because the JobGraph cleanup happens + // after the JobManagerRunner closing + assertTrue(previous.getShutdownStatus().isPresent()); + assertTrue(previous.getAllCheckpoints().isEmpty()); return new EmbeddedCompletedCheckpointStore( maxCheckpoints, previous.getAllCheckpoints(), @@ -109,12 +114,16 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() final JobID jobId = jobGraph.getJobID(); // Construct job graph store. - final Error jobGraphRemovalError = new Error("Unable to remove job graph."); + final Error temporaryError = new Error("Unable to remove job graph."); + final AtomicReference temporaryErrorRef = + new AtomicReference<>(temporaryError); final TestingJobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() .setGlobalCleanupConsumer( graph -> { - throw jobGraphRemovalError; + if (temporaryErrorRef.get() != null) { + throw temporaryErrorRef.getAndSet(null); + } }) .build(); jobGraphStore.start(null); @@ -132,8 +141,7 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() throwable -> { final Optional maybeError = ExceptionUtils.findThrowable(throwable, Error.class); - if (maybeError.isPresent() - && jobGraphRemovalError.equals(maybeError.get())) { + if (maybeError.isPresent() && temporaryError.equals(maybeError.get())) { jobGraphRemovalErrorReceived.countDown(); } else { testingFatalErrorHandlerResource @@ -169,28 +177,30 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() // This will clear internal state of election service, so a new contender can register. leaderElectionService.stop(); + assertThat( + "The JobGraph is still stored in the JobGraphStore.", + haServices.getJobGraphStore().getJobIds(), + CoreMatchers.is(Collections.singleton(jobId))); + assertThat( + "The JobResultStore has this job marked as dirty.", + haServices.getJobResultStore().getDirtyResults().stream() + .map(JobResult::getJobId) + .collect(Collectors.toSet()), + CoreMatchers.is(Collections.singleton(jobId))); + // Run a second dispatcher, that restores our finished job. final Dispatcher secondDispatcher = createRecoveredDispatcher(null); toTerminate.add(secondDispatcher); - final DispatcherGateway secondDispatcherGateway = - secondDispatcher.getSelfGateway(DispatcherGateway.class); + + // new Dispatcher becomes new leader leaderElectionService.isLeader(UUID.randomUUID()); - // Now make sure that restored job started from checkpoint. - final JobMasterGateway secondJobMasterGateway = - connectToLeadingJobMaster(leaderElectionService).get(); - try (final JobMasterTester tester = - new JobMasterTester(rpcService, jobId, secondJobMasterGateway)) { - final CompletableFuture> descriptorsFuture = - tester.deployVertices(2); - awaitStatus(secondDispatcherGateway, jobId, JobStatus.RUNNING); - final Optional maybeRestore = - descriptorsFuture.get().stream() - .map(TaskDeploymentDescriptor::getTaskRestore) - .filter(Objects::nonNull) - .findAny(); - assertTrue("Job has recovered from checkpoint.", maybeRestore.isPresent()); - } + assertThrows( + "No JobMaster will be instantiated because of the JobResult is already persisted in the JobResultStore", + TimeoutException.class, + () -> + connectToLeadingJobMaster(leaderElectionService) + .get(100, TimeUnit.MILLISECONDS)); } private JobGraph createJobGraph() { @@ -221,11 +231,11 @@ private TestingDispatcher createRecoveredDispatcher( @Nullable FatalErrorHandler fatalErrorHandler) throws Exception { final List jobGraphs = new ArrayList<>(); for (JobID jobId : haServices.getJobGraphStore().getJobIds()) { - jobGraphs.add(haServices.getJobGraphStore().recoverJobGraph(jobId)); + // there shouldn't be an overlap between dirty JobResults and recovered JobGraphs + if (!haServices.getJobResultStore().hasJobResultEntry(jobId)) { + jobGraphs.add(haServices.getJobGraphStore().recoverJobGraph(jobId)); + } } - // we need to reinstantiate the JobResultStore here to simulate a not-persisting - // JobResultStore - haServices.setJobResultStore(new EmbeddedJobResultStore()); final TestingDispatcher dispatcher = new TestingDispatcherBuilder() .setJobManagerRunnerFactory( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index f395791de8a3c..af47cf438f54b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -55,7 +55,6 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.util.ExceptionUtils; @@ -619,34 +618,6 @@ public Set getDirtyResults() throws IOException { } } - @Test - public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception { - jobGraphWriter = - TestingJobGraphStore.newBuilder() - .setGlobalCleanupConsumer( - ignored -> { - throw new Exception("Failed to Remove future"); - }) - .withAutomaticStart() - .build(); - - final TestingJobManagerRunnerFactory jobManagerRunnerFactory = - startDispatcherAndSubmitJob(); - - ArchivedExecutionGraph executionGraph = - new ArchivedExecutionGraphBuilder() - .setJobID(jobId) - .setState(JobStatus.CANCELED) - .build(); - - final TestingJobManagerRunner testingJobManagerRunner = - jobManagerRunnerFactory.takeCreatedJobManagerRunner(); - testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); - - assertLocalCleanupTriggered(jobId); - assertThat(deleteAllHABlobsFuture.isDone(), is(false)); - } - @Test public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index b9213ee44e361..dca0232f604d3 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -91,7 +91,6 @@ import org.assertj.core.api.Assertions; import org.hamcrest.Matchers; -import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -131,6 +130,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -779,7 +779,7 @@ public void testFailingJobManagerRunnerCleanup() throws Exception { // make sure we've cleaned up in correct order (including HA) assertThat( new ArrayList<>(cleanUpEvents), - equalTo(Arrays.asList(CLEANUP_JOB_GRAPH_REMOVE, CLEANUP_HA_SERVICES))); + containsInAnyOrder(CLEANUP_JOB_GRAPH_REMOVE, CLEANUP_HA_SERVICES)); } // don't fail this time @@ -1158,7 +1158,7 @@ private void testJobDataAreCleanedUpInCorrectOrder(JobStatus jobStatus) throws E assertThat( "All cleanup tasks should have been finished before marking the job as clean.", cleanUpEvents, - IsIterableContainingInAnyOrder.containsInAnyOrder( + containsInAnyOrder( CLEANUP_HA_SERVICES, CLEANUP_JOB_GRAPH_REMOVE, CLEANUP_JOB_MANAGER_RUNNER))) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleanerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleanerTest.java new file mode 100644 index 0000000000000..47a29cbd74f6f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleanerTest.java @@ -0,0 +1,227 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.function.BiFunction; +import java.util.function.Consumer; + +import static org.assertj.core.api.Assertions.assertThat; + +/** {@code DefaultResourceCleanerTest} tests {@link DefaultResourceCleaner}. */ +public class DefaultResourceCleanerTest { + + private static final Executor EXECUTOR = Executors.directExecutor(); + private static final JobID JOB_ID = new JobID(); + + private DefaultResourceCleaner testInstance; + private CleanupCallback cleanup0; + private CleanupCallback cleanup1; + + @BeforeEach + public void setup() { + cleanup0 = CleanupCallback.withoutCompletionOnCleanup(); + cleanup1 = CleanupCallback.withoutCompletionOnCleanup(); + + testInstance = new DefaultResourceCleaner(EXECUTOR); + testInstance.withCleanupOf(cleanup0).withCleanupOf(cleanup1); + } + + @Test + public void testSuccessfulConcurrentCleanup() { + CompletableFuture cleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + assertThat(cleanup1).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + + cleanup0.completeCleanup(); + assertThat(cleanupResult).isNotCompleted(); + + cleanup1.completeCleanup(); + assertThat(cleanupResult).isCompleted(); + } + + @Test + public void testConcurrentCleanupWithExceptionFirst() { + CompletableFuture cleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + assertThat(cleanup1).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + cleanup0.completeCleanupExceptionally(expectedException); + assertThat(cleanupResult).isNotCompleted(); + + cleanup1.completeCleanup(); + assertThat(cleanupResult) + .failsWithin(Duration.ZERO) + .withThrowableOfType(ExecutionException.class) + .withCause(expectedException); + } + + @Test + public void testConcurrentCleanupWithExceptionSecond() { + CompletableFuture cleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + assertThat(cleanup1).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + + cleanup0.completeCleanup(); + assertThat(cleanupResult).isNotCompleted(); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + cleanup1.completeCleanupExceptionally(expectedException); + assertThat(cleanupResult) + .failsWithin(Duration.ZERO) + .withThrowableOfType(ExecutionException.class) + .withCause(expectedException); + } + + @Test + public void testHighestPriorityCleanupBlocksAllOtherCleanups() { + final CleanupCallback highPriorityCleanup = CleanupCallback.withoutCompletionOnCleanup(); + final CleanupCallback lowerThanHighPriorityCleanup = + CleanupCallback.withCompletionOnCleanup(); + final CleanupCallback noPriorityCleanup0 = CleanupCallback.withCompletionOnCleanup(); + final CleanupCallback noPriorityCleanup1 = CleanupCallback.withCompletionOnCleanup(); + + final DefaultResourceCleaner testInstance = new DefaultResourceCleaner(EXECUTOR); + + testInstance + .withPriorityCleanupOf(highPriorityCleanup) + .withPriorityCleanupOf(lowerThanHighPriorityCleanup) + .withCleanupOf(noPriorityCleanup0) + .withPriorityCleanupOf(noPriorityCleanup1); + + final CompletableFuture overallCleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(highPriorityCleanup.isDone()).isFalse(); + assertThat(lowerThanHighPriorityCleanup.isDone()).isFalse(); + assertThat(noPriorityCleanup0.isDone()).isFalse(); + assertThat(noPriorityCleanup1.isDone()).isFalse(); + + assertThat(overallCleanupResult.isDone()).isFalse(); + + highPriorityCleanup.completeCleanup(); + + assertThat(overallCleanupResult).succeedsWithin(Duration.ofMillis(100)); + + assertThat(highPriorityCleanup.isDone()).isTrue(); + assertThat(lowerThanHighPriorityCleanup.isDone()).isTrue(); + assertThat(noPriorityCleanup0.isDone()).isTrue(); + assertThat(noPriorityCleanup1.isDone()).isTrue(); + } + + @Test + public void testMediumPriorityCleanupBlocksAllLowerPrioritizedCleanups() { + final CleanupCallback highPriorityCleanup = CleanupCallback.withCompletionOnCleanup(); + final CleanupCallback lowerThanHighPriorityCleanup = + CleanupCallback.withoutCompletionOnCleanup(); + final CleanupCallback noPriorityCleanup0 = CleanupCallback.withCompletionOnCleanup(); + final CleanupCallback noPriorityCleanup1 = CleanupCallback.withCompletionOnCleanup(); + + final DefaultResourceCleaner testInstance = new DefaultResourceCleaner(EXECUTOR); + + testInstance + .withPriorityCleanupOf(highPriorityCleanup) + .withPriorityCleanupOf(lowerThanHighPriorityCleanup) + .withCleanupOf(noPriorityCleanup0) + .withPriorityCleanupOf(noPriorityCleanup1); + + assertThat(highPriorityCleanup.isDone()).isFalse(); + + final CompletableFuture overallCleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(highPriorityCleanup.isDone()).isTrue(); + assertThat(lowerThanHighPriorityCleanup.isDone()).isFalse(); + assertThat(noPriorityCleanup0.isDone()).isFalse(); + assertThat(noPriorityCleanup1.isDone()).isFalse(); + + assertThat(overallCleanupResult.isDone()).isFalse(); + + lowerThanHighPriorityCleanup.completeCleanup(); + + assertThat(overallCleanupResult).succeedsWithin(Duration.ofMillis(100)); + + assertThat(highPriorityCleanup.isDone()).isTrue(); + assertThat(lowerThanHighPriorityCleanup.isDone()).isTrue(); + assertThat(noPriorityCleanup0.isDone()).isTrue(); + assertThat(noPriorityCleanup1.isDone()).isTrue(); + } + + private static class CleanupCallback + implements BiFunction> { + + private final CompletableFuture resultFuture = new CompletableFuture<>(); + private JobID jobId; + + private final Consumer> internalFunction; + + public static CleanupCallback withCompletionOnCleanup() { + return new CleanupCallback(resultFuture -> resultFuture.complete(null)); + } + + public static CleanupCallback withoutCompletionOnCleanup() { + return new CleanupCallback(ignoredResultFuture -> {}); + } + + private CleanupCallback(Consumer> internalFunction) { + this.internalFunction = internalFunction; + } + + @Override + public CompletableFuture apply(JobID jobId, Executor executor) { + Preconditions.checkState(this.jobId == null); + this.jobId = jobId; + + internalFunction.accept(resultFuture); + + return resultFuture; + } + + public boolean isDone() { + return resultFuture.isDone(); + } + + public JobID getProcessedJobId() { + return jobId; + } + + public void completeCleanup() { + this.resultFuture.complete(null); + } + + public void completeCleanupExceptionally(Throwable expectedException) { + this.resultFuture.completeExceptionally(expectedException); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactoryTest.java new file mode 100644 index 0000000000000..4b3ffbccc736d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactoryTest.java @@ -0,0 +1,420 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.BlobStore; +import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.dispatcher.JobManagerRunnerRegistry; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.metrics.util.TestingMetricRegistry; +import org.apache.flink.util.concurrent.Executors; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeoutException; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * {@code DispatcherResourceCleanerFactoryTest} verifies that the resources are properly cleaned up + * for both, the {@link GloballyCleanableResource} and {@link LocallyCleanableResource} of the + * {@link org.apache.flink.runtime.dispatcher.Dispatcher}. + */ +public class DispatcherResourceCleanerFactoryTest { + + private static final JobID JOB_ID = new JobID(); + + private CleanableJobManagerRegistry jobManagerRunnerRegistry; + private CleanableJobGraphWriter jobGraphWriter; + private CleanableBlobServer blobServer; + private CleanableHighAvailabilityServices highAvailabilityServices; + private JobManagerMetricGroup jobManagerMetricGroup; + + private DispatcherResourceCleanerFactory testInstance; + + @BeforeEach + public void setup() throws IOException { + jobManagerRunnerRegistry = new CleanableJobManagerRegistry(); + jobGraphWriter = new CleanableJobGraphWriter(); + blobServer = new CleanableBlobServer(); + highAvailabilityServices = new CleanableHighAvailabilityServices(); + + MetricRegistry metricRegistry = TestingMetricRegistry.builder().build(); + jobManagerMetricGroup = + JobManagerMetricGroup.createJobManagerMetricGroup( + metricRegistry, "ignored hostname"); + jobManagerMetricGroup.addJob(JOB_ID, "ignored job name"); + + testInstance = + new DispatcherResourceCleanerFactory( + Executors.directExecutor(), + jobManagerRunnerRegistry, + jobGraphWriter, + blobServer, + highAvailabilityServices, + jobManagerMetricGroup); + } + + @Test + public void testLocalResourceCleaning() { + assertGlobalCleanupNotTriggered(); + assertLocalCleanupNotTriggered(); + assertJobManagerMetricGroupNotCleaned(); + + final CompletableFuture cleanupResultFuture = + testInstance.createLocalResourceCleaner().cleanupAsync(JOB_ID); + + assertGlobalCleanupNotTriggered(); + assertLocalCleanupTriggeredWaitingForJobManagerRunnerRegistry(); + assertJobManagerMetricGroupNotCleaned(); + + assertThat(cleanupResultFuture).isNotCompleted(); + + jobManagerRunnerRegistry.completeLocalCleanup(); + + assertGlobalCleanupNotTriggered(); + assertLocalCleanupTriggered(); + assertJobManagerMetricGroupCleaned(); + + assertThat(cleanupResultFuture).isCompleted(); + } + + @Test + public void testGlobalResourceCleaning() + throws ExecutionException, InterruptedException, TimeoutException { + assertGlobalCleanupNotTriggered(); + assertLocalCleanupNotTriggered(); + assertJobManagerMetricGroupNotCleaned(); + + final CompletableFuture cleanupResultFuture = + testInstance.createGlobalResourceCleaner().cleanupAsync(JOB_ID); + + assertGlobalCleanupTriggeredWaitingForJobManagerRunnerRegistry(); + assertLocalCleanupNotTriggered(); + assertJobManagerMetricGroupNotCleaned(); + + assertThat(cleanupResultFuture).isNotCompleted(); + + jobManagerRunnerRegistry.completeGlobalCleanup(); + + assertGlobalCleanupTriggered(); + assertLocalCleanupNotTriggered(); + assertJobManagerMetricGroupCleaned(); + + assertThat(cleanupResultFuture).isCompleted(); + } + + private void assertLocalCleanupNotTriggered() { + assertThat(jobManagerRunnerRegistry.getLocalCleanupFuture()).isNotDone(); + assertThat(jobGraphWriter.getLocalCleanupFuture()).isNotDone(); + assertThat(blobServer.getLocalCleanupFuture()).isNotDone(); + assertThat(highAvailabilityServices.getLocalCleanupFuture()).isNotDone(); + } + + private void assertLocalCleanupTriggeredWaitingForJobManagerRunnerRegistry() { + assertThat(jobManagerRunnerRegistry.getLocalCleanupFuture()).isDone(); + + // the JobManagerRunnerRegistry needs to be cleaned up first + assertThat(jobGraphWriter.getLocalCleanupFuture()).isNotDone(); + assertThat(blobServer.getLocalCleanupFuture()).isNotDone(); + assertThat(highAvailabilityServices.getLocalCleanupFuture()).isNotDone(); + } + + private void assertGlobalCleanupNotTriggered() { + assertThat(jobManagerRunnerRegistry.getGlobalCleanupFuture()).isNotDone(); + assertThat(jobGraphWriter.getGlobalCleanupFuture()).isNotDone(); + assertThat(blobServer.getGlobalCleanupFuture()).isNotDone(); + assertThat(highAvailabilityServices.getGlobalCleanupFuture()).isNotDone(); + } + + private void assertGlobalCleanupTriggeredWaitingForJobManagerRunnerRegistry() { + assertThat(jobManagerRunnerRegistry.getGlobalCleanupFuture()).isDone(); + + // the JobManagerRunnerRegistry needs to be cleaned up first + assertThat(jobGraphWriter.getGlobalCleanupFuture()).isNotDone(); + assertThat(blobServer.getGlobalCleanupFuture()).isNotDone(); + assertThat(highAvailabilityServices.getGlobalCleanupFuture()).isNotDone(); + } + + private void assertJobManagerMetricGroupNotCleaned() { + assertThat(jobManagerMetricGroup.numRegisteredJobMetricGroups()).isEqualTo(1); + } + + private void assertLocalCleanupTriggered() { + assertThat(jobManagerRunnerRegistry.getLocalCleanupFuture()).isCompleted(); + assertThat(jobGraphWriter.getLocalCleanupFuture()).isCompleted(); + assertThat(blobServer.getLocalCleanupFuture()).isCompleted(); + assertThat(highAvailabilityServices.getLocalCleanupFuture()).isCompleted(); + } + + private void assertGlobalCleanupTriggered() { + assertThat(jobManagerRunnerRegistry.getGlobalCleanupFuture()).isCompleted(); + assertThat(jobGraphWriter.getGlobalCleanupFuture()).isCompleted(); + assertThat(blobServer.getGlobalCleanupFuture()).isCompleted(); + assertThat(highAvailabilityServices.getGlobalCleanupFuture()).isCompleted(); + } + + private void assertJobManagerMetricGroupCleaned() { + assertThat(jobManagerMetricGroup.numRegisteredJobMetricGroups()).isEqualTo(0); + } + + private static class AbstractTestingCleanableResource + implements LocallyCleanableResource, GloballyCleanableResource { + + private final CompletableFuture localCleanupFuture = new CompletableFuture<>(); + private final CompletableFuture globalCleanupFuture = new CompletableFuture<>(); + + @Override + public void globalCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous globalCleanup is not supported."); + } + + @Override + public void localCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous localCleanup is not supported."); + } + + @Override + public CompletableFuture localCleanupAsync(JobID jobId, Executor ignoredExecutor) { + localCleanupFuture.complete(jobId); + + return FutureUtils.completedVoidFuture(); + } + + @Override + public CompletableFuture globalCleanupAsync(JobID jobId, Executor ignoredExecutor) { + globalCleanupFuture.complete(jobId); + + return FutureUtils.completedVoidFuture(); + } + + public CompletableFuture getLocalCleanupFuture() { + return localCleanupFuture; + } + + public CompletableFuture getGlobalCleanupFuture() { + return globalCleanupFuture; + } + } + + private static class CleanableJobGraphWriter extends AbstractTestingCleanableResource + implements JobGraphWriter { + + @Override + public void putJobGraph(JobGraph jobGraph) { + throw new UnsupportedOperationException("putJobGraph operation not supported."); + } + } + + private static class CleanableHighAvailabilityServices extends AbstractTestingCleanableResource + implements HighAvailabilityServices { + + @Override + public LeaderRetrievalService getResourceManagerLeaderRetriever() { + throw new UnsupportedOperationException( + "getResourceManagerLeaderRetriever operation not supported."); + } + + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + throw new UnsupportedOperationException( + "getDispatcherLeaderRetriever operation not supported."); + } + + @Override + public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { + throw new UnsupportedOperationException( + "getJobManagerLeaderRetriever operation not supported."); + } + + @Override + public LeaderRetrievalService getJobManagerLeaderRetriever( + JobID jobID, String defaultJobManagerAddress) { + throw new UnsupportedOperationException( + "getJobManagerLeaderRetriever operation not supported."); + } + + @Override + public LeaderElectionService getResourceManagerLeaderElectionService() { + throw new UnsupportedOperationException( + "getResourceManagerLeaderElectionService operation not supported."); + } + + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + throw new UnsupportedOperationException( + "getDispatcherLeaderElectionService operation not supported."); + } + + @Override + public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { + throw new UnsupportedOperationException( + "getJobManagerLeaderElectionService operation not supported."); + } + + @Override + public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception { + throw new UnsupportedOperationException( + "getCheckpointRecoveryFactory operation not supported."); + } + + @Override + public JobGraphStore getJobGraphStore() throws Exception { + throw new UnsupportedOperationException("getJobGraphStore operation not supported."); + } + + @Override + public JobResultStore getJobResultStore() throws Exception { + throw new UnsupportedOperationException("getJobResultStore operation not supported."); + } + + @Override + public BlobStore createBlobStore() throws IOException { + throw new UnsupportedOperationException("createBlobStore operation not supported."); + } + + @Override + public void close() throws Exception { + throw new UnsupportedOperationException("close operation not supported."); + } + + @Override + public void closeAndCleanupAllData() throws Exception { + throw new UnsupportedOperationException( + "closeAndCleanupAllData operation not supported."); + } + } + + private static class CleanableBlobServer extends BlobServer { + + private final CompletableFuture localCleanupFuture = new CompletableFuture<>(); + private final CompletableFuture globalCleanupFuture = new CompletableFuture<>(); + + public CleanableBlobServer() throws IOException { + super( + new Configuration(), + new File("non-existent-file"), + new TestingBlobStoreBuilder().createTestingBlobStore()); + } + + @Override + public void globalCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous globalCleanup is not supported."); + } + + @Override + public void localCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous localCleanup is not supported."); + } + + @Override + public CompletableFuture localCleanupAsync(JobID jobId, Executor ignoredExecutor) { + localCleanupFuture.complete(jobId); + + return FutureUtils.completedVoidFuture(); + } + + @Override + public CompletableFuture globalCleanupAsync(JobID jobId, Executor ignoredExecutor) { + globalCleanupFuture.complete(jobId); + + return FutureUtils.completedVoidFuture(); + } + + public CompletableFuture getLocalCleanupFuture() { + return localCleanupFuture; + } + + public CompletableFuture getGlobalCleanupFuture() { + return globalCleanupFuture; + } + } + + private static class CleanableJobManagerRegistry extends JobManagerRunnerRegistry { + + private final CompletableFuture localCleanupFuture = new CompletableFuture<>(); + private final CompletableFuture globalCleanupFuture = new CompletableFuture<>(); + + private final CompletableFuture localCleanupResultFuture = new CompletableFuture<>(); + private final CompletableFuture globalCleanupResultFuture = new CompletableFuture<>(); + + public CleanableJobManagerRegistry() { + super(1); + } + + @Override + public void globalCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous globalCleanup is not supported."); + } + + @Override + public void localCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous localCleanup is not supported."); + } + + @Override + public CompletableFuture localCleanupAsync(JobID jobId, Executor ignoredExecutor) { + localCleanupFuture.complete(jobId); + + return localCleanupResultFuture; + } + + @Override + public CompletableFuture globalCleanupAsync(JobID jobId, Executor ignoredExecutor) { + globalCleanupFuture.complete(jobId); + + return globalCleanupResultFuture; + } + + public CompletableFuture getLocalCleanupFuture() { + return localCleanupFuture; + } + + public CompletableFuture getGlobalCleanupFuture() { + return globalCleanupFuture; + } + + public void completeLocalCleanup() { + localCleanupResultFuture.complete(null); + } + + public void completeGlobalCleanup() { + globalCleanupResultFuture.complete(null); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java new file mode 100644 index 0000000000000..9d648dccfe46d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java @@ -0,0 +1,89 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.ArrayList; +import java.util.Collection; + +/** {@code TestingResourceCleanerFactory} for adding custom {@link ResourceCleaner} creation. */ +public class TestingResourceCleanerFactory implements ResourceCleanerFactory { + + private final Collection locallyCleanableResources = + new ArrayList<>(); + private final Collection globallyCleanableResources = + new ArrayList<>(); + + public + TestingResourceCleanerFactory with(T instance) { + return withLocallyCleanableResource(instance).withGloballyCleanableResource(instance); + } + + public TestingResourceCleanerFactory withLocallyCleanableResource( + LocallyCleanableResource locallyCleanableResource) { + this.locallyCleanableResources.add(locallyCleanableResource); + + return this; + } + + public TestingResourceCleanerFactory withGloballyCleanableResource( + GloballyCleanableResource globallyCleanableResource) { + this.globallyCleanableResources.add(globallyCleanableResource); + + return this; + } + + @Override + public ResourceCleaner createLocalResourceCleaner() { + return jobId -> { + Throwable t = null; + for (LocallyCleanableResource locallyCleanableResource : locallyCleanableResources) { + try { + locallyCleanableResource.localCleanup(jobId); + } catch (Throwable throwable) { + t = ExceptionUtils.firstOrSuppressed(throwable, t); + } + } + + return t != null + ? FutureUtils.completedExceptionally(t) + : FutureUtils.completedVoidFuture(); + }; + } + + @Override + public ResourceCleaner createGlobalResourceCleaner() { + return jobId -> { + Throwable t = null; + for (GloballyCleanableResource globallyCleanableResource : globallyCleanableResources) { + try { + globallyCleanableResource.globalCleanup(jobId); + } catch (Throwable throwable) { + t = ExceptionUtils.firstOrSuppressed(throwable, t); + } + } + + return t != null + ? FutureUtils.completedExceptionally(t) + : FutureUtils.completedVoidFuture(); + }; + } +} From ecb5d0e980479e8ee32b71903b1594b61eeb3cce Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Sat, 22 Jan 2022 16:58:52 +0100 Subject: [PATCH 35/49] [hotfix] Migrates BlobServerCleanupTest to JUnit5/AssertJ --- .../runtime/blob/BlobServerCleanupTest.java | 52 ++++++++++--------- 1 file changed, 27 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java index be0ba5d72d3af..45ff27781c1f0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java @@ -28,9 +28,8 @@ import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import javax.annotation.Nullable; @@ -59,14 +58,20 @@ import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.Assert.assertEquals; /** A few tests for the cleanup of transient BLOBs at the {@link BlobServer}. */ public class BlobServerCleanupTest extends TestLogger { - private final Random rnd = new Random(); + private static final Random RANDOM = new Random(); - @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @TempDir private File temporaryFolder; + + private static byte[] createRandomData() { + final byte[] randomData = new byte[2000000]; + RANDOM.nextBytes(randomData); + + return randomData; + } @Test public void testTransientBlobNoJobCleanup() @@ -94,9 +99,8 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) final List> getOperations = new ArrayList<>(numberConcurrentGetOperations); - byte[] data = new byte[2000000]; - rnd.nextBytes(data); - byte[] data2 = Arrays.copyOfRange(data, 10, 54); + byte[] data = createRandomData(); + byte[] data2 = createRandomData(); Configuration config = new Configuration(); config.setLong(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); @@ -104,7 +108,7 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) long cleanupLowerBound; try (BlobServer server = - new BlobServer(config, temporaryFolder.newFolder(), new VoidBlobStore())) { + new BlobServer(config, temporaryFolder, new VoidBlobStore())) { ConcurrentMap, Long> transientBlobExpiryTimes = server.getBlobExpiryTimes(); @@ -136,14 +140,16 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) final Long key1ExpiryAfterGet = transientBlobExpiryTimes.get(Tuple2.of(jobId, key1)); assertThat(key1ExpiryAfterGet).isGreaterThan(key1ExpiryAfterPut); assertThat(key1ExpiryAfterGet).isGreaterThanOrEqualTo(cleanupLowerBound); - assertEquals(key2ExpiryAfterPut, transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))); + assertThat(key2ExpiryAfterPut) + .isEqualTo(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))); // access key2, verify expiry times (delay at least 1ms to also verify key1 expiry is // unchanged) Thread.sleep(1); cleanupLowerBound = System.currentTimeMillis() + cleanupInterval; verifyContents(server, jobId, key2, data2); - assertEquals(key1ExpiryAfterGet, transientBlobExpiryTimes.get(Tuple2.of(jobId, key1))); + assertThat(key1ExpiryAfterGet) + .isEqualTo(transientBlobExpiryTimes.get(Tuple2.of(jobId, key1))); assertThat(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))) .isGreaterThan(key2ExpiryAfterPut); assertThat(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))) @@ -203,15 +209,14 @@ private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) final long cleanupInterval = 1L; final Configuration configuration = new Configuration(); configuration.set(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); - final File storageDirectory = temporaryFolder.newFolder(); final TransientBlobKey transientBlobKey = TestingBlobUtils.writeTransientBlob( - storageDirectory.toPath(), jobId, new byte[] {1, 2, 3, 4}); - final File blob = BlobUtils.getStorageLocation(storageDirectory, jobId, transientBlobKey); + temporaryFolder.toPath(), jobId, new byte[] {1, 2, 3, 4}); + final File blob = BlobUtils.getStorageLocation(temporaryFolder, jobId, transientBlobKey); try (final BlobServer blobServer = - new BlobServer(configuration, storageDirectory, new VoidBlobStore())) { + new BlobServer(configuration, temporaryFolder, new VoidBlobStore())) { CommonTestUtils.waitUntilCondition( () -> !blob.exists(), Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L)), @@ -221,19 +226,17 @@ private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) @Test public void testBlobServerRetainsJobs() throws Exception { - final File storageDirectory = temporaryFolder.newFolder(); - final JobID jobId1 = new JobID(); final JobID jobId2 = new JobID(); final byte[] fileContent = {1, 2, 3, 4}; final PermanentBlobKey blobKey1 = - TestingBlobUtils.writePermanentBlob(storageDirectory.toPath(), jobId1, fileContent); + TestingBlobUtils.writePermanentBlob(temporaryFolder.toPath(), jobId1, fileContent); final PermanentBlobKey blobKey2 = - TestingBlobUtils.writePermanentBlob(storageDirectory.toPath(), jobId2, fileContent); + TestingBlobUtils.writePermanentBlob(temporaryFolder.toPath(), jobId2, fileContent); try (final BlobServer blobServer = - new BlobServer(new Configuration(), storageDirectory, new VoidBlobStore())) { + new BlobServer(new Configuration(), temporaryFolder, new VoidBlobStore())) { blobServer.retainJobs(Collections.singleton(jobId1)); assertThat(blobServer.getFile(jobId1, blobKey1)).hasBinaryContent(fileContent); @@ -314,10 +317,9 @@ public static void checkFileCountForJob( throw new IOException("File " + jobDir + " does not exist."); } } else { - assertEquals( - "Too many/few files in job dir: " + Arrays.asList(blobsForJob).toString(), - expectedCount, - blobsForJob.length); + assertThat(blobsForJob.length) + .as("Too many/few files in job dir: " + Arrays.asList(blobsForJob)) + .isEqualTo(expectedCount); } } } From 836f747cc6aed77562a4f6c9bff9ea1dce36ca4d Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 13 Jan 2022 13:40:59 +0100 Subject: [PATCH 36/49] [FLINK-25432][runtime] Makes TestingCompletedCheckpointStore more generic We need to extend the TestingCompletedCheckpointStore to enable triggering an Exception in the shutdown process on execution. This selective requirement could have been made with less changes. Instead, I decided to provide the most generic Testing* implementation offering a proper TestingCompletedCheckpointStore implementation which makes it more future-proof. --- ...ultSchedulerCheckpointCoordinatorTest.java | 16 +- .../PerJobCheckpointRecoveryTest.java | 4 +- .../TestingCompletedCheckpointStore.java | 163 ++++++++++++++++-- .../adaptive/AdaptiveSchedulerTest.java | 4 +- 4 files changed, 166 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java index 190487e973c9a..60f2c66b659ae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java @@ -59,7 +59,9 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFailedExecutionG CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); - CompletedCheckpointStore store = new TestingCompletedCheckpointStore(storeShutdownFuture); + CompletedCheckpointStore store = + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints(storeShutdownFuture); final SchedulerBase scheduler = createSchedulerAndEnableCheckpointing(counter, store); final ExecutionGraph graph = scheduler.getExecutionGraph(); @@ -85,7 +87,9 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnSuspendedExecuti CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); - CompletedCheckpointStore store = new TestingCompletedCheckpointStore(storeShutdownFuture); + CompletedCheckpointStore store = + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints(storeShutdownFuture); final SchedulerBase scheduler = createSchedulerAndEnableCheckpointing(counter, store); final ExecutionGraph graph = scheduler.getExecutionGraph(); @@ -111,7 +115,9 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFinishedExecutio CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); - CompletedCheckpointStore store = new TestingCompletedCheckpointStore(storeShutdownFuture); + CompletedCheckpointStore store = + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints(storeShutdownFuture); final SchedulerBase scheduler = createSchedulerAndEnableCheckpointing(counter, store); final ExecutionGraph graph = scheduler.getExecutionGraph(); @@ -146,7 +152,9 @@ public void testClosingSchedulerSuspendsExecutionGraphAndShutsDownCheckpointCoor CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); - CompletedCheckpointStore store = new TestingCompletedCheckpointStore(storeShutdownFuture); + CompletedCheckpointStore store = + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints(storeShutdownFuture); final SchedulerBase scheduler = createSchedulerAndEnableCheckpointing(counter, store); final ExecutionGraph graph = scheduler.getExecutionGraph(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java index 2fe17366351c9..58a0d954318dd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java @@ -36,7 +36,9 @@ public class PerJobCheckpointRecoveryTest extends TestLogger { @Test public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { final TestingCompletedCheckpointStore store = - new TestingCompletedCheckpointStore(new CompletableFuture<>()); + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + new CompletableFuture<>()); final CheckpointRecoveryFactory factory = PerJobCheckpointRecoveryFactory.withoutCheckpointStoreRecovery( maxCheckpoints -> store); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCompletedCheckpointStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCompletedCheckpointStore.java index dc9a1922a3cc4..1d474a54b3cce 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCompletedCheckpointStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCompletedCheckpointStore.java @@ -19,18 +19,54 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.util.function.TriFunction; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; +import java.util.function.Supplier; /** Test {@link CompletedCheckpointStore} implementation for testing the shutdown behavior. */ public final class TestingCompletedCheckpointStore implements CompletedCheckpointStore { - private final CompletableFuture shutdownStatus; + private final TriFunction< + CompletedCheckpoint, CheckpointsCleaner, Runnable, CompletedCheckpoint> + addCheckpointAndSubsumeOldestOneFunction; + private final BiConsumer shutdownConsumer; + private final Supplier> getAllCheckpointsSupplier; + private final Supplier getNumberOfRetainedCheckpointsSuppler; + private final Supplier getMaxNumberOfRetainedCheckpointsSupplier; + private final Supplier requiresExternalizedCheckpointsSupplier; + private final Supplier getSharedStateRegistrySupplier; - public TestingCompletedCheckpointStore(CompletableFuture shutdownStatus) { - this.shutdownStatus = shutdownStatus; + public static TestingCompletedCheckpointStore + createStoreWithShutdownCheckAndNoCompletedCheckpoints( + CompletableFuture shutdownFuture) { + return TestingCompletedCheckpointStore.builder() + .withShutdownConsumer( + ((jobStatus, ignoredCheckpointsCleaner) -> + shutdownFuture.complete(jobStatus))) + .withGetAllCheckpointsSupplier(Collections::emptyList) + .build(); + } + + private TestingCompletedCheckpointStore( + TriFunction + addCheckpointAndSubsumeOldestOneFunction, + BiConsumer shutdownConsumer, + Supplier> getAllCheckpointsSupplier, + Supplier getNumberOfRetainedCheckpointsSuppler, + Supplier getMaxNumberOfRetainedCheckpointsSupplier, + Supplier requiresExternalizedCheckpointsSupplier, + Supplier getSharedStateRegistrySupplier) { + this.addCheckpointAndSubsumeOldestOneFunction = addCheckpointAndSubsumeOldestOneFunction; + this.shutdownConsumer = shutdownConsumer; + this.getAllCheckpointsSupplier = getAllCheckpointsSupplier; + this.getNumberOfRetainedCheckpointsSuppler = getNumberOfRetainedCheckpointsSuppler; + this.getMaxNumberOfRetainedCheckpointsSupplier = getMaxNumberOfRetainedCheckpointsSupplier; + this.requiresExternalizedCheckpointsSupplier = requiresExternalizedCheckpointsSupplier; + this.getSharedStateRegistrySupplier = getSharedStateRegistrySupplier; } @Override @@ -38,41 +74,138 @@ public CompletedCheckpoint addCheckpointAndSubsumeOldestOne( CompletedCheckpoint checkpoint, CheckpointsCleaner checkpointsCleaner, Runnable postCleanup) { - throw new UnsupportedOperationException("Not implemented."); - } - - @Override - public CompletedCheckpoint getLatestCheckpoint() { - return null; + return addCheckpointAndSubsumeOldestOneFunction.apply( + checkpoint, checkpointsCleaner, postCleanup); } @Override public void shutdown(JobStatus jobStatus, CheckpointsCleaner checkpointsCleaner) { - shutdownStatus.complete(jobStatus); + shutdownConsumer.accept(jobStatus, checkpointsCleaner); } @Override public List getAllCheckpoints() { - return Collections.emptyList(); + return getAllCheckpointsSupplier.get(); } @Override public int getNumberOfRetainedCheckpoints() { - throw new UnsupportedOperationException("Not implemented."); + return getNumberOfRetainedCheckpointsSuppler.get(); } @Override public int getMaxNumberOfRetainedCheckpoints() { - throw new UnsupportedOperationException("Not implemented."); + return getMaxNumberOfRetainedCheckpointsSupplier.get(); } @Override public boolean requiresExternalizedCheckpoints() { - throw new UnsupportedOperationException("Not implemented."); + return requiresExternalizedCheckpointsSupplier.get(); } @Override public SharedStateRegistry getSharedStateRegistry() { - throw new UnsupportedOperationException("Not implemented."); + return getSharedStateRegistrySupplier.get(); + } + + public static Builder builder() { + return new TestingCompletedCheckpointStore.Builder(); + } + + /** {@code Builder} for creating {@code TestingCompletedCheckpointStore} instances. */ + public static class Builder { + + private TriFunction + addCheckpointAndSubsumeOldestOneFunction = + (ignoredCompletedCheckpoint, + ignoredCheckpointsCleaner, + ignoredPostCleanup) -> { + throw new UnsupportedOperationException( + "addCheckpointAndSubsumeOldestOne is not implemented."); + }; + private BiConsumer shutdownConsumer = + (ignoredJobStatus, ignoredCheckpointsCleaner) -> { + throw new UnsupportedOperationException("shutdown is not implemented."); + }; + private Supplier> getAllCheckpointsSupplier = + () -> { + throw new UnsupportedOperationException( + "getAllCheckpoints is not implemented."); + }; + private Supplier getNumberOfRetainedCheckpointsSuppler = + () -> { + throw new UnsupportedOperationException( + "getNumberOfRetainedCheckpointsis not implemented."); + }; + private Supplier getMaxNumberOfRetainedCheckpointsSupplier = + () -> { + throw new UnsupportedOperationException( + "getMaxNumberOfRetainedCheckpoints is not implemented."); + }; + private Supplier requiresExternalizedCheckpointsSupplier = + () -> { + throw new UnsupportedOperationException( + "requiresExternalizedCheckpoints is not implemented."); + }; + private Supplier getSharedStateRegistrySupplier = + () -> { + throw new UnsupportedOperationException( + "getSharedStateRegistry is not implemented."); + }; + + public Builder withAddCheckpointAndSubsumeOldestOneFunction( + TriFunction + addCheckpointAndSubsumeOldestOneFunction) { + this.addCheckpointAndSubsumeOldestOneFunction = + addCheckpointAndSubsumeOldestOneFunction; + return this; + } + + public Builder withShutdownConsumer( + BiConsumer shutdownConsumer) { + this.shutdownConsumer = shutdownConsumer; + return this; + } + + public Builder withGetAllCheckpointsSupplier( + Supplier> getAllCheckpointsSupplier) { + this.getAllCheckpointsSupplier = getAllCheckpointsSupplier; + return this; + } + + public Builder withGetNumberOfRetainedCheckpointsSupplier( + Supplier getNumberOfRetainedCheckpointsSuppler) { + this.getNumberOfRetainedCheckpointsSuppler = getNumberOfRetainedCheckpointsSuppler; + return this; + } + + public Builder withGetMaxNumberOfRetainedCheckpointsSupplier( + Supplier getMaxNumberOfRetainedCheckpoints) { + this.getMaxNumberOfRetainedCheckpointsSupplier = getMaxNumberOfRetainedCheckpoints; + return this; + } + + public Builder withRequiresExternalizedCheckpointsSupplier( + Supplier requiresExternalizedCheckpointsSupplier) { + this.requiresExternalizedCheckpointsSupplier = requiresExternalizedCheckpointsSupplier; + return this; + } + + public Builder withGetSharedStateRegistrySupplier( + Supplier getSharedStateRegistrySupplier) { + this.getSharedStateRegistrySupplier = getSharedStateRegistrySupplier; + return this; + } + + public TestingCompletedCheckpointStore build() { + return new TestingCompletedCheckpointStore( + addCheckpointAndSubsumeOldestOneFunction, + shutdownConsumer, + getAllCheckpointsSupplier, + getNumberOfRetainedCheckpointsSuppler, + getMaxNumberOfRetainedCheckpointsSupplier, + requiresExternalizedCheckpointsSupplier, + getSharedStateRegistrySupplier); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java index 4badfbd971254..e14aa527f741a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java @@ -806,7 +806,9 @@ public void testCloseShutsDownCheckpointingComponents() throws Exception { final CompletableFuture completedCheckpointStoreShutdownFuture = new CompletableFuture<>(); final CompletedCheckpointStore completedCheckpointStore = - new TestingCompletedCheckpointStore(completedCheckpointStoreShutdownFuture); + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + completedCheckpointStoreShutdownFuture); final CompletableFuture checkpointIdCounterShutdownFuture = new CompletableFuture<>(); From 9e2461a348f96ab4c7b6d9bf79883ea28b43268a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 13 Jan 2022 15:04:34 +0100 Subject: [PATCH 37/49] [FLINK-25432][runtime] Makes TestingCheckpointIDCounter more generic We need to extend the TestingCheckpointIDCounter to enable triggering an Exception in the shutdown process on execution. This selective requirement could have been made with less changes. Instead, I decided to provide the most generic Testing* implementation offering a proper TestingCheckpointIDCounter implementation which makes it more future-proof. --- ...ultSchedulerCheckpointCoordinatorTest.java | 16 +++- .../TestingCheckpointIDCounter.java | 89 +++++++++++++++++-- .../adaptive/AdaptiveSchedulerTest.java | 3 +- 3 files changed, 95 insertions(+), 13 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java index 60f2c66b659ae..7676bf49e0f73 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java @@ -56,7 +56,9 @@ public class DefaultSchedulerCheckpointCoordinatorTest extends TestLogger { public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFailedExecutionGraph() throws Exception { final CompletableFuture counterShutdownFuture = new CompletableFuture<>(); - CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); + CheckpointIDCounter counter = + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); CompletedCheckpointStore store = @@ -84,7 +86,9 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFailedExecutionG public void testClosingSchedulerShutsDownCheckpointCoordinatorOnSuspendedExecutionGraph() throws Exception { final CompletableFuture counterShutdownFuture = new CompletableFuture<>(); - CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); + CheckpointIDCounter counter = + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); CompletedCheckpointStore store = @@ -112,7 +116,9 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnSuspendedExecuti public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFinishedExecutionGraph() throws Exception { final CompletableFuture counterShutdownFuture = new CompletableFuture<>(); - CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); + CheckpointIDCounter counter = + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); CompletedCheckpointStore store = @@ -149,7 +155,9 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFinishedExecutio public void testClosingSchedulerSuspendsExecutionGraphAndShutsDownCheckpointCoordinator() throws Exception { final CompletableFuture counterShutdownFuture = new CompletableFuture<>(); - CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); + CheckpointIDCounter counter = + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); CompletedCheckpointStore store = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java index 79df955d121f1..2365ed59789cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java @@ -20,36 +20,109 @@ import org.apache.flink.api.common.JobStatus; import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import java.util.function.Supplier; /** Test {@link CheckpointIDCounter} implementation for testing the shutdown behavior. */ public final class TestingCheckpointIDCounter implements CheckpointIDCounter { - private final CompletableFuture shutdownStatus; + private final Runnable startRunnable; + private final Consumer shutdownConsumer; + private final Supplier getAndIncrementSupplier; + private final Supplier getSupplier; + private final Consumer setCountConsumer; - public TestingCheckpointIDCounter(CompletableFuture shutdownStatus) { - this.shutdownStatus = shutdownStatus; + public static TestingCheckpointIDCounter createStoreWithShutdownCheckAndNoStartAction( + CompletableFuture shutdownFuture) { + return TestingCheckpointIDCounter.builder() + .withStartRunnable(() -> {}) + .withShutdownConsumer(shutdownFuture::complete) + .build(); + } + + private TestingCheckpointIDCounter( + Runnable startRunnable, + Consumer shutdownConsumer, + Supplier getAndIncrementSupplier, + Supplier getSupplier, + Consumer setCountConsumer) { + this.startRunnable = startRunnable; + this.shutdownConsumer = shutdownConsumer; + this.getAndIncrementSupplier = getAndIncrementSupplier; + this.getSupplier = getSupplier; + this.setCountConsumer = setCountConsumer; } @Override - public void start() {} + public void start() { + startRunnable.run(); + } @Override public void shutdown(JobStatus jobStatus) { - shutdownStatus.complete(jobStatus); + shutdownConsumer.accept(jobStatus); } @Override public long getAndIncrement() { - throw new UnsupportedOperationException("Not implemented."); + return getAndIncrementSupplier.get(); } @Override public long get() { - throw new UnsupportedOperationException("Not implemented."); + return getSupplier.get(); } @Override public void setCount(long newId) { - throw new UnsupportedOperationException("Not implemented."); + setCountConsumer.accept(newId); + } + + public static Builder builder() { + return new Builder(); + } + + /** {@code Builder} for creating {@code TestingCheckpointIDCounter} instances. */ + public static class Builder { + + private Runnable startRunnable; + private Consumer shutdownConsumer; + private Supplier getAndIncrementSupplier; + private Supplier getSupplier; + private Consumer setCountConsumer; + + public Builder withStartRunnable(Runnable startRunnable) { + this.startRunnable = startRunnable; + return this; + } + + public Builder withShutdownConsumer(Consumer shutdownConsumer) { + this.shutdownConsumer = shutdownConsumer; + return this; + } + + public Builder withGetAndIncrementSupplier(Supplier getAndIncrementSupplier) { + this.getAndIncrementSupplier = getAndIncrementSupplier; + return this; + } + + public Builder withGetSupplier(Supplier getSupplier) { + this.getSupplier = getSupplier; + return this; + } + + public Builder withSetCountConsumer(Consumer setCountConsumer) { + this.setCountConsumer = setCountConsumer; + return this; + } + + public TestingCheckpointIDCounter build() { + return new TestingCheckpointIDCounter( + startRunnable, + shutdownConsumer, + getAndIncrementSupplier, + getSupplier, + setCountConsumer); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java index e14aa527f741a..31fba295d58d7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java @@ -813,7 +813,8 @@ public void testCloseShutsDownCheckpointingComponents() throws Exception { final CompletableFuture checkpointIdCounterShutdownFuture = new CompletableFuture<>(); final CheckpointIDCounter checkpointIdCounter = - new TestingCheckpointIDCounter(checkpointIdCounterShutdownFuture); + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + checkpointIdCounterShutdownFuture); final JobGraph jobGraph = createJobGraph(); // checkpointing components are only created if checkpointing is enabled From 183efa39cf47192087d9825185c3235f458e632b Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 9 Dec 2021 17:34:23 +0100 Subject: [PATCH 38/49] [FLINK-25432][runtime] Renames createFromInitializingJob into more generic createSparseArchivedExecutionGraph We will need to create the ArchivedExecutionGraph skeleton also in the cleanup JobManagerRunner. The renaming is necessary to align with the usage of this method. --- .../org/apache/flink/runtime/dispatcher/Dispatcher.java | 2 +- .../runtime/executiongraph/ArchivedExecutionGraph.java | 2 +- .../factories/DefaultJobMasterServiceProcessFactory.java | 2 +- .../flink/runtime/scheduler/adaptive/AdaptiveScheduler.java | 2 +- .../org/apache/flink/runtime/dispatcher/DispatcherTest.java | 6 +++--- .../runtime/executiongraph/ArchivedExecutionGraphTest.java | 4 ++-- .../jobmaster/DefaultJobMasterServiceProcessTest.java | 2 +- .../jobmaster/JobMasterServiceLeadershipRunnerTest.java | 2 +- .../flink/runtime/jobmaster/TestingJobManagerRunner.java | 2 +- .../factories/TestingJobMasterServiceProcessFactory.java | 2 +- .../factories/TestingJobMasterServiceProcessFactoryOld.java | 2 +- .../flink/runtime/scheduler/ExecutionGraphInfoTest.java | 2 +- .../flink/runtime/scheduler/adaptive/CreatedTest.java | 2 +- .../scheduler/adaptive/CreatingExecutionGraphTest.java | 2 +- 14 files changed, 17 insertions(+), 17 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index f03c2692d5545..d1f41b60f1415 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -439,7 +439,7 @@ public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) public CompletableFuture submitFailedJob( JobID jobId, String jobName, Throwable exception) { final ArchivedExecutionGraph archivedExecutionGraph = - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobName, JobStatus.FAILED, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java index a36cdeb35075b..3d594edd5f9fe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java @@ -334,7 +334,7 @@ public static ArchivedExecutionGraph createFrom( * Create a sparse ArchivedExecutionGraph for a job while it is still initializing. Most fields * will be empty, only job status and error-related fields are set. */ - public static ArchivedExecutionGraph createFromInitializingJob( + public static ArchivedExecutionGraph createSparseArchivedExecutionGraph( JobID jobId, String jobName, JobStatus jobStatus, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java index 1c60ccf698fc3..cbd7920e69dec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java @@ -68,7 +68,7 @@ public JobID getJobId() { @Override public ArchivedExecutionGraph createArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobName, jobStatus, cause, checkpointingSettings, initializationTimestamp); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java index d289f7742827c..99410e727326b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java @@ -754,7 +754,7 @@ private VertexParallelism determineParallelism(SlotAllocator slotAllocator) @Override public ArchivedExecutionGraph getArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobInformation.getJobID(), jobInformation.getName(), jobStatus, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index dca0232f604d3..bfac2b58d5724 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -525,7 +525,7 @@ public void testJobManagerRunnerInitializationFailureFailsJob() throws Exception testingJobManagerRunner.completeResultFuture( JobManagerRunnerResult.forInitializationFailure( new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobGraph.getName(), JobStatus.FAILED, @@ -687,7 +687,7 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { testingJobManagerRunner.completeResultFuture( JobManagerRunnerResult.forInitializationFailure( new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobGraph.getName(), JobStatus.FAILED, @@ -1383,7 +1383,7 @@ public TestingJobManagerRunner createJobManagerRunner( CompletableFuture.completedFuture( new ExecutionGraphInfo( ArchivedExecutionGraph - .createFromInitializingJob( + .createSparseArchivedExecutionGraph( jobGraph.getJobID(), jobGraph.getName(), JobStatus.RUNNING, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java index e751f64ad9d6b..a79592795e24e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java @@ -152,7 +152,7 @@ public void testSerialization() throws IOException, ClassNotFoundException { @Test public void testCreateFromInitializingJobForSuspendedJob() { final ArchivedExecutionGraph suspendedExecutionGraph = - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "TestJob", JobStatus.SUSPENDED, @@ -170,7 +170,7 @@ public void testCheckpointSettingsArchiving() { CheckpointCoordinatorConfiguration.builder().build(); final ArchivedExecutionGraph archivedGraph = - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "TestJob", JobStatus.INITIALIZING, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java index db7faa9fe2a2f..9b9fa8908f0ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java @@ -52,7 +52,7 @@ public class DefaultJobMasterServiceProcessTest extends TestLogger { private static final Function failedArchivedExecutionGraphFactory = (throwable -> - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, "test", JobStatus.FAILED, throwable, null, 1337)); @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java index 0b2782cfcb9ec..388264f6cb4d4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java @@ -259,7 +259,7 @@ public void testJobMasterCreationFailureCompletesJobManagerRunnerWithInitializat @Nonnull private ExecutionGraphInfo createFailedExecutionGraphInfo(FlinkException testException) { return new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobGraph.getJobID(), jobGraph.getName(), JobStatus.FAILED, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java index 7b6adf544ab65..dd049e053719e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java @@ -69,7 +69,7 @@ private TestingJobManagerRunner( final ExecutionGraphInfo suspendedExecutionGraphInfo = new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, "TestJob", JobStatus.SUSPENDED, null, null, 0L), null); terminationFuture.whenComplete( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java index 379c747d5eeb3..40aa709bf57f2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java @@ -64,7 +64,7 @@ public JobID getJobId() { @Override public ArchivedExecutionGraph createArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobName, jobStatus, cause, null, initializationTimestamp); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java index 986b1bd9be05d..2f316be42b6fd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java @@ -72,7 +72,7 @@ public JobID getJobId() { @Override public ArchivedExecutionGraph createArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, "test-job", jobStatus, cause, null, System.currentTimeMillis()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfoTest.java index ea5cb9d6a7720..e94aea2f763d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfoTest.java @@ -36,7 +36,7 @@ public class ExecutionGraphInfoTest { @Test public void testExecutionGraphHistoryBeingDerivedFromFailedExecutionGraph() { final ArchivedExecutionGraph executionGraph = - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "test job name", JobStatus.FAILED, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatedTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatedTest.java index 3709149bb649b..6887308f01b9a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatedTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatedTest.java @@ -117,7 +117,7 @@ public void goToFinished(ArchivedExecutionGraph archivedExecutionGraph) { @Override public ArchivedExecutionGraph getArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "testJob", jobStatus, cause, null, 0L); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatingExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatingExecutionGraphTest.java index 12533aa1c71b1..edf7bbc77e847 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatingExecutionGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatingExecutionGraphTest.java @@ -201,7 +201,7 @@ public void goToExecuting(ExecutionGraph executionGraph) { @Override public ArchivedExecutionGraph getArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "testJob", jobStatus, cause, null, 0L); } From 65a96bc3fcb933fbef75720ea84d92078c655cbe Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 26 Nov 2021 17:46:16 +0100 Subject: [PATCH 39/49] [FLINK-25432][runtime] Moves maximum retained checkpoints parameter extraction into utility method The number of retained checkpoints needs to be extracted in the Scheduler and the cleanup process. Therefore, the extraction logic is moved from SchedulerUtils into DefaultCompletedCheckpointStoreUtils. Additionally, DefaultCompletedCheckpointStoreUtils is renamed into CompletedCheckpointStoreUtils since its utility methods are related to the interface, not the implementation. --- .../kubernetes/utils/KubernetesUtils.java | 4 +- ...ava => CompletedCheckpointStoreUtils.java} | 38 +++++++++++++++++-- .../runtime/scheduler/SchedulerUtils.java | 21 ++-------- .../flink/runtime/util/ZooKeeperUtils.java | 4 +- ...=> CompletedCheckpointStoreUtilsTest.java} | 25 ++++++++++-- .../DefaultCompletedCheckpointStoreTest.java | 4 +- ...oKeeperCompletedCheckpointStoreITCase.java | 2 +- ...ZooKeeperCompletedCheckpointStoreTest.java | 2 +- 8 files changed, 66 insertions(+), 34 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/{DefaultCompletedCheckpointStoreUtils.java => CompletedCheckpointStoreUtils.java} (74%) rename flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/{DefaultCompletedCheckpointStoreUtilsTest.java => CompletedCheckpointStoreUtilsTest.java} (82%) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index e02d3a824e2f7..36982c7efe8a7 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -33,8 +33,8 @@ import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStoreUtils; import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStore; -import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStoreUtils; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.DefaultJobGraphStore; @@ -323,7 +323,7 @@ public static CompletedCheckpointStore createCompletedCheckpointStore( k -> k.startsWith(CHECKPOINT_ID_KEY_PREFIX), lockIdentity); Collection checkpoints = - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, KubernetesCheckpointStoreUtil.INSTANCE); return new DefaultCompletedCheckpointStore<>( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtils.java similarity index 74% rename from flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtils.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtils.java index 0c9d54782f2a9..b9ba3c9bb3729 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtils.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.persistence.ResourceVersion; import org.apache.flink.runtime.persistence.StateHandleStore; import org.apache.flink.runtime.state.RetrievableStateHandle; @@ -37,15 +39,43 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** Helper methods related to {@link DefaultCompletedCheckpointStore}. */ -public class DefaultCompletedCheckpointStoreUtils { +public class CompletedCheckpointStoreUtils { - private static final Logger LOG = - LoggerFactory.getLogger(DefaultCompletedCheckpointStoreUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(CompletedCheckpointStoreUtils.class); - private DefaultCompletedCheckpointStoreUtils() { + private CompletedCheckpointStoreUtils() { // No-op. } + /** + * Extracts maximum number of retained checkpoints configuration from the passed {@link + * Configuration}. The default value is used as a fallback if the passed value is a value larger + * than {@code 0}. + * + * @param config The configuration that is accessed. + * @param logger The {@link Logger} used for exposing the warning if the configured value is + * invalid. + * @return The maximum number of retained checkpoints based on the passed {@code Configuration}. + */ + public static int getMaximumNumberOfRetainedCheckpoints(Configuration config, Logger logger) { + final int maxNumberOfCheckpointsToRetain = + config.getInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS); + + if (maxNumberOfCheckpointsToRetain <= 0) { + // warning and use 1 as the default value if the setting in + // state.checkpoints.max-retained-checkpoints is not greater than 0. + logger.warn( + "The setting for '{} : {}' is invalid. Using default value of {}", + CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.key(), + maxNumberOfCheckpointsToRetain, + CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue()); + + return CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue(); + } + + return maxNumberOfCheckpointsToRetain; + } + /** * Fetch all {@link CompletedCheckpoint completed checkpoints} from an {@link StateHandleStore * external store}. This method is intended for retrieving an initial state of {@link diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java index ab807ecba375e..f99c51f4970b3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java @@ -20,11 +20,11 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStoreUtils; import org.apache.flink.runtime.checkpoint.DeactivatedCheckpointCompletedCheckpointStore; import org.apache.flink.runtime.checkpoint.DeactivatedCheckpointIDCounter; import org.apache.flink.runtime.client.JobExecutionException; @@ -75,25 +75,10 @@ static CompletedCheckpointStore createCompletedCheckpointStore( Logger log, JobID jobId) throws Exception { - int maxNumberOfCheckpointsToRetain = - jobManagerConfig.getInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS); - - if (maxNumberOfCheckpointsToRetain <= 0) { - // warning and use 1 as the default value if the setting in - // state.checkpoints.max-retained-checkpoints is not greater than 0. - log.warn( - "The setting for '{} : {}' is invalid. Using default value of {}", - CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.key(), - maxNumberOfCheckpointsToRetain, - CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue()); - - maxNumberOfCheckpointsToRetain = - CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue(); - } - return recoveryFactory.createRecoveredCompletedCheckpointStore( jobId, - maxNumberOfCheckpointsToRetain, + CompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( + jobManagerConfig, log), SharedStateRegistry.DEFAULT_FACTORY, ioExecutor); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java index 1d8e4916667c1..d44b385ff68a9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java @@ -26,8 +26,8 @@ import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStoreUtils; import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStore; -import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStoreUtils; import org.apache.flink.runtime.checkpoint.DefaultLastStateConnectionStateListener; import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter; import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointStoreUtil; @@ -589,7 +589,7 @@ public static CompletedCheckpointStore createCompletedCheckpoints( final ZooKeeperStateHandleStore completedCheckpointStateHandleStore = createZooKeeperStateHandleStore(client, getCheckpointsPath(), stateStorage); Collection completedCheckpoints = - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( completedCheckpointStateHandleStore, ZooKeeperCheckpointStoreUtil.INSTANCE); final CompletedCheckpointStore zooKeeperCompletedCheckpointStore = new DefaultCompletedCheckpointStore<>( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtilsTest.java similarity index 82% rename from flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtilsTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtilsTest.java index 984f1059dc0be..77481aed1ff4d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtilsTest.java @@ -20,6 +20,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.persistence.TestingStateHandleStore; import org.apache.flink.runtime.state.RetrievableStateHandle; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; @@ -27,6 +29,8 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import java.io.IOException; import java.io.Serializable; @@ -38,11 +42,12 @@ import java.util.List; import java.util.stream.Collectors; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; -/** Tests related to {@link DefaultCompletedCheckpointStoreUtils}. */ -public class DefaultCompletedCheckpointStoreUtilsTest extends TestLogger { +/** Tests related to {@link CompletedCheckpointStoreUtils}. */ +public class CompletedCheckpointStoreUtilsTest extends TestLogger { private static CompletedCheckpoint createCompletedCheckpoint(long checkpointId) { return new CompletedCheckpoint( @@ -105,7 +110,7 @@ public void testRetrievedCheckpointsAreOrderedChronologically() throws Exception .setGetAllSupplier(() -> handles) .build(); final Collection completedCheckpoints = - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, new SimpleCheckpointStoreUtil()); // Make sure checkpoints are ordered from earliest to latest. assertEquals( @@ -132,7 +137,19 @@ public void testRetrievingCheckpointsFailsIfRetrievalOfAnyCheckpointFails() thro assertThrows( FlinkException.class, () -> - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, new SimpleCheckpointStoreUtil())); } + + @ParameterizedTest(name = "actual: {0}; expected: {1}") + @CsvSource({"10,10", "0,1", "-1,1"}) + public void testGetMaximumNumberOfRetainedCheckpoints(int actualValue, int expectedValue) { + final Configuration jobManagerConfig = new Configuration(); + jobManagerConfig.setInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS, actualValue); + + assertThat( + CompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( + jobManagerConfig, log)) + .isEqualTo(expectedValue); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java index d7ea8804584f4..0447cc172b1d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java @@ -139,7 +139,7 @@ private void testCheckpointRetention( /** * We have three completed checkpoints(1, 2, 3) in the state handle store. We expect that {@link - * DefaultCompletedCheckpointStoreUtils#retrieveCompletedCheckpoints(StateHandleStore, + * CompletedCheckpointStoreUtils#retrieveCompletedCheckpoints(StateHandleStore, * CheckpointStoreUtil)} should recover the sorted checkpoints by name. */ @Test @@ -390,7 +390,7 @@ public long nameToCheckpointID(String name) { toRetain, stateHandleStore, checkpointStoreUtil, - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, checkpointStoreUtil), SharedStateRegistry.DEFAULT_FACTORY.create( org.apache.flink.util.concurrent.Executors.directExecutor(), emptyList()), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java index 65a435b6fc8cd..2deea44ff591e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java @@ -89,7 +89,7 @@ protected CompletedCheckpointStore createRecoveredCompletedCheckpointStore( maxNumberOfCheckpointsToRetain, checkpointsInZooKeeper, checkpointStoreUtil, - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( checkpointsInZooKeeper, checkpointStoreUtil), SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()), executor); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java index e56f0a38b563c..3aa7c2ddc4d84 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java @@ -104,7 +104,7 @@ public void testRecoverFailsIfDownloadFails() { assertThrows( Exception.class, () -> - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( checkpointsInZooKeeper, zooKeeperCheckpointStoreUtil)); assertThat(exception, FlinkMatchers.containsCause(ExpectedTestException.class)); } From 129f3c78ece5aa463e2b5e846d817ee712ab8156 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 29 Nov 2021 15:54:39 +0100 Subject: [PATCH 40/49] [FLINK-25432][runtime] Adds mapping from ApplicationStatus to JobStatus We need to provide a reverse search to retrieve the JobStatus from the ApplicationStatus that is provided by the JobResult. This change is straight-forward because we can expect jobs being finished that end up in the JobResultStore. Therefore, we only have to provide a transition from ApplicationStatus to JobStatus for values we there's a symmetric mapping possible. --- .../clusterframework/ApplicationStatus.java | 54 +++++++++------ .../ApplicationStatusTest.java | 65 +++++++++++++++++++ 2 files changed, 99 insertions(+), 20 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java index 16606966938e8..2636ee6b7906e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java @@ -20,24 +20,37 @@ import org.apache.flink.api.common.JobStatus; +import org.apache.flink.shaded.guava30.com.google.common.collect.BiMap; +import org.apache.flink.shaded.guava30.com.google.common.collect.EnumBiMap; + /** The status of an application. */ public enum ApplicationStatus { - /** Application finished successfully */ + /** Application finished successfully. */ SUCCEEDED(0), - /** Application encountered an unrecoverable failure or error */ + /** Application encountered an unrecoverable failure or error. */ FAILED(1443), - /** Application was canceled or killed on request */ + /** Application was canceled or killed on request. */ CANCELED(0), - /** Application status is not known */ + /** Application status is not known. */ UNKNOWN(1445); // ------------------------------------------------------------------------ - /** The associated process exit code */ + private static final BiMap JOB_STATUS_APPLICATION_STATUS_BI_MAP = + EnumBiMap.create(JobStatus.class, ApplicationStatus.class); + + static { + // only globally-terminated JobStatus have a corresponding ApplicationStatus + JOB_STATUS_APPLICATION_STATUS_BI_MAP.put(JobStatus.FAILED, ApplicationStatus.FAILED); + JOB_STATUS_APPLICATION_STATUS_BI_MAP.put(JobStatus.CANCELED, ApplicationStatus.CANCELED); + JOB_STATUS_APPLICATION_STATUS_BI_MAP.put(JobStatus.FINISHED, ApplicationStatus.SUCCEEDED); + } + + /** The associated process exit code. */ private final int processExitCode; ApplicationStatus(int exitCode) { @@ -45,7 +58,7 @@ public enum ApplicationStatus { } /** - * Gets the process exit code associated with this status + * Gets the process exit code associated with this status. * * @return The associated process exit code. */ @@ -59,20 +72,21 @@ public int processExitCode() { * #UNKNOWN}. */ public static ApplicationStatus fromJobStatus(JobStatus jobStatus) { - if (jobStatus == null) { - return UNKNOWN; - } else { - switch (jobStatus) { - case FAILED: - return FAILED; - case CANCELED: - return CANCELED; - case FINISHED: - return SUCCEEDED; - - default: - return UNKNOWN; - } + return JOB_STATUS_APPLICATION_STATUS_BI_MAP.getOrDefault(jobStatus, UNKNOWN); + } + + /** + * Derives the {@link JobStatus} from the {@code ApplicationStatus}. + * + * @return The corresponding {@code JobStatus}. + * @throws UnsupportedOperationException for {@link #UNKNOWN}. + */ + public JobStatus deriveJobStatus() { + if (!JOB_STATUS_APPLICATION_STATUS_BI_MAP.inverse().containsKey(this)) { + throw new UnsupportedOperationException( + this.name() + " cannot be mapped to a JobStatus."); } + + return JOB_STATUS_APPLICATION_STATUS_BI_MAP.inverse().get(this); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java index 24f1f70f0ffbd..6cc225d0001c5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java @@ -18,16 +18,20 @@ package org.apache.flink.runtime.clusterframework; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.util.Arrays; import java.util.stream.Collectors; import java.util.stream.StreamSupport; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link ApplicationStatus}. */ @ExtendWith(TestLoggerExtension.class) @@ -53,6 +57,67 @@ public void notSucceededNorCancelledStatusMapsToNonSuccessExitCode() { assertThat(exitCodes).doesNotContain(SUCCESS_EXIT_CODE); } + @Test + public void testJobStatusFromSuccessApplicationStatus() { + assertThat(ApplicationStatus.SUCCEEDED.deriveJobStatus()).isEqualTo(JobStatus.FINISHED); + } + + @Test + public void testJobStatusFromFailedApplicationStatus() { + assertThat(ApplicationStatus.FAILED.deriveJobStatus()).isEqualTo(JobStatus.FAILED); + } + + @Test + public void testJobStatusFromCancelledApplicationStatus() { + assertThat(ApplicationStatus.CANCELED.deriveJobStatus()).isEqualTo(JobStatus.CANCELED); + } + + @Test + public void testJobStatusFailsFromUnknownApplicationStatuses() { + assertThatThrownBy(ApplicationStatus.UNKNOWN::deriveJobStatus) + .isInstanceOf(UnsupportedOperationException.class); + } + + @Test + public void testSuccessApplicationStatusFromJobStatus() { + assertThat(ApplicationStatus.fromJobStatus(JobStatus.FINISHED)) + .isEqualTo(ApplicationStatus.SUCCEEDED); + } + + @Test + public void testFailedApplicationStatusFromJobStatus() { + assertThat(ApplicationStatus.fromJobStatus(JobStatus.FAILED)) + .isEqualTo(ApplicationStatus.FAILED); + } + + @Test + public void testCancelledApplicationStatusFromJobStatus() { + assertThat(ApplicationStatus.fromJobStatus(JobStatus.CANCELED)) + .isEqualTo(ApplicationStatus.CANCELED); + } + + @ParameterizedTest + @EnumSource( + value = JobStatus.class, + names = { + "INITIALIZING", + "CREATED", + "RUNNING", + "FAILING", + "CANCELLING", + "RESTARTING", + "SUSPENDED", + "RECONCILING" + }) + public void testUnknownApplicationStatusFromJobStatus(JobStatus jobStatus) { + assertThat(ApplicationStatus.fromJobStatus(jobStatus)).isEqualTo(ApplicationStatus.UNKNOWN); + } + + @Test + public void testUnknownApplicationStatusForMissingJobStatus() { + assertThat(ApplicationStatus.fromJobStatus(null)).isEqualTo(ApplicationStatus.UNKNOWN); + } + private static Iterable exitCodes(Iterable statuses) { return StreamSupport.stream(statuses.spliterator(), false) .map(ApplicationStatus::processExitCode) From 64ef0527b4d8014af471d57655fb611cd78f49f1 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 28 Jan 2022 14:32:00 +0100 Subject: [PATCH 41/49] [FLINK-25432][runtime] Introduces TestingDispatcher.Builder The intention is to unify the TestingDispatcherBuilder logic used in AbstractDispatcherTest and DispatcherResourceCleaner DispatcherResourceCleanupTest --- .../flink/runtime/dispatcher/Dispatcher.java | 4 +- .../DispatcherResourceCleanerFactory.java | 2 +- .../dispatcher/AbstractDispatcherTest.java | 137 +-------- .../dispatcher/DispatcherFailoverITCase.java | 12 +- .../DispatcherResourceCleanupTest.java | 48 +-- .../runtime/dispatcher/DispatcherTest.java | 49 ++- .../runtime/dispatcher/TestingDispatcher.java | 281 ++++++++++++++++++ 7 files changed, 339 insertions(+), 194 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index d1f41b60f1415..160646013c6ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.operators.ResourceSpec; @@ -212,7 +213,8 @@ private Dispatcher( new DispatcherResourceCleanerFactory(jobManagerRunnerRegistry, dispatcherServices)); } - private Dispatcher( + @VisibleForTesting + protected Dispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java index 3cfc6ea56b0c2..0584a75a6816e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java @@ -62,7 +62,7 @@ public DispatcherResourceCleanerFactory( } @VisibleForTesting - DispatcherResourceCleanerFactory( + public DispatcherResourceCleanerFactory( Executor cleanupExecutor, JobManagerRunnerRegistry jobManagerRunnerRegistry, JobGraphWriter jobGraphWriter, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java index f277e2b2e1269..e9eb8aa55a43a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java @@ -26,18 +26,10 @@ import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; -import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -53,11 +45,6 @@ import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; -import java.util.Collection; -import java.util.Collections; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ForkJoinPool; - /** Abstract test for the {@link Dispatcher} component. */ public class AbstractDispatcherTest extends TestLogger { @@ -116,6 +103,19 @@ public void setUp() throws Exception { new BlobServer(configuration, temporaryFolder.newFolder(), new VoidBlobStore()); } + protected TestingDispatcher.Builder createTestingDispatcherBuilder() { + return TestingDispatcher.builder() + .withRpcService(rpcService) + .withConfiguration(configuration) + .withHeartbeatServices(heartbeatServices) + .withHighAvailabilityServices(haServices) + .withJobGraphWriter(haServices.getJobGraphStore()) + .withJobResultStore(haServices.getJobResultStore()) + .withJobManagerRunnerFactory(JobMasterServiceLeadershipRunnerFactory.INSTANCE) + .withFatalErrorHandler(testingFatalErrorHandlerResource.getFatalErrorHandler()) + .withBlobServer(blobServer); + } + @After public void tearDown() throws Exception { if (haServices != null) { @@ -129,115 +129,4 @@ public void tearDown() throws Exception { protected BlobServer getBlobServer() { return blobServer; } - - /** A convenient builder for the {@link TestingDispatcher}. */ - public class TestingDispatcherBuilder { - - private Collection initialJobGraphs = Collections.emptyList(); - - private Collection dirtyJobResults = Collections.emptyList(); - - private DispatcherBootstrapFactory dispatcherBootstrapFactory = - (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(); - - private HeartbeatServices heartbeatServices = AbstractDispatcherTest.this.heartbeatServices; - - private HighAvailabilityServices haServices = AbstractDispatcherTest.this.haServices; - - private JobManagerRunnerFactory jobManagerRunnerFactory = - JobMasterServiceLeadershipRunnerFactory.INSTANCE; - - private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; - - private JobResultStore jobResultStore = new EmbeddedJobResultStore(); - - private FatalErrorHandler fatalErrorHandler = - testingFatalErrorHandlerResource.getFatalErrorHandler(); - - private HistoryServerArchivist historyServerArchivist = VoidHistoryServerArchivist.INSTANCE; - - TestingDispatcherBuilder setHeartbeatServices(HeartbeatServices heartbeatServices) { - this.heartbeatServices = heartbeatServices; - return this; - } - - TestingDispatcherBuilder setHaServices(HighAvailabilityServices haServices) { - this.haServices = haServices; - return this; - } - - TestingDispatcherBuilder setInitialJobGraphs(Collection initialJobGraphs) { - this.initialJobGraphs = initialJobGraphs; - return this; - } - - TestingDispatcherBuilder setDirtyJobResults(Collection dirtyJobResults) { - this.dirtyJobResults = dirtyJobResults; - return this; - } - - TestingDispatcherBuilder setDispatcherBootstrapFactory( - DispatcherBootstrapFactory dispatcherBootstrapFactory) { - this.dispatcherBootstrapFactory = dispatcherBootstrapFactory; - return this; - } - - TestingDispatcherBuilder setJobManagerRunnerFactory( - JobManagerRunnerFactory jobManagerRunnerFactory) { - this.jobManagerRunnerFactory = jobManagerRunnerFactory; - return this; - } - - TestingDispatcherBuilder setJobGraphWriter(JobGraphWriter jobGraphWriter) { - this.jobGraphWriter = jobGraphWriter; - return this; - } - - TestingDispatcherBuilder setJobResultStore(JobResultStore jobResultStore) { - this.jobResultStore = jobResultStore; - return this; - } - - public TestingDispatcherBuilder setFatalErrorHandler(FatalErrorHandler fatalErrorHandler) { - this.fatalErrorHandler = fatalErrorHandler; - return this; - } - - public TestingDispatcherBuilder setHistoryServerArchivist( - HistoryServerArchivist historyServerArchivist) { - this.historyServerArchivist = historyServerArchivist; - return this; - } - - TestingDispatcher build() throws Exception { - TestingResourceManagerGateway resourceManagerGateway = - new TestingResourceManagerGateway(); - - final MemoryExecutionGraphInfoStore executionGraphInfoStore = - new MemoryExecutionGraphInfoStore(); - - return new TestingDispatcher( - rpcService, - DispatcherId.generate(), - initialJobGraphs, - dirtyJobResults, - dispatcherBootstrapFactory, - new DispatcherServices( - configuration, - haServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - executionGraphInfoStore, - fatalErrorHandler, - historyServerArchivist, - null, - new DispatcherOperationCaches(), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - jobGraphWriter, - jobResultStore, - jobManagerRunnerFactory, - ForkJoinPool.commonPool())); - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java index 1875c59bd2e1b..1a796363404ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java @@ -237,14 +237,10 @@ private TestingDispatcher createRecoveredDispatcher( } } final TestingDispatcher dispatcher = - new TestingDispatcherBuilder() - .setJobManagerRunnerFactory( - JobMasterServiceLeadershipRunnerFactory.INSTANCE) - .setJobGraphWriter(haServices.getJobGraphStore()) - .setJobResultStore(haServices.getJobResultStore()) - .setInitialJobGraphs(jobGraphs) - .setDirtyJobResults(haServices.getJobResultStore().getDirtyResults()) - .setFatalErrorHandler( + createTestingDispatcherBuilder() + .withRecoveredJobs(jobGraphs) + .withRecoveredDirtyJobs(haServices.getJobResultStore().getDirtyResults()) + .withFatalErrorHandler( fatalErrorHandler == null ? testingFatalErrorHandlerResource.getFatalErrorHandler() : fatalErrorHandler) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index af47cf438f54b..9f27fb391e5b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -41,15 +41,12 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; -import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -87,7 +84,6 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -139,7 +135,6 @@ public class DispatcherResourceCleanupTest extends TestLogger { private CompletableFuture localCleanupFuture; private CompletableFuture globalCleanupFuture; private CompletableFuture cleanupJobHADataFuture; - private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; @BeforeClass public static void setupClass() { @@ -156,7 +151,6 @@ public void setup() throws Exception { highAvailabilityServices = new TestingHighAvailabilityServices(); clearedJobLatch = new OneShotLatch(); jobResultStore = new SingleJobResultStore(jobId, clearedJobLatch); - highAvailabilityServices.setJobResultStore(jobResultStore); cleanupJobHADataFuture = new CompletableFuture<>(); highAvailabilityServices.setGlobalCleanupFuture(cleanupJobHADataFuture); @@ -197,34 +191,16 @@ private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob( } private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { - TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); - final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); - final MemoryExecutionGraphInfoStore archivedExecutionGraphStore = - new MemoryExecutionGraphInfoStore(); dispatcher = - new TestingDispatcher( - rpcService, - DispatcherId.generate(), - Collections.emptyList(), - Collections.emptyList(), - (dispatcher, scheduledExecutor, errorHandler) -> - new NoOpDispatcherBootstrap(), - new DispatcherServices( - configuration, - highAvailabilityServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - archivedExecutionGraphStore, - testingFatalErrorHandlerResource.getFatalErrorHandler(), - VoidHistoryServerArchivist.INSTANCE, - null, - new DispatcherOperationCaches(), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - jobGraphWriter, - jobResultStore, - jobManagerRunnerFactory, - ForkJoinPool.commonPool())); + TestingDispatcher.builder() + .withRpcService(rpcService) + .withHighAvailabilityServices(highAvailabilityServices) + .withJobResultStore(jobResultStore) + .withBlobServer(blobServer) + .withFatalErrorHandler( + testingFatalErrorHandlerResource.getFatalErrorHandler()) + .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .build(); dispatcher.start(); @@ -236,6 +212,10 @@ public void teardown() throws Exception { if (dispatcher != null) { dispatcher.close(); } + + if (blobServer != null) { + blobServer.close(); + } } @AfterClass @@ -658,7 +638,6 @@ public void testFatalErrorIfJobCannotBeMarkedDirtyInJobResultStore() throws Exce throw new IOException("Expected IOException."); }) .build(); - highAvailabilityServices.setJobResultStore(jobResultStore); final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); @@ -692,7 +671,6 @@ public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws throw new IOException("Expected IOException."); }) .build(); - highAvailabilityServices.setJobResultStore(jobResultStore); final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index bfac2b58d5724..0600c39eff46e 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -176,12 +176,12 @@ private TestingDispatcher createAndStartDispatcher( JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { final TestingDispatcher dispatcher = - new TestingDispatcherBuilder() - .setHaServices(haServices) - .setHeartbeatServices(heartbeatServices) - .setJobManagerRunnerFactory(jobManagerRunnerFactory) - .setJobGraphWriter(haServices.getJobGraphStore()) - .setJobResultStore(haServices.getJobResultStore()) + createTestingDispatcherBuilder() + .withHighAvailabilityServices(haServices) + .withHeartbeatServices(heartbeatServices) + .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .withJobGraphWriter(haServices.getJobGraphStore()) + .withJobResultStore(haServices.getJobResultStore()) .build(); dispatcher.start(); return dispatcher; @@ -244,11 +244,11 @@ public void testDuplicateJobSubmissionWithGloballyTerminatedJobId() throws Excep @Test public void testDuplicateJobSubmissionWithRunningJobId() throws Exception { dispatcher = - new TestingDispatcherBuilder() - .setJobManagerRunnerFactory( + createTestingDispatcherBuilder() + .withJobManagerRunnerFactory( new ExpectedJobIdJobManagerRunnerFactory( jobId, createdJobManagerRunnerLatch)) - .setInitialJobGraphs(Collections.singleton(jobGraph)) + .withRecoveredJobs(Collections.singleton(jobGraph)) .build(); dispatcher.start(); final DispatcherGateway dispatcherGateway = @@ -468,11 +468,11 @@ public void testNoHistoryServerArchiveCreatedForSuspendedJob() throws Exception final CompletableFuture jobTerminationFuture = new CompletableFuture<>(); dispatcher = - new TestingDispatcherBuilder() - .setJobManagerRunnerFactory( + createTestingDispatcherBuilder() + .withJobManagerRunnerFactory( new FinishingJobManagerRunnerFactory( jobTerminationFuture, () -> {})) - .setHistoryServerArchivist( + .withHistoryServerArchivist( executionGraphInfo -> { archiveAttemptFuture.complete(null); return CompletableFuture.completedFuture(null); @@ -669,10 +669,9 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); dispatcher = - new TestingDispatcherBuilder() - .setJobManagerRunnerFactory(jobManagerRunnerFactory) - .setInitialJobGraphs( - Collections.singleton(JobGraphTestUtils.emptyJobGraph())) + createTestingDispatcherBuilder() + .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .withRecoveredJobs(Collections.singleton(JobGraphTestUtils.emptyJobGraph())) .build(); dispatcher.start(); @@ -715,9 +714,9 @@ public void testThatDirtilyFinishedJobsNotBeingRetriggered() throws Exception { final JobResult jobResult = TestingJobResultStore.createSuccessfulJobResult(jobGraph.getJobID()); dispatcher = - new TestingDispatcherBuilder() - .setInitialJobGraphs(Collections.singleton(jobGraph)) - .setDirtyJobResults(Collections.singleton(jobResult)) + createTestingDispatcherBuilder() + .withRecoveredJobs(Collections.singleton(jobGraph)) + .withRecoveredDirtyJobs(Collections.singleton(jobResult)) .build(); } @@ -800,7 +799,7 @@ public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { haServices.setJobGraphStore(submittedJobGraphStore); dispatcher = - new TestingDispatcherBuilder().setJobGraphWriter(submittedJobGraphStore).build(); + createTestingDispatcherBuilder().withJobGraphWriter(submittedJobGraphStore).build(); dispatcher.start(); @@ -910,9 +909,9 @@ public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception { testingJobGraphStore.start(null); dispatcher = - new TestingDispatcherBuilder() - .setInitialJobGraphs(Collections.singleton(jobGraph)) - .setJobGraphWriter(testingJobGraphStore) + createTestingDispatcherBuilder() + .withRecoveredJobs(Collections.singleton(jobGraph)) + .withJobGraphWriter(testingJobGraphStore) .build(); dispatcher.start(); @@ -1090,8 +1089,8 @@ public void testOnlyRecoveredJobsAreRetainedInTheBlobServer() throws Exception { final PermanentBlobKey blobKey2 = blobServer.putPermanent(jobId2, fileContent); dispatcher = - new TestingDispatcherBuilder() - .setInitialJobGraphs(Collections.singleton(new JobGraph(jobId1, "foobar"))) + createTestingDispatcherBuilder() + .withRecoveredJobs(Collections.singleton(new JobGraph(jobId1, "foobar"))) .build(); Assertions.assertThat(blobServer.getFile(jobId1, blobKey1)).hasBinaryContent(fileContent); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java index ba5ecca79a0e1..1b9c7474295b8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java @@ -20,16 +20,39 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; +import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.Preconditions; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collection; +import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; import java.util.function.Function; /** {@link Dispatcher} implementation used for testing purposes. */ @@ -56,6 +79,57 @@ class TestingDispatcher extends Dispatcher { this.startFuture = new CompletableFuture<>(); } + private TestingDispatcher( + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + Collection recoveredDirtyJobs, + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + HeartbeatServices heartbeatServices, + BlobServer blobServer, + FatalErrorHandler fatalErrorHandler, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricServiceQueryAddress, + Executor ioExecutor, + HistoryServerArchivist historyServerArchivist, + ExecutionGraphInfoStore executionGraphInfoStore, + JobManagerRunnerFactory jobManagerRunnerFactory, + DispatcherBootstrapFactory dispatcherBootstrapFactory, + DispatcherOperationCaches dispatcherOperationCaches, + JobManagerRunnerRegistry jobManagerRunnerRegistry, + ResourceCleanerFactory resourceCleanerFactory) + throws Exception { + super( + rpcService, + fencingToken, + recoveredJobs, + recoveredDirtyJobs, + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + heartbeatServices, + blobServer, + fatalErrorHandler, + jobGraphWriter, + jobResultStore, + jobManagerMetricGroup, + metricServiceQueryAddress, + ioExecutor, + historyServerArchivist, + executionGraphInfoStore, + jobManagerRunnerFactory, + dispatcherBootstrapFactory, + dispatcherOperationCaches, + jobManagerRunnerRegistry, + resourceCleanerFactory); + + this.startFuture = new CompletableFuture<>(); + } + @Override public void onStart() throws Exception { try { @@ -91,4 +165,211 @@ CompletableFuture getNumberJobs(Time timeout) { void waitUntilStarted() { startFuture.join(); } + + public static TestingDispatcher.Builder builder() { + return new Builder(); + } + + public static class Builder { + private RpcService rpcService = new TestingRpcService(); + private DispatcherId fencingToken = DispatcherId.generate(); + private Collection recoveredJobs = Collections.emptyList(); + private Collection recoveredDirtyJobs = Collections.emptyList(); + private HighAvailabilityServices highAvailabilityServices = + new TestingHighAvailabilityServices(); + + private TestingResourceManagerGateway resourceManagerGateway = + new TestingResourceManagerGateway(); + private GatewayRetriever resourceManagerGatewayRetriever = + () -> CompletableFuture.completedFuture(resourceManagerGateway); + private HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); + + private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; + private JobResultStore jobResultStore = new EmbeddedJobResultStore(); + + private Configuration configuration = new Configuration(); + + // even-though it's labeled as @Nullable, it's a mandatory field that needs to be set before + // building the Dispatcher instance + @Nullable private BlobServer blobServer = null; + private FatalErrorHandler fatalErrorHandler = new TestingFatalErrorHandler(); + private JobManagerMetricGroup jobManagerMetricGroup = + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(); + @Nullable private String metricServiceQueryAddress = null; + private Executor ioExecutor = ForkJoinPool.commonPool(); + private HistoryServerArchivist historyServerArchivist = VoidHistoryServerArchivist.INSTANCE; + private ExecutionGraphInfoStore executionGraphInfoStore = + new MemoryExecutionGraphInfoStore(); + private JobManagerRunnerFactory jobManagerRunnerFactory = + new TestingJobManagerRunnerFactory(0); + private DispatcherBootstrapFactory dispatcherBootstrapFactory = + (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(); + private DispatcherOperationCaches dispatcherOperationCaches = + new DispatcherOperationCaches(); + private JobManagerRunnerRegistry jobManagerRunnerRegistry = new JobManagerRunnerRegistry(1); + @Nullable private ResourceCleanerFactory resourceCleanerFactory; + + public Builder withRpcService(RpcService rpcService) { + this.rpcService = rpcService; + return this; + } + + public Builder withFencingToken(DispatcherId fencingToken) { + this.fencingToken = fencingToken; + return this; + } + + public Builder withRecoveredJobs(Collection recoveredJobs) { + this.recoveredJobs = recoveredJobs; + return this; + } + + public Builder withRecoveredDirtyJobs(Collection recoveredDirtyJobs) { + this.recoveredDirtyJobs = recoveredDirtyJobs; + return this; + } + + public Builder withHighAvailabilityServices( + HighAvailabilityServices highAvailabilityServices) { + this.highAvailabilityServices = highAvailabilityServices; + return this; + } + + public Builder withResourceManagerGateway( + TestingResourceManagerGateway resourceManagerGateway) { + this.resourceManagerGateway = resourceManagerGateway; + return this; + } + + public Builder withResourceManagerGatewayRetriever( + GatewayRetriever resourceManagerGatewayRetriever) { + this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; + return this; + } + + public Builder withHeartbeatServices(HeartbeatServices heartbeatServices) { + this.heartbeatServices = heartbeatServices; + return this; + } + + public Builder withJobGraphWriter(JobGraphWriter jobGraphWriter) { + this.jobGraphWriter = jobGraphWriter; + return this; + } + + public Builder withJobResultStore(JobResultStore jobResultStore) { + this.jobResultStore = jobResultStore; + return this; + } + + public Builder withConfiguration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + public Builder withBlobServer(BlobServer blobServer) { + this.blobServer = blobServer; + return this; + } + + public Builder withFatalErrorHandler(FatalErrorHandler fatalErrorHandler) { + this.fatalErrorHandler = fatalErrorHandler; + return this; + } + + public Builder withJobManagerMetricGroup(JobManagerMetricGroup jobManagerMetricGroup) { + this.jobManagerMetricGroup = jobManagerMetricGroup; + return this; + } + + public Builder withMetricServiceQueryAddress(@Nullable String metricServiceQueryAddress) { + this.metricServiceQueryAddress = metricServiceQueryAddress; + return this; + } + + public Builder withIoExecutor(Executor ioExecutor) { + this.ioExecutor = ioExecutor; + return this; + } + + public Builder withHistoryServerArchivist(HistoryServerArchivist historyServerArchivist) { + this.historyServerArchivist = historyServerArchivist; + return this; + } + + public Builder withExecutionGraphInfoStore( + ExecutionGraphInfoStore executionGraphInfoStore) { + this.executionGraphInfoStore = executionGraphInfoStore; + return this; + } + + public Builder withJobManagerRunnerFactory( + JobManagerRunnerFactory jobManagerRunnerFactory) { + this.jobManagerRunnerFactory = jobManagerRunnerFactory; + return this; + } + + public Builder withDispatcherBootstrapFactory( + DispatcherBootstrapFactory dispatcherBootstrapFactory) { + this.dispatcherBootstrapFactory = dispatcherBootstrapFactory; + return this; + } + + public Builder withDispatcherOperationCaches( + DispatcherOperationCaches dispatcherOperationCaches) { + this.dispatcherOperationCaches = dispatcherOperationCaches; + return this; + } + + public Builder withJobManagerRunnerRegistry( + JobManagerRunnerRegistry jobManagerRunnerRegistry) { + this.jobManagerRunnerRegistry = jobManagerRunnerRegistry; + return this; + } + + public Builder withResourceCleanerFactory(ResourceCleanerFactory resourceCleanerFactory) { + this.resourceCleanerFactory = resourceCleanerFactory; + return this; + } + + private ResourceCleanerFactory createDefaultResourceCleanerFactory() { + return new DispatcherResourceCleanerFactory( + ioExecutor, + jobManagerRunnerRegistry, + jobGraphWriter, + blobServer, + highAvailabilityServices, + jobManagerMetricGroup); + } + + public TestingDispatcher build() throws Exception { + return new TestingDispatcher( + rpcService, + fencingToken, + recoveredJobs, + recoveredDirtyJobs, + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + heartbeatServices, + Preconditions.checkNotNull( + blobServer, + "No BlobServer is specified for building the TestingDispatcher"), + fatalErrorHandler, + jobGraphWriter, + jobResultStore, + jobManagerMetricGroup, + metricServiceQueryAddress, + ioExecutor, + historyServerArchivist, + executionGraphInfoStore, + jobManagerRunnerFactory, + dispatcherBootstrapFactory, + dispatcherOperationCaches, + jobManagerRunnerRegistry, + resourceCleanerFactory != null + ? resourceCleanerFactory + : createDefaultResourceCleanerFactory()); + } + } } From ab85e7b58309b795df6bb6d7c7f85daac1b6b298 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 9 Dec 2021 17:34:55 +0100 Subject: [PATCH 42/49] [FLINK-25432] Adds CheckpointResourcesCleanupRunner Adds CleanupRunnerFactory and an implementing class. Efforts about adding the job name are out-sourced into FLINK-25632. --- .../CheckpointResourcesCleanupRunner.java | 242 +++++++ ...eckpointResourcesCleanupRunnerFactory.java | 51 ++ .../cleanup/CleanupRunnerFactory.java | 39 ++ .../CheckpointResourcesCleanupRunnerTest.java | 589 ++++++++++++++++++ 4 files changed, 921 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRunnerFactory.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java new file mode 100644 index 0000000000000..86b707d09ce99 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java @@ -0,0 +1,242 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CheckpointsCleaner; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStoreUtils; +import org.apache.flink.runtime.dispatcher.UnavailableDispatcherOperationException; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.webmonitor.JobDetails; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.state.SharedStateRegistryFactory; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * {@code CheckpointResourcesCleanupRunner} implements {@link JobManagerRunner} in a way, that only + * the checkpoint-related resources are instantiated. It triggers any job-specific cleanup that's + * usually performed by the {@link JobMaster} without rebuilding the corresponding {@link + * org.apache.flink.runtime.executiongraph.ExecutionGraph}. + */ +public class CheckpointResourcesCleanupRunner implements JobManagerRunner { + + private static final Logger LOG = + LoggerFactory.getLogger(CheckpointResourcesCleanupRunner.class); + + private final JobResult jobResult; + private final CheckpointRecoveryFactory checkpointRecoveryFactory; + private final CheckpointsCleaner checkpointsCleaner; + private final SharedStateRegistryFactory sharedStateRegistryFactory; + private final Configuration jobManagerConfiguration; + private final Executor cleanupExecutor; + + private final long initializationTimestamp; + + // we have to have two separate futures because closeAsync relies on the completion of + // getResultFuture which is always already completed but the cleanupFuture is only + // instantiated when calling start + private CompletableFuture cleanupFuture; + private final CompletableFuture closeFuture = new CompletableFuture<>(); + + private CompletedCheckpointStore completedCheckpointStore; + private CheckpointIDCounter checkpointIDCounter; + + public CheckpointResourcesCleanupRunner( + JobResult jobResult, + CheckpointRecoveryFactory checkpointRecoveryFactory, + CheckpointsCleaner checkpointsCleaner, + SharedStateRegistryFactory sharedStateRegistryFactory, + Configuration jobManagerConfiguration, + Executor cleanupExecutor, + long initializationTimestamp) { + this.jobResult = Preconditions.checkNotNull(jobResult); + this.checkpointRecoveryFactory = Preconditions.checkNotNull(checkpointRecoveryFactory); + this.checkpointsCleaner = Preconditions.checkNotNull(checkpointsCleaner); + this.sharedStateRegistryFactory = Preconditions.checkNotNull(sharedStateRegistryFactory); + this.jobManagerConfiguration = Preconditions.checkNotNull(jobManagerConfiguration); + this.cleanupExecutor = Preconditions.checkNotNull(cleanupExecutor); + this.initializationTimestamp = initializationTimestamp; + } + + @Override + public CompletableFuture closeAsync() { + return closeFuture; + } + + @Override + public void start() throws Exception { + cleanupFuture = + CompletableFuture.runAsync(this::initializeAccessingComponents, cleanupExecutor) + .thenApply( + result -> { + Exception exception = null; + try { + completedCheckpointStore.shutdown( + getJobStatus(), checkpointsCleaner); + } catch (Exception e) { + exception = e; + } + + try { + checkpointIDCounter.shutdown(getJobStatus()); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); + } + + if (exception != null) { + throw new CompletionException(exception); + } + + return null; + }); + + FutureUtils.forward(cleanupFuture, closeFuture); + } + + private void initializeAccessingComponents() { + initializeCompletedCheckpointStore(); + initializeCheckpointIDCounter(); + } + + private void initializeCompletedCheckpointStore() { + try { + this.completedCheckpointStore = + checkpointRecoveryFactory.createRecoveredCompletedCheckpointStore( + getJobID(), + CompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( + jobManagerConfiguration, LOG), + sharedStateRegistryFactory, + cleanupExecutor); + } catch (Exception e) { + throw new CompletionException( + "Error occurred while initializing the CompletedCheckpointStore access.", e); + } + } + + private void initializeCheckpointIDCounter() { + try { + this.checkpointIDCounter = + checkpointRecoveryFactory.createCheckpointIDCounter(getJobID()); + } catch (Exception e) { + throw new CompletionException( + "Error occurred while initializing the CheckpointIDCounter access.", e); + } + } + + @Override + public CompletableFuture getJobMasterGateway() { + return FutureUtils.completedExceptionally( + new UnavailableDispatcherOperationException( + "Unable to get JobMasterGateway for job in cleanup phase. The requested operation is not available in that stage.")); + } + + @Override + public CompletableFuture getResultFuture() { + return CompletableFuture.completedFuture( + JobManagerRunnerResult.forSuccess(createExecutionGraphInfoFromJobResult())); + } + + @Override + public JobID getJobID() { + return jobResult.getJobId(); + } + + @Override + public CompletableFuture cancel(Time timeout) { + Preconditions.checkState( + cleanupFuture != null, + "The CheckpointResourcesCleanupRunner was not started, yet."); + if (cleanupFuture.cancel(true)) { + return CompletableFuture.completedFuture(Acknowledge.get()); + } + + return FutureUtils.completedExceptionally( + new FlinkException("Cleanup task couldn't be cancelled.")); + } + + @Override + public CompletableFuture requestJobStatus(Time timeout) { + return CompletableFuture.completedFuture(getJobStatus()); + } + + @Override + public CompletableFuture requestJobDetails(Time timeout) { + return requestJob(timeout) + .thenApply( + executionGraphInfo -> + JobDetails.createDetailsForJob( + executionGraphInfo.getArchivedExecutionGraph())); + } + + @Override + public CompletableFuture requestJob(Time timeout) { + return CompletableFuture.completedFuture(createExecutionGraphInfoFromJobResult()); + } + + @Override + public boolean isInitialized() { + return true; + } + + private ExecutionGraphInfo createExecutionGraphInfoFromJobResult() { + return generateExecutionGraphInfo(jobResult, initializationTimestamp); + } + + private JobStatus getJobStatus() { + return getJobStatus(jobResult); + } + + private static JobStatus getJobStatus(JobResult jobResult) { + return jobResult.getApplicationStatus().deriveJobStatus(); + } + + private static ExecutionGraphInfo generateExecutionGraphInfo( + JobResult jobResult, long initializationTimestamp) { + return new ExecutionGraphInfo( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( + jobResult.getJobId(), + "unknown", + getJobStatus(jobResult), + jobResult.getSerializedThrowable().orElse(null), + null, + initializationTimestamp)); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerFactory.java new file mode 100644 index 0000000000000..302d318336b92 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerFactory.java @@ -0,0 +1,51 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CheckpointsCleaner; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.state.SharedStateRegistry; + +import java.util.concurrent.Executor; + +/** + * {@code CheckpointResourcesCleanupRunnerFactory} implements {@link CleanupRunnerFactory} providing + * a factory method for creating {@link CheckpointResourcesCleanupRunner} instances. + */ +public enum CheckpointResourcesCleanupRunnerFactory implements CleanupRunnerFactory { + INSTANCE; + + @Override + public CheckpointResourcesCleanupRunner create( + JobResult jobResult, + CheckpointRecoveryFactory checkpointRecoveryFactory, + Configuration configuration, + Executor cleanupExecutor) { + return new CheckpointResourcesCleanupRunner( + jobResult, + checkpointRecoveryFactory, + new CheckpointsCleaner(), + SharedStateRegistry.DEFAULT_FACTORY, + configuration, + cleanupExecutor, + System.currentTimeMillis()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRunnerFactory.java new file mode 100644 index 0000000000000..d9d8ebaf2cd0d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRunnerFactory.java @@ -0,0 +1,39 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobResult; + +import java.util.concurrent.Executor; + +/** + * {@code CleanupRunnerFactory} provides a factory method for creating {@link + * CheckpointResourcesCleanupRunner} instances. + */ +@FunctionalInterface +public interface CleanupRunnerFactory { + JobManagerRunner create( + JobResult jobResult, + CheckpointRecoveryFactory checkpointRecoveryFactory, + Configuration configuration, + Executor cleanupExecutor); +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java new file mode 100644 index 0000000000000..0e16bd43b98c2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java @@ -0,0 +1,589 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CheckpointsCleaner; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.TestingCheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.TestingCheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.TestingCompletedCheckpointStore; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.UnavailableDispatcherOperationException; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.ErrorInfo; +import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.SharedStateRegistryFactory; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedThrowable; +import org.apache.flink.util.concurrent.Executors; +import org.apache.flink.util.function.ThrowingConsumer; + +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * {@code CheckpointResourcesCleanupRunnerTest} tests the {@link CheckpointResourcesCleanupRunner} + * implementation. + */ +public class CheckpointResourcesCleanupRunnerTest { + + private static final ThrowingConsumer + BEFORE_START = ignored -> {}; + private static final ThrowingConsumer + AFTER_START = CheckpointResourcesCleanupRunner::start; + private static final ThrowingConsumer + AFTER_CLOSE = + runner -> { + runner.start(); + runner.close(); + }; + + @Test + public void testIsInitializedBeforeStart() throws Exception { + testIsInitialized(BEFORE_START); + } + + @Test + public void testIsInitializedAfterStart() throws Exception { + testIsInitialized(AFTER_START); + } + + @Test + public void testIsInitializedAfterClose() throws Exception { + testIsInitialized(AFTER_CLOSE); + } + + private static void testIsInitialized( + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + final CheckpointResourcesCleanupRunner testInstance = new TestInstanceBuilder().build(); + preCheckLifecycleHandling.accept(testInstance); + + assertThat(testInstance.isInitialized()).isTrue(); + } + + @Test + public void testCloseAsyncBeforeStart() { + final CheckpointResourcesCleanupRunner testInstance = new TestInstanceBuilder().build(); + assertThat(testInstance.closeAsync()).isNotCompleted(); + } + + @Test + public void testSuccessfulCloseAsyncAfterStart() throws Exception { + final CompletableFuture completedCheckpointStoreShutdownFuture = + new CompletableFuture<>(); + final CompletableFuture checkpointIdCounterShutdownFuture = + new CompletableFuture<>(); + + final HaltingCheckpointRecoveryFactory checkpointRecoveryFactory = + new HaltingCheckpointRecoveryFactory( + completedCheckpointStoreShutdownFuture, checkpointIdCounterShutdownFuture); + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withCheckpointRecoveryFactory(checkpointRecoveryFactory) + .withExecutor(ForkJoinPool.commonPool()) + .build(); + testInstance.start(); + + assertThat(completedCheckpointStoreShutdownFuture) + .as("The CompletedCheckpointStore shouldn't have been shut down, yet.") + .isNotCompleted(); + assertThat(checkpointIdCounterShutdownFuture) + .as("The CheckpointIDCounter shouldn't have been shut down, yet.") + .isNotCompleted(); + + assertThat(testInstance.closeAsync()) + .as( + "closeAsync shouldn't have been completed, yet, since the shutdown of the components is not completed.") + .isNotCompleted(); + + checkpointRecoveryFactory.triggerCreation(); + + assertThat(completedCheckpointStoreShutdownFuture) + .as("The CompletedCheckpointStore should have been shut down properly.") + .succeedsWithin(Duration.ofMillis(100)) + .isEqualTo(JobStatus.FINISHED); + assertThat(checkpointIdCounterShutdownFuture) + .as("The CheckpointIDCounter should have been shut down properly.") + .succeedsWithin(Duration.ofMillis(100)) + .isEqualTo(JobStatus.FINISHED); + + assertThat(testInstance.closeAsync()).succeedsWithin(Duration.ofMillis(100)); + } + + @Test + public void testCloseAsyncAfterStartAndErrorInCompletedCheckpointStoreShutdown() + throws Exception { + final CompletableFuture checkpointIdCounterShutdownFuture = + new CompletableFuture<>(); + + final HaltingCheckpointRecoveryFactory checkpointRecoveryFactory = + new HaltingCheckpointRecoveryFactory( + TestingCompletedCheckpointStore.builder() + .withShutdownConsumer( + (ignoredJobStatus, ignoredCheckpointsCleaner) -> { + throw new RuntimeException( + "Expected RuntimeException simulating an error during shutdown."); + }) + .build(), + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + checkpointIdCounterShutdownFuture)); + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withCheckpointRecoveryFactory(checkpointRecoveryFactory) + .withExecutor(ForkJoinPool.commonPool()) + .build(); + testInstance.start(); + + assertThat(checkpointIdCounterShutdownFuture) + .as("The CheckpointIDCounter shouldn't have been shut down, yet.") + .isNotCompleted(); + + assertThat(testInstance.closeAsync()) + .as( + "closeAsync shouldn't have been completed, yet, since the shutdown of the components is not completed.") + .isNotCompleted(); + + checkpointRecoveryFactory.triggerCreation(); + + assertThat(checkpointIdCounterShutdownFuture) + .as("The CheckpointIDCounter should have been shut down properly.") + .succeedsWithin(Duration.ofMillis(100)) + .isEqualTo(JobStatus.FINISHED); + + assertThat(testInstance.closeAsync()) + .failsWithin(Duration.ofMillis(100)) + .withThrowableOfType(ExecutionException.class) + .withCauseInstanceOf(RuntimeException.class); + } + + @Test + public void testCloseAsyncAfterStartAndErrorInCheckpointIDCounterShutdown() throws Exception { + final CompletableFuture completedCheckpointStoreShutdownFuture = + new CompletableFuture<>(); + + final HaltingCheckpointRecoveryFactory checkpointRecoveryFactory = + new HaltingCheckpointRecoveryFactory( + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + completedCheckpointStoreShutdownFuture), + TestingCheckpointIDCounter.builder() + .withShutdownConsumer( + ignoredJobStatus -> { + throw new RuntimeException( + "Expected RuntimeException simulating an error during shutdown."); + }) + .build()); + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withCheckpointRecoveryFactory(checkpointRecoveryFactory) + .withExecutor(ForkJoinPool.commonPool()) + .build(); + testInstance.start(); + + assertThat(completedCheckpointStoreShutdownFuture) + .as("The CompletedCheckpointStore shouldn't have been shut down, yet.") + .isNotCompleted(); + + assertThat(testInstance.closeAsync()) + .as( + "closeAsync shouldn't have been completed, yet, since the shutdown of the components is not completed.") + .isNotCompleted(); + + checkpointRecoveryFactory.triggerCreation(); + + assertThat(completedCheckpointStoreShutdownFuture) + .as("The CompletedCheckpointStore should have been shut down properly.") + .succeedsWithin(Duration.ofMillis(100)) + .isEqualTo(JobStatus.FINISHED); + + assertThat(testInstance.closeAsync()) + .failsWithin(Duration.ofMillis(100)) + .withThrowableOfType(ExecutionException.class) + .withCauseInstanceOf(RuntimeException.class); + } + + @Test + public void testResultFutureWithSuccessBeforeStart() throws Exception { + testResultFutureWithSuccess(BEFORE_START); + } + + @Test + public void testResultFutureWithSuccessAfterStart() throws Exception { + testResultFutureWithSuccess(AFTER_START); + } + + @Test + public void testResultFutureWithSuccessAfterClose() throws Exception { + testResultFutureWithSuccess(AFTER_CLOSE); + } + + private static void testResultFutureWithSuccess( + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + testResultFuture(createDummySuccessJobResult(), preCheckLifecycleHandling); + } + + @Test + public void testResultFutureWithErrorBeforeStart() throws Exception { + testResultFutureWithError(BEFORE_START); + } + + @Test + public void testResultFutureWithErrorAfterStart() throws Exception { + testResultFutureWithError(AFTER_START); + } + + @Test + public void testResultFutureWithErrorAfterClose() throws Exception { + testResultFutureWithError(AFTER_CLOSE); + } + + private static void testResultFutureWithError( + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + final SerializedThrowable expectedError = + new SerializedThrowable(new Exception("Expected exception")); + final CompletableFuture actualResult = + testResultFuture( + createJobResultWithFailure(expectedError), preCheckLifecycleHandling); + + assertThat(actualResult) + .succeedsWithin(Duration.ZERO) + .extracting(JobManagerRunnerResult::getExecutionGraphInfo) + .extracting(ExecutionGraphInfo::getArchivedExecutionGraph) + .extracting(AccessExecutionGraph::getFailureInfo) + .extracting(ErrorInfo::getException) + .isEqualTo(expectedError); + } + + private static CompletableFuture testResultFuture( + JobResult jobResult, + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder().withJobResult(jobResult).build(); + preCheckLifecycleHandling.accept(testInstance); + + assertThat(testInstance.getResultFuture()) + .succeedsWithin(Duration.ZERO) + .extracting(JobManagerRunnerResult::isSuccess) + .isEqualTo(true); + + return testInstance.getResultFuture(); + } + + @Test + public void testGetJobID() { + final JobID jobId = new JobID(); + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withJobResult(createJobResult(jobId, ApplicationStatus.CANCELED)) + .build(); + assertThat(testInstance.getJobID()).isEqualTo(jobId); + } + + @Test + public void testGetJobMasterGatewayBeforeStart() throws Exception { + testGetJobMasterGateway(BEFORE_START); + } + + @Test + public void testGetJobMasterGatewayAfterStart() throws Exception { + testGetJobMasterGateway(AFTER_START); + } + + @Test + public void testGetJobMasterGatewayAfterClose() throws Exception { + testGetJobMasterGateway(AFTER_CLOSE); + } + + private static void testGetJobMasterGateway( + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + final CheckpointResourcesCleanupRunner testInstance = new TestInstanceBuilder().build(); + preCheckLifecycleHandling.accept(testInstance); + + assertThat(testInstance.getJobMasterGateway()) + .failsWithin(Duration.ZERO) + .withThrowableOfType(ExecutionException.class) + .withCauseInstanceOf(UnavailableDispatcherOperationException.class); + } + + @Test + public void testRequestJob_ExceptionHistory() { + testRequestJob( + createDummySuccessJobResult(), + System.currentTimeMillis(), + actualExecutionGraphInfo -> + assertThat(actualExecutionGraphInfo) + .extracting(ExecutionGraphInfo::getExceptionHistory) + .asList() + .isEmpty()); + } + + @Test + public void testRequestJob_JobName() { + testRequestJobExecutionGraph( + createDummySuccessJobResult(), + System.currentTimeMillis(), + actualExecutionGraph -> + assertThat(actualExecutionGraph) + .extracting(AccessExecutionGraph::getJobName) + .isEqualTo("unknown")); + } + + @Test + public void testRequestJob_JobId() { + final JobResult jobResult = createDummySuccessJobResult(); + testRequestJobExecutionGraph( + jobResult, + System.currentTimeMillis(), + actualExecutionGraph -> + assertThat(actualExecutionGraph) + .extracting(AccessExecutionGraph::getJobID) + .isEqualTo(jobResult.getJobId())); + } + + @Test + public void testRequestJob_JobState() { + final JobResult jobResult = createDummySuccessJobResult(); + testRequestJobExecutionGraph( + jobResult, + System.currentTimeMillis(), + actualExecutionGraph -> + assertThat(actualExecutionGraph) + .extracting(AccessExecutionGraph::getState) + .isEqualTo(jobResult.getApplicationStatus().deriveJobStatus())); + } + + @Test + public void testRequestJob_InitiatizationTimestamp() { + final long initializationTimestamp = System.currentTimeMillis(); + testRequestJobExecutionGraph( + createDummySuccessJobResult(), + initializationTimestamp, + actualExecutionGraph -> + assertThat(actualExecutionGraph.getStatusTimestamp(JobStatus.INITIALIZING)) + .isEqualTo(initializationTimestamp)); + } + + @Test + public void testRequestJobWithFailure() { + final SerializedThrowable expectedError = + new SerializedThrowable(new Exception("Expected exception")); + final JobResult jobResult = createJobResultWithFailure(expectedError); + testRequestJobExecutionGraph( + jobResult, + System.currentTimeMillis(), + actualExecutionGraph -> + assertThat(actualExecutionGraph) + .extracting(AccessExecutionGraph::getFailureInfo) + .extracting(ErrorInfo::getException) + .isEqualTo(expectedError)); + } + + private static void testRequestJobExecutionGraph( + JobResult jobResult, + long initializationTimestamp, + ThrowingConsumer assertion) { + testRequestJob( + jobResult, + initializationTimestamp, + actualExecutionGraphInfo -> + assertThat(actualExecutionGraphInfo) + .extracting(ExecutionGraphInfo::getArchivedExecutionGraph) + .satisfies(assertion::accept)); + } + + private static void testRequestJob( + JobResult jobResult, + long initializationTimestamp, + ThrowingConsumer assertion) { + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withJobResult(jobResult) + .withInitializationTimestamp(initializationTimestamp) + .build(); + + final CompletableFuture response = + testInstance.requestJob(Time.milliseconds(0)); + assertThat(response).succeedsWithin(Duration.ZERO).satisfies(assertion::accept); + } + + private static JobResult createDummySuccessJobResult() { + return createJobResult(new JobID(), ApplicationStatus.SUCCEEDED); + } + + private static JobResult createJobResultWithFailure(SerializedThrowable throwable) { + return new JobResult.Builder() + .jobId(new JobID()) + .applicationStatus(ApplicationStatus.FAILED) + .serializedThrowable(throwable) + .netRuntime(1) + .build(); + } + + private static JobResult createJobResult(JobID jobId, ApplicationStatus applicationStatus) { + return new JobResult.Builder() + .jobId(jobId) + .applicationStatus(applicationStatus) + .netRuntime(1) + .build(); + } + + private static CheckpointRecoveryFactory createCheckpointRecoveryFactory() { + return new TestingCheckpointRecoveryFactory( + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + new CompletableFuture<>()), + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + new CompletableFuture<>())); + } + + private static class TestInstanceBuilder { + + private JobResult jobResult = createDummySuccessJobResult(); + private CheckpointRecoveryFactory checkpointRecoveryFactory = + createCheckpointRecoveryFactory(); + private CheckpointsCleaner checkpointsCleaner = new CheckpointsCleaner(); + private SharedStateRegistryFactory sharedStateRegistryFactory = + SharedStateRegistry.DEFAULT_FACTORY; + private Executor executor = Executors.directExecutor(); + private Configuration configuration = new Configuration(); + private long initializationTimestamp = System.currentTimeMillis(); + + public TestInstanceBuilder withJobResult(JobResult jobResult) { + this.jobResult = jobResult; + return this; + } + + public TestInstanceBuilder withCheckpointRecoveryFactory( + CheckpointRecoveryFactory checkpointRecoveryFactory) { + this.checkpointRecoveryFactory = checkpointRecoveryFactory; + return this; + } + + public TestInstanceBuilder withCheckpointsCleaner(CheckpointsCleaner checkpointsCleaner) { + this.checkpointsCleaner = checkpointsCleaner; + return this; + } + + public TestInstanceBuilder withSharedStateRegistryFactory( + SharedStateRegistryFactory sharedStateRegistryFactory) { + this.sharedStateRegistryFactory = sharedStateRegistryFactory; + return this; + } + + public TestInstanceBuilder withExecutor(Executor executor) { + this.executor = executor; + return this; + } + + public TestInstanceBuilder withConfiguration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + public TestInstanceBuilder withInitializationTimestamp(long initializationTimestamp) { + this.initializationTimestamp = initializationTimestamp; + return this; + } + + public CheckpointResourcesCleanupRunner build() { + return new CheckpointResourcesCleanupRunner( + jobResult, + checkpointRecoveryFactory, + checkpointsCleaner, + sharedStateRegistryFactory, + configuration, + executor, + initializationTimestamp); + } + } + + private static class HaltingCheckpointRecoveryFactory implements CheckpointRecoveryFactory { + + private final CompletedCheckpointStore completedCheckpointStore; + private final CheckpointIDCounter checkpointIDCounter; + + private final OneShotLatch creationLatch = new OneShotLatch(); + + public HaltingCheckpointRecoveryFactory( + CompletableFuture completableCheckpointStoreShutDownFuture, + CompletableFuture checkpointIDCounterShutDownFuture) { + this( + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + completableCheckpointStoreShutDownFuture), + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + checkpointIDCounterShutDownFuture)); + } + + public HaltingCheckpointRecoveryFactory( + CompletedCheckpointStore completedCheckpointStore, + CheckpointIDCounter checkpointIDCounter) { + this.completedCheckpointStore = Preconditions.checkNotNull(completedCheckpointStore); + this.checkpointIDCounter = Preconditions.checkNotNull(checkpointIDCounter); + } + + @Override + public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( + JobID jobId, + int maxNumberOfCheckpointsToRetain, + SharedStateRegistryFactory sharedStateRegistryFactory, + Executor ioExecutor) + throws Exception { + creationLatch.await(); + return completedCheckpointStore; + } + + @Override + public CheckpointIDCounter createCheckpointIDCounter(JobID jobId) throws Exception { + creationLatch.await(); + return checkpointIDCounter; + } + + public void triggerCreation() { + creationLatch.trigger(); + } + } +} From abb7f1d5fbd735087abc1a489a1edb13d713cc7a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 16:19:13 +0100 Subject: [PATCH 43/49] [FLINK-25432] Refactors createJobManagerRunner to be an initializing method --- .../flink/runtime/dispatcher/Dispatcher.java | 28 +++++++++---------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 160646013c6ae..015450bc40599 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -545,8 +545,9 @@ private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Excep Preconditions.checkState(!jobManagerRunnerRegistry.isRegistered(jobGraph.getJobID())); long initializationTimestamp = System.currentTimeMillis(); JobManagerRunner jobManagerRunner = - createJobManagerRunner(jobGraph, initializationTimestamp); + initializeJobManagerRunner(jobGraph, initializationTimestamp); + jobManagerRunner.start(); jobManagerRunnerRegistry.register(jobManagerRunner); final JobID jobId = jobGraph.getJobID(); @@ -603,23 +604,20 @@ private CleanupJobState jobManagerRunnerFailed(JobID jobId, Throwable throwable) return CleanupJobState.LOCAL; } - JobManagerRunner createJobManagerRunner(JobGraph jobGraph, long initializationTimestamp) + JobManagerRunner initializeJobManagerRunner(JobGraph jobGraph, long initializationTimestamp) throws Exception { final RpcService rpcService = getRpcService(); - JobManagerRunner runner = - jobManagerRunnerFactory.createJobManagerRunner( - jobGraph, - configuration, - rpcService, - highAvailabilityServices, - heartbeatServices, - jobManagerSharedServices, - new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup), - fatalErrorHandler, - initializationTimestamp); - runner.start(); - return runner; + return jobManagerRunnerFactory.createJobManagerRunner( + jobGraph, + configuration, + rpcService, + highAvailabilityServices, + heartbeatServices, + jobManagerSharedServices, + new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup), + fatalErrorHandler, + initializationTimestamp); } @Override From 06ca371aa60ccb7a6c29dfbd5370d78e0aa1c1d4 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 15 Dec 2021 16:30:05 +0100 Subject: [PATCH 44/49] [FLINK-25432] Integrates CheckpointResourcesCleanupRunner into Dispatcher --- .../ApplicationDispatcherBootstrapITCase.java | 4 +- .../flink/runtime/dispatcher/Dispatcher.java | 71 ++++++++++++++++--- .../dispatcher/DispatcherFailoverITCase.java | 27 ++++--- 3 files changed, 79 insertions(+), 23 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java index 2b49f0c87512f..c20a959db4894 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java @@ -214,11 +214,11 @@ public JobResultStore getJobResultStore() { assertThat( jobResultStore.hasDirtyJobResultEntry( ApplicationDispatcherBootstrap.ZERO_JOB_ID)) - .isTrue(); + .isFalse(); assertThat( jobResultStore.hasCleanJobResultEntry( ApplicationDispatcherBootstrap.ZERO_JOB_ID)) - .isFalse(); + .isTrue(); } @Test diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 015450bc40599..fe69704759df8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -29,10 +29,12 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.checkpoint.Checkpoints; +import org.apache.flink.runtime.checkpoint.CheckpointsCleaner; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunner; import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleaner; import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; @@ -73,6 +75,7 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcServiceUtils; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -130,6 +133,8 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint recoveredJobs; + private final Collection recoveredDirtyJobs; + private final DispatcherBootstrapFactory dispatcherBootstrapFactory; private final ExecutionGraphInfoStore executionGraphInfoStore; @@ -273,6 +278,8 @@ protected Dispatcher( this.recoveredJobs = new HashSet<>(recoveredJobs); + this.recoveredDirtyJobs = new HashSet<>(recoveredDirtyJobs); + this.blobServer.retainJobs( recoveredJobs.stream().map(JobGraph::getJobID).collect(Collectors.toSet())); @@ -310,7 +317,9 @@ public void onStart() throws Exception { throw exception; } + startCleanupRetries(); startRecoveredJobs(); + this.dispatcherBootstrap = this.dispatcherBootstrapFactory.create( getSelfGateway(DispatcherGateway.class), @@ -355,7 +364,7 @@ private void startRecoveredJobs() { private void runRecoveredJob(final JobGraph recoveredJob) { checkNotNull(recoveredJob); try { - runJob(recoveredJob, ExecutionType.RECOVERY); + initializeAndStartJobManagerRunner(recoveredJob, ExecutionType.RECOVERY); } catch (Throwable throwable) { onFatalError( new DispatcherException( @@ -365,6 +374,26 @@ private void runRecoveredJob(final JobGraph recoveredJob) { } } + private void startCleanupRetries() { + recoveredDirtyJobs.forEach(this::runCleanupRetry); + recoveredDirtyJobs.clear(); + } + + private void runCleanupRetry(final JobResult jobResult) { + checkNotNull(jobResult); + + try { + initializeAndStartCheckpointJobDataCleanupRunner(jobResult); + } catch (Throwable throwable) { + onFatalError( + new DispatcherException( + String.format( + "Could not start cleanup retry for job %s.", + jobResult.getJobId()), + throwable)); + } + } + private void handleStartDispatcherServicesException(Exception e) throws Exception { try { stopDispatcherServices(); @@ -538,19 +567,30 @@ private CompletableFuture internalSubmitJob(JobGraph jobGraph) { private void persistAndRunJob(JobGraph jobGraph) throws Exception { jobGraphWriter.putJobGraph(jobGraph); - runJob(jobGraph, ExecutionType.SUBMISSION); + initializeAndStartJobManagerRunner(jobGraph, ExecutionType.SUBMISSION); } - private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Exception { + private void initializeAndStartJobManagerRunner(JobGraph jobGraph, ExecutionType executionType) + throws Exception { Preconditions.checkState(!jobManagerRunnerRegistry.isRegistered(jobGraph.getJobID())); - long initializationTimestamp = System.currentTimeMillis(); - JobManagerRunner jobManagerRunner = - initializeJobManagerRunner(jobGraph, initializationTimestamp); + final JobManagerRunner jobManagerRunner = initializeJobManagerRunner(jobGraph); + runJob(jobManagerRunner, executionType); + } + + private void initializeAndStartCheckpointJobDataCleanupRunner(JobResult jobResult) + throws Exception { + Preconditions.checkState(!jobManagerRunnerRegistry.isRegistered(jobResult.getJobId())); + final JobManagerRunner checkpointJobDataCleanupRunner = + initializeCheckpointJobDataCleanupRunner(jobResult); + runJob(checkpointJobDataCleanupRunner, ExecutionType.RECOVERY); + } + private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionType) + throws Exception { jobManagerRunner.start(); jobManagerRunnerRegistry.register(jobManagerRunner); - final JobID jobId = jobGraph.getJobID(); + final JobID jobId = jobManagerRunner.getJobID(); final CompletableFuture cleanupJobStateFuture = jobManagerRunner @@ -604,8 +644,7 @@ private CleanupJobState jobManagerRunnerFailed(JobID jobId, Throwable throwable) return CleanupJobState.LOCAL; } - JobManagerRunner initializeJobManagerRunner(JobGraph jobGraph, long initializationTimestamp) - throws Exception { + private JobManagerRunner initializeJobManagerRunner(JobGraph jobGraph) throws Exception { final RpcService rpcService = getRpcService(); return jobManagerRunnerFactory.createJobManagerRunner( @@ -617,7 +656,19 @@ JobManagerRunner initializeJobManagerRunner(JobGraph jobGraph, long initializati jobManagerSharedServices, new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup), fatalErrorHandler, - initializationTimestamp); + System.currentTimeMillis()); + } + + private JobManagerRunner initializeCheckpointJobDataCleanupRunner(JobResult jobResult) + throws Exception { + return new CheckpointResourcesCleanupRunner( + jobResult, + highAvailabilityServices.getCheckpointRecoveryFactory(), + new CheckpointsCleaner(), + SharedStateRegistry.DEFAULT_FACTORY, + configuration, + ioExecutor, + System.currentTimeMillis()); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java index 1a796363404ea..6e6c9cb360634 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java @@ -19,6 +19,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.checkpoint.EmbeddedCompletedCheckpointStore; import org.apache.flink.runtime.checkpoint.PerJobCheckpointRecoveryFactory; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; @@ -36,10 +37,13 @@ import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.TimeUtils; import org.hamcrest.CoreMatchers; +import org.hamcrest.collection.IsEmptyCollection; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -55,13 +59,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; /** An integration test for various fail-over scenarios of the {@link Dispatcher} component. */ @@ -122,6 +123,7 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() .setGlobalCleanupConsumer( graph -> { if (temporaryErrorRef.get() != null) { + // only make the failure happen once throw temporaryErrorRef.getAndSet(null); } }) @@ -191,16 +193,19 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() // Run a second dispatcher, that restores our finished job. final Dispatcher secondDispatcher = createRecoveredDispatcher(null); toTerminate.add(secondDispatcher); - - // new Dispatcher becomes new leader leaderElectionService.isLeader(UUID.randomUUID()); - assertThrows( - "No JobMaster will be instantiated because of the JobResult is already persisted in the JobResultStore", - TimeoutException.class, - () -> - connectToLeadingJobMaster(leaderElectionService) - .get(100, TimeUnit.MILLISECONDS)); + CommonTestUtils.waitUntilCondition( + () -> haServices.getJobResultStore().getDirtyResults().isEmpty(), + Deadline.fromNow(TimeUtils.toDuration(TIMEOUT))); + + assertThat( + "The JobGraph is not stored in the JobGraphStore.", + haServices.getJobGraphStore().getJobIds(), + IsEmptyCollection.empty()); + assertTrue( + "The JobResultStore has the job listed as clean.", + haServices.getJobResultStore().hasJobResultEntry(jobId)); } private JobGraph createJobGraph() { From 6f16fe8ab21a021278c00f566ae12e65ac529ad0 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 28 Jan 2022 15:03:33 +0100 Subject: [PATCH 45/49] [FLINK-25432] Integrates CleanupRunnerFactory into test infrastructure --- .../flink/runtime/dispatcher/Dispatcher.java | 15 +++-- .../dispatcher/DispatcherServices.java | 14 ++++- .../dispatcher/JobDispatcherFactory.java | 4 +- .../dispatcher/SessionDispatcherFactory.java | 4 +- .../dispatcher/AbstractDispatcherTest.java | 2 + .../DispatcherResourceCleanupTest.java | 4 +- .../runtime/dispatcher/DispatcherTest.java | 11 ++-- .../dispatcher/MiniDispatcherTest.java | 33 +++++----- .../runtime/dispatcher/TestingDispatcher.java | 13 +++- .../TestingJobManagerRunnerFactory.java | 39 +++--------- ...bMasterServiceLeadershipRunnerFactory.java | 61 +++++++++++++++++++ .../cleanup/TestingCleanupRunnerFactory.java | 52 ++++++++++++++++ .../runner/DefaultDispatcherRunnerITCase.java | 21 +++++-- 13 files changed, 197 insertions(+), 76 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobMasterServiceLeadershipRunnerFactory.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanupRunnerFactory.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index fe69704759df8..4e04c29d66ed1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -29,12 +29,11 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.checkpoint.Checkpoints; -import org.apache.flink.runtime.checkpoint.CheckpointsCleaner; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunner; +import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleaner; import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; @@ -75,7 +74,6 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcServiceUtils; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -140,6 +138,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint(2); @@ -661,14 +663,11 @@ private JobManagerRunner initializeJobManagerRunner(JobGraph jobGraph) throws Ex private JobManagerRunner initializeCheckpointJobDataCleanupRunner(JobResult jobResult) throws Exception { - return new CheckpointResourcesCleanupRunner( + return cleanupRunnerFactory.create( jobResult, highAvailabilityServices.getCheckpointRecoveryFactory(), - new CheckpointsCleaner(), - SharedStateRegistry.DEFAULT_FACTORY, configuration, - ioExecutor, - System.currentTimeMillis()); + ioExecutor); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java index 5e3e32982f551..12112b15b83e7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultStore; @@ -65,6 +66,8 @@ public class DispatcherServices { private final JobManagerRunnerFactory jobManagerRunnerFactory; + private final CleanupRunnerFactory cleanupRunnerFactory; + private final Executor ioExecutor; DispatcherServices( @@ -82,6 +85,7 @@ public class DispatcherServices { JobGraphWriter jobGraphWriter, JobResultStore jobResultStore, JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory, Executor ioExecutor) { this.configuration = Preconditions.checkNotNull(configuration, "Configuration"); this.highAvailabilityServices = @@ -104,6 +108,8 @@ public class DispatcherServices { this.jobResultStore = Preconditions.checkNotNull(jobResultStore, "JobResultStore"); this.jobManagerRunnerFactory = Preconditions.checkNotNull(jobManagerRunnerFactory, "JobManagerRunnerFactory"); + this.cleanupRunnerFactory = + Preconditions.checkNotNull(cleanupRunnerFactory, "CleanupRunnerFactory"); this.ioExecutor = Preconditions.checkNotNull(ioExecutor, "IOExecutor"); } @@ -164,6 +170,10 @@ JobManagerRunnerFactory getJobManagerRunnerFactory() { return jobManagerRunnerFactory; } + CleanupRunnerFactory getCleanupRunnerFactory() { + return cleanupRunnerFactory; + } + public Executor getIoExecutor() { return ioExecutor; } @@ -171,7 +181,8 @@ public Executor getIoExecutor() { public static DispatcherServices from( PartialDispatcherServicesWithJobPersistenceComponents partialDispatcherServicesWithJobPersistenceComponents, - JobManagerRunnerFactory jobManagerRunnerFactory) { + JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory) { return new DispatcherServices( partialDispatcherServicesWithJobPersistenceComponents.getConfiguration(), partialDispatcherServicesWithJobPersistenceComponents.getHighAvailabilityServices(), @@ -192,6 +203,7 @@ public static DispatcherServices from( partialDispatcherServicesWithJobPersistenceComponents.getJobGraphWriter(), partialDispatcherServicesWithJobPersistenceComponents.getJobResultStore(), jobManagerRunnerFactory, + cleanupRunnerFactory, partialDispatcherServicesWithJobPersistenceComponents.getIoExecutor()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java index 20dc32ba4e32b..ba800e9ff2a16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobResult; @@ -64,7 +65,8 @@ public MiniDispatcher createDispatcher( fencingToken, DispatcherServices.from( partialDispatcherServicesWithJobPersistenceComponents, - JobMasterServiceLeadershipRunnerFactory.INSTANCE), + JobMasterServiceLeadershipRunnerFactory.INSTANCE, + CheckpointResourcesCleanupRunnerFactory.INSTANCE), recoveredJobGraph, recoveredDirtyJob, dispatcherBootstrapFactory, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java index bd7bbad33f91c..e885958740bbb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; @@ -47,6 +48,7 @@ public StandaloneDispatcher createDispatcher( dispatcherBootstrapFactory, DispatcherServices.from( partialDispatcherServicesWithJobPersistenceComponents, - JobMasterServiceLeadershipRunnerFactory.INSTANCE)); + JobMasterServiceLeadershipRunnerFactory.INSTANCE, + CheckpointResourcesCleanupRunnerFactory.INSTANCE)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java index e9eb8aa55a43a..7794de3333a49 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; @@ -112,6 +113,7 @@ protected TestingDispatcher.Builder createTestingDispatcherBuilder() { .withJobGraphWriter(haServices.getJobGraphStore()) .withJobResultStore(haServices.getJobResultStore()) .withJobManagerRunnerFactory(JobMasterServiceLeadershipRunnerFactory.INSTANCE) + .withCleanupRunnerFactory(CheckpointResourcesCleanupRunnerFactory.INSTANCE) .withFatalErrorHandler(testingFatalErrorHandlerResource.getFatalErrorHandler()) .withBlobServer(blobServer); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 9f27fb391e5b1..1ca6f3349e2bd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -182,8 +182,8 @@ private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob() throws Exce private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob( int numBlockingJobManagerRunners) throws Exception { - final TestingJobManagerRunnerFactory testingJobManagerRunnerFactoryNG = - new TestingJobManagerRunnerFactory(numBlockingJobManagerRunners); + final TestingJobMasterServiceLeadershipRunnerFactory testingJobManagerRunnerFactoryNG = + new TestingJobMasterServiceLeadershipRunnerFactory(numBlockingJobManagerRunners); startDispatcher(testingJobManagerRunnerFactoryNG); submitJobAndWait(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 0600c39eff46e..04cfe41831641 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -504,8 +504,8 @@ public void testNoHistoryServerArchiveCreatedForSuspendedJob() throws Exception @Test public void testJobManagerRunnerInitializationFailureFailsJob() throws Exception { - final TestingJobManagerRunnerFactory testingJobManagerRunnerFactory = - new TestingJobManagerRunnerFactory(); + final TestingJobMasterServiceLeadershipRunnerFactory testingJobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); dispatcher = createAndStartDispatcher( @@ -666,8 +666,8 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { final FlinkException testException = new FlinkException("Test exception"); jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); - final TestingJobManagerRunnerFactory jobManagerRunnerFactory = - new TestingJobManagerRunnerFactory(); + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); dispatcher = createTestingDispatcherBuilder() .withJobManagerRunnerFactory(jobManagerRunnerFactory) @@ -1339,13 +1339,14 @@ public CompletableFuture closeAsync() { } private static final class BlockingJobManagerRunnerFactory - extends TestingJobManagerRunnerFactory { + extends TestingJobMasterServiceLeadershipRunnerFactory { @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; private TestingJobManagerRunner testingRunner; BlockingJobManagerRunnerFactory( @Nonnull ThrowingRunnable jobManagerRunnerCreationLatch) { + super(0); this.jobManagerRunnerCreationLatch = jobManagerRunnerCreationLatch; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index af40e71c2c8ec..04f559f5d9a3a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -22,10 +22,10 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; @@ -95,7 +95,8 @@ public class MiniDispatcherTest extends TestLogger { private TestingHighAvailabilityServices highAvailabilityServices; - private TestingJobManagerRunnerFactory testingJobManagerRunnerFactory; + private TestingJobMasterServiceLeadershipRunnerFactory testingJobManagerRunnerFactory; + private TestingCleanupRunnerFactory testingCleanupRunnerFactory; @BeforeClass public static void setupClass() throws IOException { @@ -119,7 +120,8 @@ public static void setupClass() throws IOException { public void setup() throws Exception { highAvailabilityServices = new TestingHighAvailabilityServicesBuilder().build(); - testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); + testingJobManagerRunnerFactory = new TestingJobMasterServiceLeadershipRunnerFactory(); + testingCleanupRunnerFactory = new TestingCleanupRunnerFactory(); } @AfterClass @@ -155,22 +157,19 @@ public void testSingleJobRecovery() throws Exception { /** Tests that the {@link MiniDispatcher} recovers the single job with which it was started. */ @Test public void testDirtyJobResultCleanup() throws Exception { - final OneShotLatch dispatcherBootstrapLatch = new OneShotLatch(); + final JobID jobId = new JobID(); final MiniDispatcher miniDispatcher = createMiniDispatcher( ClusterEntrypoint.ExecutionMode.DETACHED, null, - TestingJobResultStore.createSuccessfulJobResult(new JobID()), - (dispatcher, scheduledExecutor, errorHandler) -> { - dispatcherBootstrapLatch.trigger(); - return new NoOpDispatcherBootstrap(); - }); + TestingJobResultStore.createSuccessfulJobResult(jobId)); miniDispatcher.start(); try { - dispatcherBootstrapLatch.await(); - assertThat(testingJobManagerRunnerFactory.getQueueSize(), is(0)); + final TestingJobManagerRunner testingCleanupRunner = + testingCleanupRunnerFactory.takeCreatedJobManagerRunner(); + assertThat(testingCleanupRunner.getJobID(), is(jobId)); } finally { RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); } @@ -273,18 +272,13 @@ public void testShutdownIfJobCancelledInNormalMode() throws Exception { private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode executionMode) throws Exception { - return createMiniDispatcher( - executionMode, - jobGraph, - null, - (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap()); + return createMiniDispatcher(executionMode, jobGraph, null); } private MiniDispatcher createMiniDispatcher( ClusterEntrypoint.ExecutionMode executionMode, @Nullable JobGraph recoveredJobGraph, - @Nullable JobResult recoveredDirtyJob, - DispatcherBootstrapFactory dispatcherBootstrapFactory) + @Nullable JobResult recoveredDirtyJob) throws Exception { return new MiniDispatcher( rpcService, @@ -304,10 +298,11 @@ private MiniDispatcher createMiniDispatcher( highAvailabilityServices.getJobGraphStore(), highAvailabilityServices.getJobResultStore(), testingJobManagerRunnerFactory, + testingCleanupRunnerFactory, ForkJoinPool.commonPool()), recoveredJobGraph, recoveredDirtyJob, - dispatcherBootstrapFactory, + (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), executionMode); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java index 1b9c7474295b8..0b3587e3cbaf1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java @@ -22,8 +22,10 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultStore; @@ -98,6 +100,7 @@ private TestingDispatcher( HistoryServerArchivist historyServerArchivist, ExecutionGraphInfoStore executionGraphInfoStore, JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherOperationCaches dispatcherOperationCaches, JobManagerRunnerRegistry jobManagerRunnerRegistry, @@ -122,6 +125,7 @@ private TestingDispatcher( historyServerArchivist, executionGraphInfoStore, jobManagerRunnerFactory, + cleanupRunnerFactory, dispatcherBootstrapFactory, dispatcherOperationCaches, jobManagerRunnerRegistry, @@ -201,7 +205,8 @@ public static class Builder { private ExecutionGraphInfoStore executionGraphInfoStore = new MemoryExecutionGraphInfoStore(); private JobManagerRunnerFactory jobManagerRunnerFactory = - new TestingJobManagerRunnerFactory(0); + new TestingJobMasterServiceLeadershipRunnerFactory(); + private CleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); private DispatcherBootstrapFactory dispatcherBootstrapFactory = (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(); private DispatcherOperationCaches dispatcherOperationCaches = @@ -309,6 +314,11 @@ public Builder withJobManagerRunnerFactory( return this; } + public Builder withCleanupRunnerFactory(CleanupRunnerFactory cleanupRunnerFactory) { + this.cleanupRunnerFactory = cleanupRunnerFactory; + return this; + } + public Builder withDispatcherBootstrapFactory( DispatcherBootstrapFactory dispatcherBootstrapFactory) { this.dispatcherBootstrapFactory = dispatcherBootstrapFactory; @@ -364,6 +374,7 @@ public TestingDispatcher build() throws Exception { historyServerArchivist, executionGraphInfoStore, jobManagerRunnerFactory, + cleanupRunnerFactory, dispatcherBootstrapFactory, dispatcherOperationCaches, jobManagerRunnerRegistry, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java index 10d9c011b1ecb..4a70baae513ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java @@ -18,19 +18,10 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; -import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; -import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.Preconditions; -import javax.annotation.Nonnull; - import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicInteger; @@ -39,46 +30,30 @@ * Testing implementation of {@link JobManagerRunnerFactory} which returns a {@link * TestingJobManagerRunner}. */ -public class TestingJobManagerRunnerFactory implements JobManagerRunnerFactory { +public class TestingJobManagerRunnerFactory { private final BlockingQueue createdJobManagerRunner = new ArrayBlockingQueue<>(16); private final AtomicInteger numBlockingJobManagerRunners; - public TestingJobManagerRunnerFactory() { - this(0); - } - - public TestingJobManagerRunnerFactory(int numBlockingJobManagerRunners) { + protected TestingJobManagerRunnerFactory(int numBlockingJobManagerRunners) { this.numBlockingJobManagerRunners = new AtomicInteger(numBlockingJobManagerRunners); } - @Override - public TestingJobManagerRunner createJobManagerRunner( - JobGraph jobGraph, - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - JobManagerSharedServices jobManagerServices, - JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, - FatalErrorHandler fatalErrorHandler, - long initializationTimestamp) - throws Exception { + protected TestingJobManagerRunner offerTestingJobManagerRunner(JobID jobId) { final TestingJobManagerRunner testingJobManagerRunner = - createTestingJobManagerRunner(jobGraph); + createTestingJobManagerRunner(jobId); Preconditions.checkState( createdJobManagerRunner.offer(testingJobManagerRunner), "Unable to persist created the new runner."); return testingJobManagerRunner; } - @Nonnull - private TestingJobManagerRunner createTestingJobManagerRunner(JobGraph jobGraph) { + private TestingJobManagerRunner createTestingJobManagerRunner(JobID jobId) { final boolean blockingTermination = numBlockingJobManagerRunners.getAndDecrement() > 0; return TestingJobManagerRunner.newBuilder() - .setJobId(jobGraph.getJobID()) + .setJobId(jobId) .setBlockingTermination(blockingTermination) .build(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobMasterServiceLeadershipRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobMasterServiceLeadershipRunnerFactory.java new file mode 100644 index 0000000000000..0eb6d4c427052 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobMasterServiceLeadershipRunnerFactory.java @@ -0,0 +1,61 @@ +/* + * 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.flink.runtime.dispatcher; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; +import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; + +/** + * {@code TestingJobMasterServiceLeadershipRunnerFactory} implements {@code JobManagerRunnerFactory} + * providing a factory method usually used for {@link + * org.apache.flink.runtime.jobmaster.JobMasterServiceLeadershipRunner} creations. + */ +public class TestingJobMasterServiceLeadershipRunnerFactory extends TestingJobManagerRunnerFactory + implements JobManagerRunnerFactory { + + public TestingJobMasterServiceLeadershipRunnerFactory() { + this(0); + } + + public TestingJobMasterServiceLeadershipRunnerFactory(int numBlockingJobManagerRunners) { + super(numBlockingJobManagerRunners); + } + + @Override + public TestingJobManagerRunner createJobManagerRunner( + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + JobManagerSharedServices jobManagerServices, + JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, + FatalErrorHandler fatalErrorHandler, + long initializationTimestamp) + throws Exception { + return offerTestingJobManagerRunner(jobGraph.getJobID()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanupRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanupRunnerFactory.java new file mode 100644 index 0000000000000..223cc656abbec --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanupRunnerFactory.java @@ -0,0 +1,52 @@ +/* + * 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.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; + +import java.util.concurrent.Executor; + +/** + * {@code TestingCleanupRunnerFactory} implements {@link CleanupRunnerFactory} providing a factory + * method usually used for {@link CheckpointResourcesCleanupRunner} creations. + */ +public class TestingCleanupRunnerFactory extends TestingJobManagerRunnerFactory + implements CleanupRunnerFactory { + + public TestingCleanupRunnerFactory() { + super(0); + } + + @Override + public TestingJobManagerRunner create( + JobResult jobResult, + CheckpointRecoveryFactory checkpointRecoveryFactory, + Configuration configuration, + Executor cleanupExecutor) { + try { + return offerTestingJobManagerRunner(jobResult.getJobId()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index 2ea326b64d2e1..ff1b2b7f52a6e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -33,8 +33,10 @@ import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; -import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; +import org.apache.flink.runtime.dispatcher.TestingJobMasterServiceLeadershipRunnerFactory; import org.apache.flink.runtime.dispatcher.TestingPartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -167,11 +169,13 @@ private DispatcherGateway electLeaderAndRetrieveGateway(UUID firstLeaderSessionI @Test public void leaderChange_withBlockingJobManagerTermination_doesNotAffectNewLeader() throws Exception { - final TestingJobManagerRunnerFactory jobManagerRunnerFactory = - new TestingJobManagerRunnerFactory(1); + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(1); + final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); dispatcherRunnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner( - new TestingDispatcherFactory(jobManagerRunnerFactory)); + new TestingDispatcherFactory( + jobManagerRunnerFactory, cleanupRunnerFactory)); jobGraphStore = new SingleJobJobGraphStore(jobGraph); try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { @@ -221,9 +225,13 @@ public void leaderChange_withBlockingJobManagerTermination_doesNotAffectNewLeade private static class TestingDispatcherFactory implements DispatcherFactory { private final JobManagerRunnerFactory jobManagerRunnerFactory; + private final CleanupRunnerFactory cleanupRunnerFactory; - private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory) { + private TestingDispatcherFactory( + JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory) { this.jobManagerRunnerFactory = jobManagerRunnerFactory; + this.cleanupRunnerFactory = cleanupRunnerFactory; } @Override @@ -244,7 +252,8 @@ public Dispatcher createDispatcher( dispatcherBootstrapFactory, DispatcherServices.from( partialDispatcherServicesWithJobPersistenceComponents, - jobManagerRunnerFactory)); + jobManagerRunnerFactory, + cleanupRunnerFactory)); } } From 16afda44f757591a0a127368b58e7447aa98f61a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 13 Jan 2022 19:53:06 +0100 Subject: [PATCH 46/49] [FLINK-25432] Adds test cases for checking whether the CleanupRunner is created properly by the Dispatcher --- .../runtime/dispatcher/DispatcherTest.java | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 04cfe41831641..b8074fb6f5832 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.executiongraph.ErrorInfo; @@ -791,6 +792,51 @@ public void testFailingJobManagerRunnerCleanup() throws Exception { awaitStatus(dispatcherGateway, jobId, JobStatus.RUNNING); } + @Test + public void testJobCleanupWithoutRecoveredJobGraph() throws Exception { + final JobID jobIdOfRecoveredDirtyJobs = new JobID(); + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); + final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); + + final OneShotLatch dispatcherBootstrapLatch = new OneShotLatch(); + dispatcher = + createTestingDispatcherBuilder() + .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .withCleanupRunnerFactory(cleanupRunnerFactory) + .withRecoveredDirtyJobs( + Collections.singleton( + new JobResult.Builder() + .jobId(jobIdOfRecoveredDirtyJobs) + .applicationStatus(ApplicationStatus.SUCCEEDED) + .netRuntime(1) + .build())) + .withDispatcherBootstrapFactory( + (ignoredDispatcherGateway, + ignoredScheduledExecutor, + ignoredFatalErrorHandler) -> { + dispatcherBootstrapLatch.trigger(); + return new NoOpDispatcherBootstrap(); + }) + .build(); + + dispatcher.start(); + + dispatcherBootstrapLatch.await(); + + final TestingJobManagerRunner cleanupRunner = + cleanupRunnerFactory.takeCreatedJobManagerRunner(); + assertThat( + "The CleanupJobManagerRunner has the wrong job ID attached.", + cleanupRunner.getJobID(), + is(jobIdOfRecoveredDirtyJobs)); + + assertThat( + "No JobMaster should have been started.", + jobManagerRunnerFactory.getQueueSize(), + is(0)); + } + @Test public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { final TestingJobGraphStore submittedJobGraphStore = From dea89fc9981b2a65402bdc50959fcecc378d8970 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 6 Jan 2022 10:59:08 +0100 Subject: [PATCH 47/49] [FLINK-25432][runtime] Moves BlobServer cleanup tests into BlobServerCleanupTest Initially, the BlobServer cleanup with implicitly tested in DispatcherResourceCleanupTest. I moved the tests into BlobServerCleanupTest and added additional test cases. --- .../runtime/blob/BlobServerCleanupTest.java | 165 +++++++++++++++++- .../DispatcherResourceCleanupTest.java | 150 +++------------- 2 files changed, 179 insertions(+), 136 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java index 45ff27781c1f0..c74969c24bbab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java @@ -27,6 +27,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.function.BiConsumerWithException; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -41,8 +42,10 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Random; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentMap; @@ -58,6 +61,7 @@ import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; /** A few tests for the cleanup of transient BLOBs at the {@link BlobServer}. */ public class BlobServerCleanupTest extends TestLogger { @@ -73,6 +77,21 @@ private static byte[] createRandomData() { return randomData; } + private static BlobServer createTestInstance(String storageDirectoryPath, long cleanupInterval) + throws IOException { + return createTestInstance(storageDirectoryPath, cleanupInterval, new VoidBlobStore()); + } + + private static BlobServer createTestInstance( + String storageDirectoryPath, long cleanupInterval, BlobStore blobStore) + throws IOException { + final Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, storageDirectoryPath); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); + + return new BlobServer(config, new File(storageDirectoryPath), blobStore); + } + @Test public void testTransientBlobNoJobCleanup() throws IOException, InterruptedException, ExecutionException { @@ -102,13 +121,10 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) byte[] data = createRandomData(); byte[] data2 = createRandomData(); - Configuration config = new Configuration(); - config.setLong(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); - long cleanupLowerBound; try (BlobServer server = - new BlobServer(config, temporaryFolder, new VoidBlobStore())) { + createTestInstance(temporaryFolder.getAbsolutePath(), cleanupInterval)) { ConcurrentMap, Long> transientBlobExpiryTimes = server.getBlobExpiryTimes(); @@ -194,6 +210,139 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) } } + @Test + public void testLocalCleanup() throws Exception { + final TestingBlobStore blobStore = + createTestingBlobStoreBuilder() + .setDeleteAllFunction( + jobDataToDelete -> + fail( + "No deleteAll call is expected to be triggered but was for %s.", + jobDataToDelete)) + .createTestingBlobStore(); + testSuccessfulCleanup(new JobID(), BlobServer::localCleanup, blobStore); + } + + @Test + public void testGlobalCleanup() throws Exception { + final Set actuallyDeletedJobData = new HashSet<>(); + final JobID jobId = new JobID(); + final TestingBlobStore blobStore = + createTestingBlobStoreBuilder() + .setDeleteAllFunction( + jobDataToDelete -> { + actuallyDeletedJobData.add(jobDataToDelete); + return true; + }) + .createTestingBlobStore(); + testSuccessfulCleanup(jobId, BlobServer::globalCleanup, blobStore); + + assertThat(actuallyDeletedJobData).containsExactlyInAnyOrder(jobId); + } + + @Test + public void testGlobalCleanupUnsuccessfulInBlobStore() throws Exception { + final TestingBlobStore blobStore = + createTestingBlobStoreBuilder() + .setDeleteAllFunction(jobDataToDelete -> false) + .createTestingBlobStore(); + + testFailedCleanup( + new JobID(), + (testInstance, jobId) -> + assertThatThrownBy(() -> testInstance.globalCleanup(new JobID())) + .isInstanceOf(FlinkException.class), + blobStore); + } + + @Test + public void testGlobalCleanupFailureInBlobStore() throws Exception { + final RuntimeException actualException = new RuntimeException("Expected RuntimeException"); + final TestingBlobStore blobStore = + createTestingBlobStoreBuilder() + .setDeleteAllFunction( + jobDataToDelete -> { + throw actualException; + }) + .createTestingBlobStore(); + + testFailedCleanup( + new JobID(), + (testInstance, jobId) -> + assertThatThrownBy(() -> testInstance.globalCleanup(new JobID())) + .isEqualTo(actualException), + blobStore); + } + + private TestingBlobStoreBuilder createTestingBlobStoreBuilder() { + return new TestingBlobStoreBuilder() + .setDeleteFunction( + (jobId, blobKey) -> { + throw new UnsupportedOperationException( + "Deletion of individual blobs is not supported."); + }); + } + + private void testFailedCleanup( + JobID jobId, + BiConsumerWithException callback, + BlobStore blobStore) + throws Exception { + testCleanup(jobId, callback, blobStore, 2); + } + + private void testSuccessfulCleanup( + JobID jobId, + BiConsumerWithException callback, + BlobStore blobStore) + throws Exception { + testCleanup(jobId, callback, blobStore, 0); + } + + private void testCleanup( + JobID jobId, + BiConsumerWithException callback, + BlobStore blobStore, + int expectedFileCountAfterCleanup) + throws Exception { + final JobID otherJobId = new JobID(); + try (BlobServer testInstance = + createTestInstance( + temporaryFolder.getAbsolutePath(), Integer.MAX_VALUE, blobStore)) { + testInstance.start(); + + final BlobKey transientDataBlobKey = + put(testInstance, jobId, createRandomData(), TRANSIENT_BLOB); + final BlobKey otherTransientDataBlobKey = + put(testInstance, otherJobId, createRandomData(), TRANSIENT_BLOB); + + final BlobKey permanentDataBlobKey = + put(testInstance, jobId, createRandomData(), PERMANENT_BLOB); + final BlobKey otherPermanentDataBlobKey = + put(testInstance, otherJobId, createRandomData(), PERMANENT_BLOB); + + checkFilesExist( + jobId, + Arrays.asList(transientDataBlobKey, permanentDataBlobKey), + testInstance, + true); + checkFilesExist( + otherJobId, + Arrays.asList(otherTransientDataBlobKey, otherPermanentDataBlobKey), + testInstance, + true); + + callback.accept(testInstance, jobId); + + checkFileCountForJob(expectedFileCountAfterCleanup, jobId, testInstance); + checkFilesExist( + otherJobId, + Arrays.asList(otherTransientDataBlobKey, otherPermanentDataBlobKey), + testInstance, + true); + } + } + @Test public void testBlobServerExpiresRecoveredTransientJobBlob() throws Exception { runBlobServerExpiresRecoveredTransientBlob(new JobID()); @@ -207,8 +356,6 @@ public void testBlobServerExpiresRecoveredTransientNoJobBlob() throws Exception private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) throws Exception { final long cleanupInterval = 1L; - final Configuration configuration = new Configuration(); - configuration.set(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); final TransientBlobKey transientBlobKey = TestingBlobUtils.writeTransientBlob( @@ -216,7 +363,7 @@ private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) final File blob = BlobUtils.getStorageLocation(temporaryFolder, jobId, transientBlobKey); try (final BlobServer blobServer = - new BlobServer(configuration, temporaryFolder, new VoidBlobStore())) { + createTestInstance(temporaryFolder.getAbsolutePath(), cleanupInterval)) { CommonTestUtils.waitUntilCondition( () -> !blob.exists(), Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L)), @@ -236,7 +383,9 @@ public void testBlobServerRetainsJobs() throws Exception { TestingBlobUtils.writePermanentBlob(temporaryFolder.toPath(), jobId2, fileContent); try (final BlobServer blobServer = - new BlobServer(new Configuration(), temporaryFolder, new VoidBlobStore())) { + createTestInstance( + temporaryFolder.getAbsolutePath(), + BlobServerOptions.CLEANUP_INTERVAL.defaultValue())) { blobServer.retainJobs(Collections.singleton(jobId1)); assertThat(blobServer.getFile(jobId1, blobKey1)).hasBinaryContent(fileContent); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 1ca6f3349e2bd..4edca49465d55 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -24,15 +24,13 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.BlobStore; -import org.apache.flink.runtime.blob.PermanentBlobKey; -import org.apache.flink.runtime.blob.TestingBlobStore; +import org.apache.flink.runtime.blob.BlobUtils; import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.cleanup.TestingResourceCleanerFactory; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -57,9 +55,8 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Reference; import org.apache.flink.util.TestLogger; -import org.apache.flink.util.concurrent.FutureUtils; -import org.apache.flink.util.function.ThrowingConsumer; import org.hamcrest.core.IsInstanceOf; import org.junit.After; @@ -74,7 +71,6 @@ import javax.annotation.Nullable; -import java.io.File; import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; @@ -112,8 +108,6 @@ public class DispatcherResourceCleanupTest extends TestLogger { private JobGraph jobGraph; - private Configuration configuration; - private JobResultStore jobResultStore; private TestingHighAvailabilityServices highAvailabilityServices; @@ -126,12 +120,6 @@ public class DispatcherResourceCleanupTest extends TestLogger { private BlobServer blobServer; - private PermanentBlobKey permanentBlobKey; - - private File blobFile; - - private CompletableFuture storedHABlobFuture; - private CompletableFuture deleteAllHABlobsFuture; private CompletableFuture localCleanupFuture; private CompletableFuture globalCleanupFuture; private CompletableFuture cleanupJobHADataFuture; @@ -146,34 +134,20 @@ public void setup() throws Exception { jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); jobId = jobGraph.getJobID(); - configuration = new Configuration(); - highAvailabilityServices = new TestingHighAvailabilityServices(); clearedJobLatch = new OneShotLatch(); jobResultStore = new SingleJobResultStore(jobId, clearedJobLatch); cleanupJobHADataFuture = new CompletableFuture<>(); highAvailabilityServices.setGlobalCleanupFuture(cleanupJobHADataFuture); - storedHABlobFuture = new CompletableFuture<>(); - deleteAllHABlobsFuture = new CompletableFuture<>(); - - final TestingBlobStore testingBlobStore = - new TestingBlobStoreBuilder() - .setPutFunction( - (file, jobId, blobKey) -> storedHABlobFuture.complete(blobKey)) - .setDeleteAllFunction(deleteAllHABlobsFuture::complete) - .createTestingBlobStore(); - globalCleanupFuture = new CompletableFuture<>(); localCleanupFuture = new CompletableFuture<>(); blobServer = - new TestingBlobServer( - configuration, - temporaryFolder.newFolder(), - testingBlobStore, - jobId -> globalCleanupFuture.complete(jobId), - jobId -> localCleanupFuture.complete(jobId)); + BlobUtils.createBlobServer( + new Configuration(), + Reference.owned(temporaryFolder.newFolder()), + new TestingBlobStoreBuilder().createTestingBlobStore()); } private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob() throws Exception { @@ -191,15 +165,26 @@ private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob( } private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { + final JobManagerRunnerRegistry jobManagerRunnerRegistry = new JobManagerRunnerRegistry(2); dispatcher = TestingDispatcher.builder() .withRpcService(rpcService) .withHighAvailabilityServices(highAvailabilityServices) .withJobResultStore(jobResultStore) + .withJobManagerRunnerRegistry(jobManagerRunnerRegistry) .withBlobServer(blobServer) .withFatalErrorHandler( testingFatalErrorHandlerResource.getFatalErrorHandler()) .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .withResourceCleanerFactory( + new TestingResourceCleanerFactory() + // JobManagerRunnerRegistry needs to be added explicitly + // because cleaning it will trigger the closeAsync latch + // provided by TestingJobManagerRunner + .with(jobManagerRunnerRegistry) + .withGloballyCleanableResource( + globalCleanupFuture::complete) + .withLocallyCleanableResource(localCleanupFuture::complete)) .build(); dispatcher.start(); @@ -233,34 +218,11 @@ public void testBlobServerCleanupWhenJobFinished() throws Exception { // complete the job finishJob(jobManagerRunnerFactory.takeCreatedJobManagerRunner()); - assertThatHABlobsHaveBeenRemoved(); - } - - private void assertThatHABlobsHaveBeenRemoved() - throws InterruptedException, ExecutionException, TimeoutException { assertGlobalCleanupTriggered(jobId); - - // verify that we also cleared the BlobStore - assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); - - assertThat(blobFile.exists(), is(false)); } private CompletableFuture submitJob() { - try { - // upload a blob to the blob server - permanentBlobKey = blobServer.putPermanent(jobId, new byte[256]); - jobGraph.addUserJarBlobKey(permanentBlobKey); - blobFile = blobServer.getStorageLocation(jobId, permanentBlobKey); - - assertThat(blobFile.exists(), is(true)); - - // verify that we stored the blob also in the BlobStore - assertThat(storedHABlobFuture.join(), equalTo(permanentBlobKey)); - return dispatcherGateway.submitJob(jobGraph, timeout); - } catch (IOException ioe) { - return FutureUtils.completedExceptionally(ioe); - } + return dispatcherGateway.submitJob(jobGraph, timeout); } private void submitJobAndWait() { @@ -278,17 +240,6 @@ public void testBlobServerCleanupWhenJobNotFinished() throws Exception { suspendJob(testingJobManagerRunner); assertLocalCleanupTriggered(jobId); - assertThat(blobFile.exists(), is(false)); - - // verify that we did not clear the BlobStore - try { - deleteAllHABlobsFuture.get(50L, TimeUnit.MILLISECONDS); - fail("We should not delete the HA blobs."); - } catch (TimeoutException ignored) { - // expected - } - - assertThat(deleteAllHABlobsFuture.isDone(), is(false)); } /** Tests that the uploaded blobs are being cleaned up in case of a job submission failure. */ @@ -304,7 +255,7 @@ public void testBlobServerCleanupWhenJobSubmissionFails() throws Exception { assertThat(ee, FlinkMatchers.containsCause(JobSubmissionException.class)); } - assertThatHABlobsHaveBeenRemoved(); + assertGlobalCleanupTriggered(jobId); } @Test @@ -314,17 +265,6 @@ public void testBlobServerCleanupWhenClosingDispatcher() throws Exception { dispatcher.closeAsync().get(); assertLocalCleanupTriggered(jobId); - assertThat(blobFile.exists(), is(false)); - - // verify that we did not clear the BlobStore - try { - deleteAllHABlobsFuture.get(50L, TimeUnit.MILLISECONDS); - fail("We should not delete the HA blobs."); - } catch (TimeoutException ignored) { - // expected - } - - assertThat(deleteAllHABlobsFuture.isDone(), is(false)); } @Test @@ -357,7 +297,6 @@ public void testHACleanupWhenJobFinishedWhileClosingDispatcher() throws Exceptio dispatcherTerminationFuture.get(); assertGlobalCleanupTriggered(jobId); - assertThat(deleteAllHABlobsFuture.get(), is(jobId)); } /** @@ -454,7 +393,7 @@ public void testDuplicateJobSubmissionDoesNotDeleteJobMetaData() throws Exceptio finishJob(testingJobManagerRunnerFactoryNG.takeCreatedJobManagerRunner()); } - assertThatHABlobsHaveBeenRemoved(); + assertGlobalCleanupTriggered(jobId); } @Test @@ -503,8 +442,6 @@ private void terminateJobWithState( private void assertThatNoCleanupWasTriggered() { assertThat(globalCleanupFuture.isDone(), is(false)); assertThat(localCleanupFuture.isDone(), is(false)); - assertThat(deleteAllHABlobsFuture.isDone(), is(false)); - assertThat(blobFile.exists(), is(true)); } @Test @@ -614,7 +551,6 @@ public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); assertGlobalCleanupTriggered(jobId); - assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); } private void assertLocalCleanupTriggered(JobID jobId) @@ -625,7 +561,7 @@ private void assertLocalCleanupTriggered(JobID jobId) private void assertGlobalCleanupTriggered(JobID jobId) throws ExecutionException, InterruptedException, TimeoutException { - assertThat(localCleanupFuture.get(100, TimeUnit.MILLISECONDS), equalTo(jobId)); + assertThat(localCleanupFuture.isDone(), is(false)); assertThat(globalCleanupFuture.get(100, TimeUnit.MILLISECONDS), equalTo(jobId)); } @@ -700,48 +636,6 @@ public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws assertThat(dirtyJobFuture.get().getJobId(), is(jobId)); } - private static final class TestingBlobServer extends BlobServer { - - private final ThrowingConsumer globalCleanupConsumer; - private final ThrowingConsumer localCleanupConsumer; - - /** - * Instantiates a new BLOB server and binds it to a free network port. - * - * @param config Configuration to be used to instantiate the BlobServer - * @param blobStore BlobStore to store blobs persistently - * @param globalCleanupConsumer The consumer called along the actual {@link - * #globalCleanup(JobID)} call. - * @param localCleanupConsumer The consumer called along the actual {@link - * #localCleanup(JobID)} call. - * @throws IOException thrown if the BLOB server cannot bind to a free network port or if - * the (local or distributed) file storage cannot be created or is not usable - */ - public TestingBlobServer( - Configuration config, - File storageDirectory, - BlobStore blobStore, - ThrowingConsumer globalCleanupConsumer, - ThrowingConsumer localCleanupConsumer) - throws IOException { - super(config, storageDirectory, blobStore); - this.globalCleanupConsumer = globalCleanupConsumer; - this.localCleanupConsumer = localCleanupConsumer; - } - - @Override - public void globalCleanup(JobID jobId) throws Exception { - super.globalCleanup(jobId); - globalCleanupConsumer.accept(jobId); - } - - @Override - public void localCleanup(JobID jobId) throws IOException { - super.localCleanup(jobId); - localCleanupConsumer.accept(jobId); - } - } - private static final class QueueJobManagerRunnerFactory implements JobManagerRunnerFactory { private final Queue jobManagerRunners; From 9fbdb0953ea024f9c9a85fa6ea35abd99d19356c Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 6 Jan 2022 14:46:50 +0100 Subject: [PATCH 48/49] [FLINK-25432] Removes custom TestingHighAvailabilityServices --- .../DispatcherResourceCleanupTest.java | 19 ++----------------- 1 file changed, 2 insertions(+), 17 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 4edca49465d55..626b902547195 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -36,7 +36,6 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultEntry; import org.apache.flink.runtime.highavailability.JobResultStore; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobmaster.JobManagerRunner; @@ -110,8 +109,6 @@ public class DispatcherResourceCleanupTest extends TestLogger { private JobResultStore jobResultStore; - private TestingHighAvailabilityServices highAvailabilityServices; - private OneShotLatch clearedJobLatch; private TestingDispatcher dispatcher; @@ -122,7 +119,6 @@ public class DispatcherResourceCleanupTest extends TestLogger { private CompletableFuture localCleanupFuture; private CompletableFuture globalCleanupFuture; - private CompletableFuture cleanupJobHADataFuture; @BeforeClass public static void setupClass() { @@ -134,11 +130,8 @@ public void setup() throws Exception { jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); jobId = jobGraph.getJobID(); - highAvailabilityServices = new TestingHighAvailabilityServices(); clearedJobLatch = new OneShotLatch(); jobResultStore = new SingleJobResultStore(jobId, clearedJobLatch); - cleanupJobHADataFuture = new CompletableFuture<>(); - highAvailabilityServices.setGlobalCleanupFuture(cleanupJobHADataFuture); globalCleanupFuture = new CompletableFuture<>(); localCleanupFuture = new CompletableFuture<>(); @@ -169,7 +162,6 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th dispatcher = TestingDispatcher.builder() .withRpcService(rpcService) - .withHighAvailabilityServices(highAvailabilityServices) .withJobResultStore(jobResultStore) .withJobManagerRunnerRegistry(jobManagerRunnerRegistry) .withBlobServer(blobServer) @@ -402,8 +394,7 @@ public void testHaDataCleanupWhenJobFinished() throws Exception { TestingJobManagerRunner jobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); finishJob(jobManagerRunner); - JobID jobID = cleanupJobHADataFuture.get(2000, TimeUnit.MILLISECONDS); - assertThat(jobID, is(this.jobId)); + assertGlobalCleanupTriggered(jobId); } @Test @@ -412,13 +403,7 @@ public void testHaDataCleanupWhenJobNotFinished() throws Exception { TestingJobManagerRunner jobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); suspendJob(jobManagerRunner); - try { - cleanupJobHADataFuture.get(10L, TimeUnit.MILLISECONDS); - fail("We should not delete the HA data for job."); - } catch (TimeoutException ignored) { - // expected - } - assertThat(cleanupJobHADataFuture.isDone(), is(false)); + assertLocalCleanupTriggered(jobId); } private void finishJob(TestingJobManagerRunner takeCreatedJobManagerRunner) { From 965dc9aa30aa484aa704c3e9f028d31ca58f64b2 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 28 Jan 2022 18:06:43 +0100 Subject: [PATCH 49/49] [FLINK-25432][runtime] Moves cleanup test from DispatcherTest into DispatcherResourceCleanupTest --- .../DispatcherResourceCleanupTest.java | 115 ++++++++++++++ .../runtime/dispatcher/DispatcherTest.java | 144 +----------------- 2 files changed, 116 insertions(+), 143 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 626b902547195..f62e7f5da5722 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -43,6 +43,8 @@ import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -56,6 +58,7 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.Reference; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.ThrowingRunnable; import org.hamcrest.core.IsInstanceOf; import org.junit.After; @@ -68,6 +71,7 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; @@ -77,11 +81,15 @@ import java.util.Optional; import java.util.Queue; import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage; +import static org.apache.flink.runtime.dispatcher.AbstractDispatcherTest.awaitStatus; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -621,6 +629,113 @@ public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws assertThat(dirtyJobFuture.get().getJobId(), is(jobId)); } + /** Tests that a failing {@link JobManagerRunner} will be properly cleaned up. */ + @Test + public void testFailingJobManagerRunnerCleanup() throws Exception { + final FlinkException testException = new FlinkException("Test exception."); + final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(2); + + final BlockingJobManagerRunnerFactory blockingJobManagerRunnerFactory = + new BlockingJobManagerRunnerFactory( + () -> { + final Optional maybeException = queue.take(); + if (maybeException.isPresent()) { + throw maybeException.get(); + } + }); + + startDispatcher(blockingJobManagerRunnerFactory); + + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + // submit and fail during job master runner construction + queue.offer(Optional.of(testException)); + try { + dispatcherGateway.submitJob(jobGraph, Time.minutes(1)).get(); + fail("A FlinkException is expected"); + } catch (Throwable expectedException) { + assertThat(expectedException, containsCause(FlinkException.class)); + assertThat(expectedException, containsMessage(testException.getMessage())); + // make sure we've cleaned up in correct order (including HA) + assertGlobalCleanupTriggered(jobId); + } + + // don't fail this time + queue.offer(Optional.empty()); + // submit job again + dispatcherGateway.submitJob(jobGraph, Time.minutes(1L)).get(); + blockingJobManagerRunnerFactory.setJobStatus(JobStatus.RUNNING); + + // Ensure job is running + awaitStatus(dispatcherGateway, jobId, JobStatus.RUNNING); + } + + private static final class BlockingJobManagerRunnerFactory + extends TestingJobMasterServiceLeadershipRunnerFactory { + + @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; + private TestingJobManagerRunner testingRunner; + + BlockingJobManagerRunnerFactory( + @Nonnull ThrowingRunnable jobManagerRunnerCreationLatch) { + super(0); + this.jobManagerRunnerCreationLatch = jobManagerRunnerCreationLatch; + } + + @Override + public TestingJobManagerRunner createJobManagerRunner( + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + JobManagerSharedServices jobManagerSharedServices, + JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, + FatalErrorHandler fatalErrorHandler, + long initializationTimestamp) + throws Exception { + jobManagerRunnerCreationLatch.run(); + + this.testingRunner = + super.createJobManagerRunner( + jobGraph, + configuration, + rpcService, + highAvailabilityServices, + heartbeatServices, + jobManagerSharedServices, + jobManagerJobMetricGroupFactory, + fatalErrorHandler, + initializationTimestamp); + + TestingJobMasterGateway testingJobMasterGateway = + new TestingJobMasterGatewayBuilder() + .setRequestJobSupplier( + () -> + CompletableFuture.completedFuture( + new ExecutionGraphInfo( + ArchivedExecutionGraph + .createSparseArchivedExecutionGraph( + jobGraph.getJobID(), + jobGraph.getName(), + JobStatus.RUNNING, + null, + null, + 1337)))) + .build(); + testingRunner.completeJobMasterGatewayFuture(testingJobMasterGateway); + return testingRunner; + } + + public void setJobStatus(JobStatus newStatus) { + Preconditions.checkState( + testingRunner != null, + "JobManagerRunner must be created before this method is available"); + this.testingRunner.setJobStatus(newStatus); + } + } + private static final class QueueJobManagerRunnerFactory implements JobManagerRunnerFactory { private final Queue jobManagerRunners; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index b8074fb6f5832..5d3ffecb8509c 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -59,7 +59,6 @@ import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceProcessFactory; import org.apache.flink.runtime.jobmaster.factories.TestingJobMasterServiceFactory; -import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -88,7 +87,6 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TimeUtils; -import org.apache.flink.util.function.ThrowingRunnable; import org.assertj.core.api.Assertions; import org.hamcrest.Matchers; @@ -109,11 +107,9 @@ import java.nio.file.Paths; import java.time.Duration; import java.util.ArrayDeque; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.Optional; import java.util.Queue; import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; @@ -126,12 +122,10 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; -import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -721,77 +715,6 @@ public void testThatDirtilyFinishedJobsNotBeingRetriggered() throws Exception { .build(); } - /** Tests that a failing {@link JobManagerRunner} will be properly cleaned up. */ - @Test - public void testFailingJobManagerRunnerCleanup() throws Exception { - final FlinkException testException = new FlinkException("Test exception."); - final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(2); - - final BlockingJobManagerRunnerFactory blockingJobManagerRunnerFactory = - new BlockingJobManagerRunnerFactory( - () -> { - final Optional maybeException = queue.take(); - if (maybeException.isPresent()) { - throw maybeException.get(); - } - }); - - final BlockingQueue cleanUpEvents = new LinkedBlockingQueue<>(); - - // Track cleanup - ha-services - final CompletableFuture cleanupJobData = new CompletableFuture<>(); - haServices.setGlobalCleanupFuture(cleanupJobData); - cleanupJobData.thenAccept(jobId -> cleanUpEvents.add(CLEANUP_HA_SERVICES)); - - // Track cleanup - job-graph - final TestingJobGraphStore jobGraphStore = - TestingJobGraphStore.newBuilder() - .setLocalCleanupConsumer( - jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_RELEASE)) - .setGlobalCleanupConsumer( - jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_REMOVE)) - .build(); - jobGraphStore.start(null); - haServices.setJobGraphStore(jobGraphStore); - - // Track cleanup - job result store - haServices.setJobResultStore( - TestingJobResultStore.builder() - .withMarkResultAsCleanConsumer( - jobID -> cleanUpEvents.add(CLEANUP_JOB_RESULT_STORE)) - .build()); - - dispatcher = - createAndStartDispatcher( - heartbeatServices, haServices, blockingJobManagerRunnerFactory); - - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - // submit and fail during job master runner construction - queue.offer(Optional.of(testException)); - try { - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - fail("A FlinkException is expected"); - } catch (Throwable expectedException) { - assertThat(expectedException, containsCause(FlinkException.class)); - assertThat(expectedException, containsMessage(testException.getMessage())); - // make sure we've cleaned up in correct order (including HA) - assertThat( - new ArrayList<>(cleanUpEvents), - containsInAnyOrder(CLEANUP_JOB_GRAPH_REMOVE, CLEANUP_HA_SERVICES)); - } - - // don't fail this time - queue.offer(Optional.empty()); - // submit job again - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - blockingJobManagerRunnerFactory.setJobStatus(JobStatus.RUNNING); - - // Ensure job is running - awaitStatus(dispatcherGateway, jobId, JobStatus.RUNNING); - } - @Test public void testJobCleanupWithoutRecoveredJobGraph() throws Exception { final JobID jobIdOfRecoveredDirtyJobs = new JobID(); @@ -1384,71 +1307,6 @@ public CompletableFuture closeAsync() { } } - private static final class BlockingJobManagerRunnerFactory - extends TestingJobMasterServiceLeadershipRunnerFactory { - - @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; - private TestingJobManagerRunner testingRunner; - - BlockingJobManagerRunnerFactory( - @Nonnull ThrowingRunnable jobManagerRunnerCreationLatch) { - super(0); - this.jobManagerRunnerCreationLatch = jobManagerRunnerCreationLatch; - } - - @Override - public TestingJobManagerRunner createJobManagerRunner( - JobGraph jobGraph, - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - JobManagerSharedServices jobManagerSharedServices, - JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, - FatalErrorHandler fatalErrorHandler, - long initializationTimestamp) - throws Exception { - jobManagerRunnerCreationLatch.run(); - - this.testingRunner = - super.createJobManagerRunner( - jobGraph, - configuration, - rpcService, - highAvailabilityServices, - heartbeatServices, - jobManagerSharedServices, - jobManagerJobMetricGroupFactory, - fatalErrorHandler, - initializationTimestamp); - - TestingJobMasterGateway testingJobMasterGateway = - new TestingJobMasterGatewayBuilder() - .setRequestJobSupplier( - () -> - CompletableFuture.completedFuture( - new ExecutionGraphInfo( - ArchivedExecutionGraph - .createSparseArchivedExecutionGraph( - jobGraph.getJobID(), - jobGraph.getName(), - JobStatus.RUNNING, - null, - null, - 1337)))) - .build(); - testingRunner.completeJobMasterGatewayFuture(testingJobMasterGateway); - return testingRunner; - } - - public void setJobStatus(JobStatus newStatus) { - Preconditions.checkState( - testingRunner != null, - "JobManagerRunner must be created before this method is available"); - this.testingRunner.setJobStatus(newStatus); - } - } - private static final class InitializationTimestampCapturingJobManagerRunnerFactory implements JobManagerRunnerFactory { private final BlockingQueue initializationTimestampQueue;