Skip to content

Commit

Permalink
[FLINK-18656][network,metrics] Fix startDelay metric for unaligned ch…
Browse files Browse the repository at this point in the history
…eckpoints

Before this fix, startDelay metric was always set to zero for unaligned checkpoints.
  • Loading branch information
pnowojski committed Jul 24, 2020
1 parent 13a67e2 commit 77a6fb4
Show file tree
Hide file tree
Showing 2 changed files with 48 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -325,6 +325,7 @@ private synchronized void handleNewCheckpoint(CheckpointBarrier barrier) throws
allBarriersReceivedFuture.completeExceptionally(exception);
}

handler.markCheckpointStart(barrier.getTimestamp());
currentReceivedCheckpointId = barrierId;
storeNewBuffers.entrySet().forEach(storeNewBuffer -> storeNewBuffer.setValue(true));
numBarriersReceived = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction;
import org.apache.flink.util.function.ThrowingRunnable;

import org.hamcrest.Matchers;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
Expand All @@ -64,6 +65,7 @@
import static org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter.NO_OP;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;

/**
Expand Down Expand Up @@ -227,6 +229,42 @@ public void testMultiChannelWithBarriers() throws Exception {
assertInflightData();
}

@Test
public void testMetrics() throws Exception {
ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(1);
inputGate = createInputGate(3, handler);
int bufferSize = 100;
long checkpointId = 1;
long sleepTime = 10;

long checkpointBarrierCreation = System.currentTimeMillis();

Thread.sleep(sleepTime);

addSequence(inputGate,
createBuffer(0, bufferSize), createBuffer(1, bufferSize), createBuffer(2, bufferSize),
createBarrier(checkpointId, 1, checkpointBarrierCreation),
createBuffer(0, bufferSize), createBuffer(1, bufferSize), createBuffer(2, bufferSize),
createBarrier(checkpointId, 0),
createBuffer(0, bufferSize), createBuffer(1, bufferSize), createBuffer(2, bufferSize));

long startDelay = System.currentTimeMillis() - checkpointBarrierCreation;

Thread.sleep(sleepTime);

addSequence(inputGate,
createBarrier(checkpointId, 2),
createBuffer(0, bufferSize), createBuffer(1, bufferSize), createBuffer(2, bufferSize),
createEndOfPartition(0), createEndOfPartition(1), createEndOfPartition(2));

assertEquals(checkpointId, inputGate.getCheckpointBarrierHandler().getLatestCheckpointId());
assertThat(inputGate.getCheckpointStartDelayNanos() / 1_000_000, Matchers.greaterThanOrEqualTo(sleepTime));
assertThat(inputGate.getCheckpointStartDelayNanos() / 1_000_000, Matchers.lessThanOrEqualTo(startDelay));

while (inputGate.pollNext().isPresent()) {
}
}

@Test
public void testMultiChannelTrailingInflightData() throws Exception {
ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(1);
Expand Down Expand Up @@ -629,11 +667,15 @@ public void testEndOfStreamWithPendingCheckpoint() throws Exception {
// ------------------------------------------------------------------------

private BufferOrEvent createBarrier(long checkpointId, int channel) {
return createBarrier(checkpointId, channel, System.currentTimeMillis());
}

private BufferOrEvent createBarrier(long checkpointId, int channel, long timestamp) {
sizeCounter++;
return new BufferOrEvent(
new CheckpointBarrier(
checkpointId,
System.currentTimeMillis(),
timestamp,
CheckpointOptions.forCheckpointWithDefaultLocation()),
new InputChannelInfo(0, channel));
}
Expand All @@ -643,6 +685,10 @@ private BufferOrEvent createCancellationBarrier(long checkpointId, int channel)
return new BufferOrEvent(new CancelCheckpointMarker(checkpointId), new InputChannelInfo(0, channel));
}

private BufferOrEvent createBuffer(int channel, int size) {
return new BufferOrEvent(TestBufferFactory.createBuffer(size), new InputChannelInfo(0, channel));
}

private BufferOrEvent createBuffer(int channel) {
final int size = sizeCounter++;
return new BufferOrEvent(TestBufferFactory.createBuffer(size), new InputChannelInfo(0, channel));
Expand Down

0 comments on commit 77a6fb4

Please sign in to comment.