Skip to content

Commit

Permalink
Change PeriodicSequence to report backlog accurately
Browse files Browse the repository at this point in the history
  • Loading branch information
scwhittle committed Sep 19, 2024
1 parent ebcb2db commit 459ad40
Show file tree
Hide file tree
Showing 2 changed files with 56 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
Expand Down Expand Up @@ -164,7 +165,9 @@ public void checkDone() throws IllegalStateException {

@Override
public IsBounded isBounded() {
return IsBounded.BOUNDED;
return range.getTo() == BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()
? IsBounded.UNBOUNDED
: IsBounded.BOUNDED;
}

@Override
Expand Down Expand Up @@ -213,6 +216,13 @@ public RestrictionTracker.TruncateResult<OffsetRange> truncate() {
return null;
}

@GetSize
public double getSize(
@Element SequenceDefinition sequence, @Restriction OffsetRange offsetRange) {
long nowMilliSec = Instant.now().getMillis();
return sequenceBacklogBytes(sequence.durationMilliSec, nowMilliSec, offsetRange);
}

@ProcessElement
public ProcessContinuation processElement(
@Element SequenceDefinition srcElement,
Expand Down Expand Up @@ -257,4 +267,26 @@ public ProcessContinuation processElement(
public PCollection<Instant> expand(PCollection<SequenceDefinition> input) {
return input.apply(ParDo.of(new PeriodicSequenceFn()));
}

private static final int ENCODED_INSTANT_BYTES = 8;

private static long ceilDiv(long a, long b) {
long result = Math.floorDiv(a, b);
if (a % b != 0) {
++result;
}
return result;
}

@VisibleForTesting
static long sequenceBacklogBytes(
long durationMilliSec, long nowMilliSec, OffsetRange offsetRange) {
// Find the # of outputs expected for overlap of offsetRange and [-inf, now)
long start = ceilDiv(offsetRange.getFrom(), durationMilliSec);
long end = ceilDiv(Math.min(nowMilliSec, offsetRange.getTo() - 1), durationMilliSec);
if (start >= end) {
return 0;
}
return ENCODED_INSTANT_BYTES * (end - start);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.beam.sdk.io.range.OffsetRange;
import org.apache.beam.sdk.testing.NeedsRunner;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
Expand All @@ -42,7 +43,6 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Streams;
import org.joda.time.Duration;
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
Expand All @@ -51,8 +51,6 @@
/** Tests for PeriodicSequence. */
@RunWith(JUnit4.class)
public class PeriodicSequenceTest {
@Rule public transient TestPipeline p = TestPipeline.create();

public static class ExtractTsDoFn<InputT>
extends DoFn<InputT, TimestampedValue<KV<InputT, Instant>>> {

Expand All @@ -76,6 +74,7 @@ public void processElement(
UsesUnboundedSplittableParDo.class
})
public void testOutputsProperElements() {
TestPipeline p = TestPipeline.create();
Instant startTime = Instant.now().plus(Duration.standardSeconds(2));
Duration interval = Duration.millis(250);
long intervalMillis = interval.getMillis();
Expand Down Expand Up @@ -136,4 +135,25 @@ public void testOutputsProperElements() {

p.run().waitUntilFinish();
}

@Test
public void testBacklogBytes() {
assertEquals(
0, PeriodicSequence.sequenceBacklogBytes(10, 100, new OffsetRange(100, Long.MAX_VALUE)));
assertEquals(
8, PeriodicSequence.sequenceBacklogBytes(10, 100, new OffsetRange(90, Long.MAX_VALUE)));
assertEquals(
0, PeriodicSequence.sequenceBacklogBytes(10, 100, new OffsetRange(91, Long.MAX_VALUE)));
assertEquals(
8, PeriodicSequence.sequenceBacklogBytes(10, 100, new OffsetRange(89, Long.MAX_VALUE)));
assertEquals(
16, PeriodicSequence.sequenceBacklogBytes(10, 101, new OffsetRange(81, Long.MAX_VALUE)));
assertEquals(
8 * 10000 / 100,
PeriodicSequence.sequenceBacklogBytes(100, 10000, new OffsetRange(0, Long.MAX_VALUE)));
assertEquals(
0, PeriodicSequence.sequenceBacklogBytes(10, 10000, new OffsetRange(10011, 10025)));
assertEquals(
8, PeriodicSequence.sequenceBacklogBytes(10, 10100, new OffsetRange(10011, 10025)));
}
}

0 comments on commit 459ad40

Please sign in to comment.