Skip to content

Conversation

@rehmanmuradali
Copy link
Contributor

@rehmanmuradali rehmanmuradali commented Jan 17, 2020

Please add a meaningful description for your change here


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

Post-Commit Tests Status (on master branch)

Lang SDK Apex Dataflow Flink Gearpump Samza Spark
Go Build Status --- --- Build Status --- --- Build Status
Java Build Status Build Status Build Status Build Status
Build Status
Build Status
Build Status Build Status Build Status
Build Status
Build Status
Python Build Status
Build Status
Build Status
Build Status
--- Build Status
Build Status
Build Status
Build Status
--- --- Build Status
XLang --- --- --- Build Status --- --- ---

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable Build Status Build Status
Build Status
Build Status Build Status
Portable --- Build Status --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

@rehmanmuradali
Copy link
Contributor Author

R: @reuvenlax

@aaltay
Copy link
Member

aaltay commented Jan 23, 2020

Pinging to trigger tests.

@amaliujia
Copy link
Contributor

Run Java PreCommit

@amaliujia
Copy link
Contributor

Run JavaPortabilityApi PreCommit

@rehmanmuradali
Copy link
Contributor Author

@reuvenlax I have completed outputTimestamp functionality. Need your input with the failed test cases. I think it is expected with the addition of this feature.

@iemejia
Copy link
Member

iemejia commented Jan 28, 2020

retest this please

@iemejia
Copy link
Member

iemejia commented Jan 28, 2020

Run Direct ValidatesRunner

@rehmanmuradali
Copy link
Contributor Author

rehmanmuradali commented Jan 28, 2020

@reuvenlax really need your input with the failed test cases as the functionality is completed.

@mwalenia
Copy link
Member

Run Java PreCommit

public Timer withOutputTimestampOffset(Duration outputTimestampOffset) {
this.outputTimestampOffset = outputTimestampOffset;
return this;
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure that we need withOutputTimestampOffset - I think withOutputTimestamp is sufficient.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

: target.minus(offset.minus(outputTimestampOffset));
}

if (TimeDomain.EVENT_TIME.equals(spec.getTimeDomain())) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that we should verify that the output timestamp is > the timestamp of the input message (if in processElement) or the output timestamp of the firing timer (if in processTimer). The < check remains correct - even for processing-time timers.

Copy link
Contributor Author

@rehmanmuradali rehmanmuradali Jan 30, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax , does that mean we need to compare output timestamp with timerInternal.currentInputWatermarkTime()? I think it will fail our previous test case of output timestamp with event timer as we have set output timestamp as 5 while having a timestamp of input element as 9.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

if (outputTimestamp == null) {
if (outputTimestamp == null && TimeDomain.EVENT_TIME.equals(spec.getTimeDomain())) {
outputTimestamp = target;
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that if the timer is processing time, then then default outputTimestamp should be that of the input element (or the output time of the firing timer if in processTimer).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

TimerSpec spec = (TimerSpec) signature.timerDeclarations().get(timerId).field().get(fn);
return new TimerInternalsTimer(
window, getNamespace(), timerId, spec, stepContext.timerInternals());
window, getNamespace(), timerId, spec, fireTimestamp(), stepContext.timerInternals());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this should be timestamp(), not fireTimestamp().

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

window(),
getNamespace(),
spec,
fireTimestamp(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto - this should be timestamp() not fireTimestamp()

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

outputTimestamp,
elementInputTimestamp);

if (TimeDomain.EVENT_TIME.equals(spec.getTimeDomain())) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

check outputTimestamp is in the window for processing-timer as well.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

new DoFn<KV<String, Integer>, Integer>() {

@TimerId(timerId)
private final TimerSpec timer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you want the second DoFn to set an event-time timer. That way you can test that the watermark hold in the first DoFn prevents firing of the second DoFn's timer, which it will only do if the second DoFn's timer is even time.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax updated

Instant earliest = THE_END_OF_TIME.get();
for (NavigableSet<TimerData> timers : processingTimers.values()) {
if (!timers.isEmpty()) {
earliest = INSTANT_ORDERING.min(timers.first().getTimestamp(), earliest);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if this is sufficient. The NavigableSet is ordered by timer firing time, not by outputTimestamp. This means that the logic to just look at the first timer (which was correct before we had holds) is probably no longer enough. we probably need to iterate over the set (or we need to have a second data structure to hold the watermark holds and take the minimum of that data structure).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax added a new function to calculate the earliest outputTimestamp from NavigableSet.

@iemejia
Copy link
Member

iemejia commented Jan 31, 2020

retest this please

@iemejia
Copy link
Member

iemejia commented Jan 31, 2020

Run Direct ValidatesRunner

target,
windowExpiry);
// For processing timers
if (outputTimestamp == null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

are we missing an else clause here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax we only set outputTimestamp here. I think If that is already set before then we don't need anything. I think else clause is not required here.

private final TimerSpec spec;
private Instant target;
private Instant outputTimestamp;
private Instant elementInputTimestamp;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

final

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

!outputTimestamp.isBefore(elementInputTimestamp),
"output timestamp %s should be after input message timestamp or output timestamp of firing timers %s",
outputTimestamp,
elementInputTimestamp);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

check this before setting outputTimestamp above (and only if outputTimestamp != null)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

@reuvenlax
Copy link
Contributor

retest this please

1 similar comment
@iemejia
Copy link
Member

iemejia commented Feb 2, 2020

retest this please

@reuvenlax
Copy link
Contributor

Run Java PreCommit

@kennknowles
Copy link
Member

run dataflow validatesrunner

@kennknowles
Copy link
Member

run flink validatesrunner

// DoFn timer's watermark hold. This timer should not fire until the previous timer
// fires and removes
// the watermark hold.
timer.offset(Duration.standardSeconds(8)).setRelative();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just use timer.set, not timer.offset

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

.addElements(KV.of("key", 1))
// Normally this would case fn2's timer to expire, but it shouldn't here because of
// the output timestamp.
.advanceProcessingTime(Duration.standardSeconds(9))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you might have to advance processing time to at least 11.

You also need to advance the watermark to at least 10 to allow the timer to fire. Right now this isn't testing anything, because the input watermark is preventing the timer from firing.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax, I think if we advance processing time to 11, it will fire fn1's timer as we have set delivery time offset as 10. If we advance processing time by 9 and watermark by 11 then it will test the functionality?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, but you need to advance the watermark before calling addElements.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

*/
public synchronized Instant getEarliestTimerTimestamp() {
Instant earliest = THE_END_OF_TIME.get();
for (NavigableSet<TimerData> timers : processingTimers.values()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about the other getEarliestTimerTimestamp function? Does that need to be updated?

Copy link
Contributor Author

@rehmanmuradali rehmanmuradali Feb 6, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@reuvenlax done

@aaltay
Copy link
Member

aaltay commented Feb 7, 2020

Pinging to trigger tests.

@reuvenlax
Copy link
Contributor

Run Python2_PVR_Flink PreCommit

@reuvenlax reuvenlax merged commit a005fd7 into apache:master Feb 9, 2020
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants