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 28, 2020
1 parent c3bf5f9 commit 5dccc99
Show file tree
Hide file tree
Showing 4 changed files with 106 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,11 @@ public long getAlignmentDurationNanos() {
return activeHandler.getAlignmentDurationNanos();
}

@Override
public long getCheckpointStartDelayNanos() {
return activeHandler.getCheckpointStartDelayNanos();
}

@Override
public boolean hasInflightData(long checkpointId, InputChannelInfo channelInfo) {
// should only be called for unaligned checkpoint
Expand Down
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 @@ -47,6 +47,9 @@
import static org.apache.flink.runtime.checkpoint.CheckpointType.CHECKPOINT;
import static org.apache.flink.runtime.checkpoint.CheckpointType.SAVEPOINT;
import static org.apache.flink.runtime.io.network.api.serialization.EventSerializer.toBuffer;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
Expand Down Expand Up @@ -83,6 +86,43 @@ public void testAlternation() throws Exception {
assertEquals(barriers, target.triggeredCheckpoints);
}

@Test
public void testMetricsAlternation() throws Exception {
int numChannels = 2;
TestInvokable target = new TestInvokable();
CheckpointedInputGate gate = buildGate(target, numChannels);

long checkpoint1CreationTime = System.currentTimeMillis() - 10;
sendBarrier(1, checkpoint1CreationTime, CHECKPOINT, gate, 0);
Thread.sleep(10);
sendBarrier(1, checkpoint1CreationTime, CHECKPOINT, gate, 1);
assertMetrics(gate.getCheckpointBarrierHandler(), 1L, 0L, 10_000_000L);

long checkpoint2CreationTime = System.currentTimeMillis() - 5;
sendBarrier(2, checkpoint2CreationTime, SAVEPOINT, gate, 0);
assertMetrics(gate.getCheckpointBarrierHandler(), 2L, 0L, 5_000_000L);
Thread.sleep(5);
sendBarrier(2, checkpoint2CreationTime, SAVEPOINT, gate, 1);
assertMetrics(gate.getCheckpointBarrierHandler(), 2L, 5_000_000L, 5_000_000L);

long checkpoint3CreationTime = System.currentTimeMillis() - 7;
sendBarrier(3, checkpoint3CreationTime, CHECKPOINT, gate, 0);
assertMetrics(gate.getCheckpointBarrierHandler(), 3L, 0L, 7_000_000L);
Thread.sleep(7);
sendBarrier(3, checkpoint2CreationTime, CHECKPOINT, gate, 1);
assertMetrics(gate.getCheckpointBarrierHandler(), 3L, 0L, 7_000_000L);
}

private void assertMetrics(
CheckpointBarrierHandler checkpointBarrierHandler,
long latestCheckpointId,
long alignmentDurationNanos,
long startDelayNanos) {
assertThat(checkpointBarrierHandler.getLatestCheckpointId(), equalTo(latestCheckpointId));
assertThat(checkpointBarrierHandler.getAlignmentDurationNanos(), greaterThanOrEqualTo(alignmentDurationNanos));
assertThat(checkpointBarrierHandler.getCheckpointStartDelayNanos(), greaterThanOrEqualTo(startDelayNanos));
}

@Test
public void testPreviousHandlerReset() throws Exception {
SingleInputGate inputGate = new SingleInputGateBuilder().setNumberOfChannels(2).build();
Expand Down Expand Up @@ -197,8 +237,15 @@ private void testBarrierHandling(CheckpointType checkpointType) throws Exception
}
}

private void sendBarrier(long id, CheckpointType type, TestInputChannel channel, CheckpointedInputGate gate) throws Exception {
channel.read(barrier(id, type).retainBuffer());
private void sendBarrier(long barrierId, long barrierCreationTime, CheckpointType type, CheckpointedInputGate gate, int channelId) throws Exception {
TestInputChannel channel = (TestInputChannel) gate.getChannel(channelId);
channel.read(barrier(barrierId, type, barrierCreationTime).retainBuffer());
while (gate.pollNext().isPresent()) {
}
}

private void sendBarrier(long barrierId, CheckpointType type, TestInputChannel channel, CheckpointedInputGate gate) throws Exception {
channel.read(barrier(barrierId, type).retainBuffer());
while (gate.pollNext().isPresent()) {
}
}
Expand All @@ -213,10 +260,14 @@ private static AlternatingCheckpointBarrierHandler barrierHandler(SingleInputGat
target);
}

private Buffer barrier(long id, CheckpointType checkpointType) throws IOException {
private Buffer barrier(long barrierId, CheckpointType checkpointType) throws IOException {
return barrier(barrierId, checkpointType, System.currentTimeMillis());
}

private Buffer barrier(long barrierId, CheckpointType checkpointType, long barrierTimestamp) throws IOException {
return toBuffer(new CheckpointBarrier(
id,
System.currentTimeMillis(),
barrierId,
barrierTimestamp,
new CheckpointOptions(checkpointType, CheckpointStorageLocationReference.getDefault(), true, true)));
}

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,39 @@ 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));
}

@Test
public void testMultiChannelTrailingInflightData() throws Exception {
ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler(1);
Expand Down Expand Up @@ -629,11 +664,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 +682,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 5dccc99

Please sign in to comment.