-
Notifications
You must be signed in to change notification settings - Fork 4.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[BEAM-94] Add UnboundedCountingInput#withRate #17
Changes from all commits
e2f9688
9470cf2
2dc8d5a
3708e03
ec37b48
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
package com.google.cloud.dataflow.sdk.io; | ||
|
||
import static com.google.common.base.Preconditions.checkArgument; | ||
import static com.google.common.base.Preconditions.checkNotNull; | ||
|
||
import com.google.cloud.dataflow.sdk.coders.AvroCoder; | ||
import com.google.cloud.dataflow.sdk.coders.Coder; | ||
|
@@ -29,6 +30,7 @@ | |
import com.google.cloud.dataflow.sdk.values.PCollection; | ||
import com.google.common.collect.ImmutableList; | ||
|
||
import org.joda.time.Duration; | ||
import org.joda.time.Instant; | ||
|
||
import java.io.IOException; | ||
|
@@ -81,6 +83,14 @@ public static BoundedSource<Long> upTo(long numElements) { | |
return new BoundedCountingSource(0, numElements); | ||
} | ||
|
||
/** | ||
* Create a new {@link UnboundedCountingSource}. | ||
*/ | ||
// package-private to return a typed UnboundedCountingSource rather than the UnboundedSource type. | ||
static UnboundedCountingSource createUnbounded() { | ||
return new UnboundedCountingSource(0, 1, 1L, Duration.ZERO, new NowTimestampFn()); | ||
} | ||
|
||
/** | ||
* Creates an {@link UnboundedSource} that will produce numbers starting from {@code 0} up to | ||
* {@link Long#MAX_VALUE}. | ||
|
@@ -113,7 +123,7 @@ public static UnboundedSource<Long, CounterMark> unbounded() { | |
@Deprecated | ||
public static UnboundedSource<Long, CounterMark> unboundedWithTimestampFn( | ||
SerializableFunction<Long, Instant> timestampFn) { | ||
return new UnboundedCountingSource(0, 1, timestampFn); | ||
return new UnboundedCountingSource(0, 1, 1L, Duration.ZERO, timestampFn); | ||
} | ||
|
||
///////////////////////////////////////////////////////////////////////////////////////////// | ||
|
@@ -226,11 +236,15 @@ public void close() throws IOException {} | |
/** | ||
* An implementation of {@link CountingSource} that produces an unbounded {@link PCollection}. | ||
*/ | ||
private static class UnboundedCountingSource extends UnboundedSource<Long, CounterMark> { | ||
static class UnboundedCountingSource extends UnboundedSource<Long, CounterMark> { | ||
/** The first number (>= 0) generated by this {@link UnboundedCountingSource}. */ | ||
private final long start; | ||
/** The interval between numbers generated by this {@link UnboundedCountingSource}. */ | ||
private final long stride; | ||
/** The number of elements to produce each period. */ | ||
private final long elementsPerPeriod; | ||
/** The time between producing numbers from this {@link UnboundedCountingSource}. */ | ||
private final Duration period; | ||
/** The function used to produce timestamps for the generated elements. */ | ||
private final SerializableFunction<Long, Instant> timestampFn; | ||
|
||
|
@@ -243,13 +257,45 @@ private static class UnboundedCountingSource extends UnboundedSource<Long, Count | |
* | ||
* <p>Note that the timestamps produced by {@code timestampFn} may not decrease. | ||
*/ | ||
public UnboundedCountingSource( | ||
long start, long stride, SerializableFunction<Long, Instant> timestampFn) { | ||
private UnboundedCountingSource( | ||
long start, | ||
long stride, | ||
long elementsPerPeriod, | ||
Duration period, | ||
SerializableFunction<Long, Instant> timestampFn) { | ||
this.start = start; | ||
this.stride = stride; | ||
checkArgument( | ||
elementsPerPeriod > 0L, | ||
"Must produce at least one element per period, got %s", | ||
elementsPerPeriod); | ||
this.elementsPerPeriod = elementsPerPeriod; | ||
checkArgument( | ||
period.getMillis() >= 0L, "Must have a non-negative period length, got %s", period); | ||
this.period = period; | ||
this.timestampFn = timestampFn; | ||
} | ||
|
||
/** | ||
* Returns an {@link UnboundedCountingSource} like this one with the specified period. Elements | ||
* will be produced with an interval between them equal to the period. | ||
*/ | ||
public UnboundedCountingSource withRate(long elementsPerPeriod, Duration period) { | ||
return new UnboundedCountingSource(start, stride, elementsPerPeriod, period, timestampFn); | ||
} | ||
|
||
/** | ||
* Returns an {@link UnboundedCountingSource} like this one where the timestamp of output | ||
* elements are supplied by the specified function. | ||
* | ||
* <p>Note that timestamps produced by {@code timestampFn} may not decrease. | ||
*/ | ||
public UnboundedCountingSource withTimestampFn( | ||
SerializableFunction<Long, Instant> timestampFn) { | ||
checkNotNull(timestampFn); | ||
return new UnboundedCountingSource(start, stride, elementsPerPeriod, period, timestampFn); | ||
} | ||
|
||
/** | ||
* Splits an unbounded source {@code desiredNumSplits} ways by giving each split every | ||
* {@code desiredNumSplits}th element that this {@link UnboundedCountingSource} | ||
|
@@ -270,7 +316,9 @@ public List<? extends UnboundedSource<Long, CountingSource.CounterMark>> generat | |
for (int i = 0; i < desiredNumSplits; ++i) { | ||
// Starts offset by the original stride. Using Javadoc example, this generates starts of | ||
// 0, 2, and 4. | ||
splits.add(new UnboundedCountingSource(start + i * stride, newStride, timestampFn)); | ||
splits.add( | ||
new UnboundedCountingSource( | ||
start + i * stride, newStride, elementsPerPeriod, period, timestampFn)); | ||
} | ||
return splits.build(); | ||
} | ||
|
@@ -304,6 +352,7 @@ private static class UnboundedCountingReader extends UnboundedReader<Long> { | |
private UnboundedCountingSource source; | ||
private long current; | ||
private Instant currentTimestamp; | ||
private Instant firstStarted; | ||
|
||
public UnboundedCountingReader(UnboundedCountingSource source, CounterMark mark) { | ||
this.source = source; | ||
|
@@ -313,11 +362,15 @@ public UnboundedCountingReader(UnboundedCountingSource source, CounterMark mark) | |
this.current = source.start - source.stride; | ||
} else { | ||
this.current = mark.getLastEmitted(); | ||
this.firstStarted = mark.getStartTime(); | ||
} | ||
} | ||
|
||
@Override | ||
public boolean start() throws IOException { | ||
if (firstStarted == null) { | ||
this.firstStarted = Instant.now(); | ||
} | ||
return advance(); | ||
} | ||
|
||
|
@@ -327,19 +380,33 @@ public boolean advance() throws IOException { | |
if (Long.MAX_VALUE - source.stride < current) { | ||
return false; | ||
} | ||
current += source.stride; | ||
long nextValue = current + source.stride; | ||
if (expectedValue() < nextValue) { | ||
return false; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Return false will indicate no more input available. (Based on advance() javadoc) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's the correct behavior - if we're not at or beyond the next time we should produce, we have no input available, but may in the future. |
||
} | ||
current = nextValue; | ||
currentTimestamp = source.timestampFn.apply(current); | ||
return true; | ||
} | ||
|
||
private long expectedValue() { | ||
if (source.period.getMillis() == 0L) { | ||
return Long.MAX_VALUE; | ||
} | ||
double periodsElapsed = | ||
(Instant.now().getMillis() - firstStarted.getMillis()) | ||
/ (double) source.period.getMillis(); | ||
return (long) (source.elementsPerPeriod * periodsElapsed); | ||
} | ||
|
||
@Override | ||
public Instant getWatermark() { | ||
return source.timestampFn.apply(current); | ||
} | ||
|
||
@Override | ||
public CounterMark getCheckpointMark() { | ||
return new CounterMark(current); | ||
return new CounterMark(current, firstStarted); | ||
} | ||
|
||
@Override | ||
|
@@ -359,6 +426,12 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { | |
|
||
@Override | ||
public void close() throws IOException {} | ||
|
||
@Override | ||
public long getSplitBacklogBytes() { | ||
long expected = expectedValue(); | ||
return Math.max(0L, 8 * (expected - current) / source.stride); | ||
} | ||
} | ||
|
||
/** | ||
|
@@ -369,12 +442,14 @@ public void close() throws IOException {} | |
public static class CounterMark implements UnboundedSource.CheckpointMark { | ||
/** The last value emitted. */ | ||
private final long lastEmitted; | ||
private final Instant startTime; | ||
|
||
/** | ||
* Creates a checkpoint mark reflecting the last emitted value. | ||
*/ | ||
public CounterMark(long lastEmitted) { | ||
public CounterMark(long lastEmitted, Instant startTime) { | ||
this.lastEmitted = lastEmitted; | ||
this.startTime = startTime; | ||
} | ||
|
||
/** | ||
|
@@ -384,11 +459,19 @@ public long getLastEmitted() { | |
return lastEmitted; | ||
} | ||
|
||
/** | ||
* Returns the time the reader was started. | ||
*/ | ||
public Instant getStartTime() { | ||
return startTime; | ||
} | ||
|
||
///////////////////////////////////////////////////////////////////////////////////// | ||
|
||
@SuppressWarnings("unused") // For AvroCoder | ||
private CounterMark() { | ||
this.lastEmitted = 0L; | ||
this.startTime = Instant.now(); | ||
} | ||
|
||
@Override | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
new blank lines necessary?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
no, reverted.