You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Daniel Collins (Jira)" <ji...@apache.org> on 2022/04/06 14:07:00 UTC

[jira] [Comment Edited] (BEAM-14253) pubsublite.ReadWriteIT failing in beam_PostCommit_Java_DataflowV1 and V2

    [ https://issues.apache.org/jira/browse/BEAM-14253?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17518149#comment-17518149 ] 

Daniel Collins edited comment on BEAM-14253 at 4/6/22 2:06 PM:
---------------------------------------------------------------

{{This appears to be this code in SubscriptionPartitionLoader:}}


@GetInitialWatermarkEstimatorState
public Instant getInitialWatermarkEstimatorState(@Timestamp Instant initial) {
  checkArgument(initial.equals(BoundedWindow.TIMESTAMP_MIN_VALUE));
  return initial; 
}



This is worrying because it implies that dataflow watermarks are not performing as expected for SDFs. I think I need [~lukecwik@gmail.com] to take a look at this- can you think of how the initial watermark for this would not be MIN_VALUE? 
 


was (Author: dpcollins-google):
{{This appears to be this code in SubscriptionPartitionLoader:}}

{{
@GetInitialWatermarkEstimatorState
public Instant getInitialWatermarkEstimatorState(@Timestamp Instant initial) \{
  checkArgument(initial.equals(BoundedWindow.TIMESTAMP_MIN_VALUE));
  return initial; 
\}
}}

This is worrying because it implies that dataflow watermarks are not performing as expected for SDFs. I think I need [~lukecwik@gmail.com] to take a look at this- can you think of how the initial watermark for this would not be MIN_VALUE? 
 

> pubsublite.ReadWriteIT failing in beam_PostCommit_Java_DataflowV1 and V2
> ------------------------------------------------------------------------
>
>                 Key: BEAM-14253
>                 URL: https://issues.apache.org/jira/browse/BEAM-14253
>             Project: Beam
>          Issue Type: Sub-task
>          Components: io-java-gcp, test-failures
>            Reporter: Daniel Oliveira
>            Assignee: Daniel Collins
>            Priority: P1
>
> Example: https://ci-beam.apache.org/job/beam_PostCommit_Java_DataflowV1/1455/testReport/junit/org.apache.beam.sdk.io.gcp.pubsublite/ReadWriteIT/testReadWrite/
> {noformat}
> java.lang.AssertionError: Did not receive signal on projects/apache-beam-testing/subscriptions/result-subscription--586739339276181574 in 300s
> {noformat}
> Dataflow logs show this, might be related:
> {noformat}
> Error message from worker: java.lang.IllegalArgumentException
>         org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:127)
>         org.apache.beam.sdk.io.gcp.pubsublite.internal.SubscriptionPartitionLoader$GeneratorFn.getInitialWatermarkEstimatorState(SubscriptionPartitionLoader.java:76)
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)