You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/06/08 00:02:33 UTC

[GitHub] [beam] Naireen opened a new pull request, #21740: convert windmill min timestamp to beam min timestamp

Naireen opened a new pull request, #21740:
URL: https://github.com/apache/beam/pull/21740

   **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)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Add a link to the appropriate issue in your description, if applicable. This will automatically link the pull request to the issue.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Naireen commented on a diff in pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
Naireen commented on code in PR #21740:
URL: https://github.com/apache/beam/pull/21740#discussion_r892644849


##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java:
##########
@@ -56,6 +56,7 @@ public void testWindmillToHarnessTimestamp() {
     assertEquals(new Instant(-17), windmillToHarnessTimestamp(-16987));
     assertEquals(new Instant(-17), windmillToHarnessTimestamp(-17000));
     assertEquals(new Instant(-18), windmillToHarnessTimestamp(-17001));
+    assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, windmillToHarnessTimestamp(Long.MIN_VALUE + 1));

Review Comment:
   Yeah I did, it did fail prior to this code change.  Rounding down gives you -290308-12-21T19:59:05.224Z, instead of the desired -290308-12-21T19:59:05.225Z



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay merged pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
aaltay merged PR #21740:
URL: https://github.com/apache/beam/pull/21740


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] scwhittle commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
scwhittle commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1154217788

   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Naireen commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
Naireen commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1152579950

   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
aaltay commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1155942843

   @kennknowles - precommit flaked 5 times in a row. We need to do something about it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kennknowles commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
kennknowles commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1156672746

   Yes. We have someone already working on "org.apache.beam.sdk.io.pulsar.PulsarIOTest.testReadFromSimpleTopic: Trying to claim offset 1655305408194 while last attempted was 1655305409570" but I think it just needs to be disabled. I also saw a lot of docker issues. Tests that use a local server in a docker container should probably be moved off of the main job. Because the job uses gradle's `buildDependents` it is hard to make fine-grained adjustments. I'm thinking about it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] asf-ci commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1149288075

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
aaltay commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1155396315

   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on a diff in pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
Abacn commented on code in PR #21740:
URL: https://github.com/apache/beam/pull/21740#discussion_r899158211


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtils.java:
##########
@@ -45,6 +45,9 @@ public static Instant windmillToHarnessTimestamp(long timestampUs) {
     // Windmill should never send us an unknown timestamp.
     Preconditions.checkArgument(timestampUs != Long.MIN_VALUE);
     Instant result = new Instant(divideAndRoundDown(timestampUs, 1000));
+    if (result.isBefore(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
+      return BoundedWindow.TIMESTAMP_MIN_VALUE;

Review Comment:
   This condition prevented timestamp overflow and broke unit test WindmillTimerInternalsTest.testTimerDataToFromTimer(<timestamp>=BoundedWindow.TIMESTAMP_MIN_VALUE). If this is intended behavior, should adjust the test case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] scwhittle commented on a diff in pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
scwhittle commented on code in PR #21740:
URL: https://github.com/apache/beam/pull/21740#discussion_r892216449


##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java:
##########
@@ -56,6 +56,7 @@ public void testWindmillToHarnessTimestamp() {
     assertEquals(new Instant(-17), windmillToHarnessTimestamp(-16987));
     assertEquals(new Instant(-17), windmillToHarnessTimestamp(-17000));
     assertEquals(new Instant(-18), windmillToHarnessTimestamp(-17001));
+    assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, windmillToHarnessTimestamp(Long.MIN_VALUE + 1));

Review Comment:
   Did you verify this test fails without the change to WindmillTimeUtils?
   Looking again I'm wondering if the divideAndRoundDown would have already ended up with this value.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
aaltay commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1155525576

   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
aaltay commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1155942535

   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Naireen commented on a diff in pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
Naireen commented on code in PR #21740:
URL: https://github.com/apache/beam/pull/21740#discussion_r892644849


##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java:
##########
@@ -56,6 +56,7 @@ public void testWindmillToHarnessTimestamp() {
     assertEquals(new Instant(-17), windmillToHarnessTimestamp(-16987));
     assertEquals(new Instant(-17), windmillToHarnessTimestamp(-17000));
     assertEquals(new Instant(-18), windmillToHarnessTimestamp(-17001));
+    assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, windmillToHarnessTimestamp(Long.MIN_VALUE + 1));

Review Comment:
   Yeah I did, it did fail prior to this code change. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kennknowles commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
kennknowles commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1157666948

   Hmm this is green but in https://ci-beam.apache.org/view/PostCommit/job/beam_PreCommit_Java_Cron/5560/ and all subsequent tests (including current PRs) we are seeing `org.apache.beam.runners.dataflow.worker.WindmillTimerInternalsTest.testTimerDataToFromTimer` fail with
   
   ```
   Expected: <TimerData{timerId=0:-9223372036854775, timerFamilyId=, namespace=Global, timestamp=-290308-12-21T19:59:05.225Z, outputTimestamp=-290308-12-21T19:59:05.224Z, domain=EVENT_TIME, deleted=false}>
        but: was <TimerData{timerId=0:-9223372036854775, timerFamilyId=, namespace=Global, timestamp=-290308-12-21T19:59:05.225Z, outputTimestamp=-290308-12-21T19:59:05.225Z, domain=EVENT_TIME, deleted=false}>
   ```
   
   and based on timing and code touched this PR seems the most likely to be related. TBH I don't get it since this seems like it is deterministic and would have been red on this PR


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
aaltay commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1156739655

   > Yes. We have someone already working on "org.apache.beam.sdk.io.pulsar.PulsarIOTest.testReadFromSimpleTopic: Trying to claim offset 1655305408194 while last attempted was 1655305409570" but I think it just needs to be disabled. I also saw a lot of docker issues. Tests that use a local server in a docker container should probably be moved off of the main job. Because the job uses gradle's `buildDependents` it is hard to make fine-grained adjustments. I'm thinking about it.
   
   Thank you.
   
   I think I saw other issues as well. But I did not save the jenkins links. IO was the general root cause.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
Abacn commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1157729375

   "timestamp=-290308-12-21T19:59:05.225Z" is a negative year. It is an overflow. Looks like the conversion treated overflow integers in different way.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] asf-ci commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1149288074

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Naireen commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
Naireen commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1149298407

   R: @scwhittle 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] scwhittle commented on a diff in pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
scwhittle commented on code in PR #21740:
URL: https://github.com/apache/beam/pull/21740#discussion_r892250466


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtils.java:
##########
@@ -44,6 +44,10 @@ public class WindmillTimeUtils {
   public static Instant windmillToHarnessTimestamp(long timestampUs) {
     // Windmill should never send us an unknown timestamp.
     Preconditions.checkArgument(timestampUs != Long.MIN_VALUE);
+    // Set windmill min timestamp to beam min timestamp
+    if (timestampUs == Long.MIN_VALUE + 1) {
+      return BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }

Review Comment:
   actually would other values (ie Long.MIN_VALUE + 2) also be problematic (though that timestamp would be less likely to be observed)
   
   maybe this should calculate result as before but then bounds check it with TIMESTAMP.MIN_VALUE similar to how it bound checks TIMESTAMP_MAX_VALUE



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kennknowles commented on pull request #21740: convert windmill min timestamp to beam min timestamp

Posted by GitBox <gi...@apache.org>.
kennknowles commented on PR #21740:
URL: https://github.com/apache/beam/pull/21740#issuecomment-1155752413

   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org