[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16689921#comment-16689921 ] ASF GitHub Bot commented on FLINK-10531: asfgit closed pull request #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java index 3b2e4746b62..1a572f314c5 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; @@ -74,6 +75,8 @@ public static void main(String[] args) throws Exception { setupEnvironment(env, pt); + final MonotonicTTLTimeProvider ttlTimeProvider = setBackendWithCustomTTLTimeProvider(env); + int keySpace = pt.getInt(UPDATE_GENERATOR_SRC_KEYSPACE.key(), UPDATE_GENERATOR_SRC_KEYSPACE.defaultValue()); long sleepAfterElements = pt.getLong(UPDATE_GENERATOR_SRC_SLEEP_AFTER_ELEMENTS.key(), UPDATE_GENERATOR_SRC_SLEEP_AFTER_ELEMENTS.defaultValue()); @@ -90,11 +93,27 @@ public static void main(String[] args) throws Exception { .addSource(new TtlStateUpdateSource(keySpace, sleepAfterElements, sleepTime)) .name("TtlStateUpdateSource") .keyBy(TtlStateUpdate::getKey) - .flatMap(new TtlVerifyUpdateFunction(ttlConfig, reportStatAfterUpdatesNum)) + .flatMap(new TtlVerifyUpdateFunction(ttlConfig, ttlTimeProvider, reportStatAfterUpdatesNum)) .name("TtlVerifyUpdateFunction") .addSink(new PrintSinkFunction<>()) .name("PrintFailedVerifications"); env.execute("State TTL test job"); } + + /** +* Sets the state backend to a new {@link StubStateBackend} which has a {@link MonotonicTTLTimeProvider}. +* +* @param env The {@link StreamExecutionEnvironment} of the job. +* @return The {@link MonotonicTTLTimeProvider}. +*/ + private static MonotonicTTLTimeProvider setBackendWithCustomTTLTimeProvider(StreamExecutionEnvironment env) { + final MonotonicTTLTimeProvider ttlTimeProvider = new MonotonicTTLTimeProvider(); + + final StateBackend configuredBackend = env.getStateBackend(); + final StateBackend stubBackend = new StubStateBackend(configuredBackend, ttlTimeProvider); + env.setStateBackend(stubBackend); + + return ttlTimeProvider; + } } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/MonotonicTTLTimeProvider.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/MonotonicTTLTimeProvider.java new file mode 100644 index 000..0b5637d36c4 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/MonotonicTTLTimeProvider.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16688409#comment-16688409 ] ASF GitHub Bot commented on FLINK-10531: azagrebin commented on a change in pull request #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#discussion_r233938203 ## File path: flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java ## @@ -90,11 +107,83 @@ public static void main(String[] args) throws Exception { .addSource(new TtlStateUpdateSource(keySpace, sleepAfterElements, sleepTime)) .name("TtlStateUpdateSource") .keyBy(TtlStateUpdate::getKey) - .flatMap(new TtlVerifyUpdateFunction(ttlConfig, reportStatAfterUpdatesNum)) + .flatMap(new TtlVerifyUpdateFunction(ttlConfig, ttlTimeProvider, reportStatAfterUpdatesNum)) .name("TtlVerifyUpdateFunction") .addSink(new PrintSinkFunction<>()) .name("PrintFailedVerifications"); env.execute("State TTL test job"); } + + /** +* Sets the state backend to a new {@link StubStateBackend} which has a {@link MonotonicTTLTimeProvider}. +* +* @param env The {@link StreamExecutionEnvironment} of the job. +* @return The {@link MonotonicTTLTimeProvider}. +*/ + private static MonotonicTTLTimeProvider setBackendWithCustomTTLTimeProvider(StreamExecutionEnvironment env) { + final MonotonicTTLTimeProvider ttlTimeProvider = new MonotonicTTLTimeProvider(); + + final StateBackend configuredBackend = env.getStateBackend(); + final StateBackend stubBackend = new StubStateBackend(configuredBackend, ttlTimeProvider); + env.setStateBackend(stubBackend); + + return ttlTimeProvider; + } + + /** +* A stub implementation of the {@link StateBackend} that allows the use of +* a custom {@link TtlTimeProvider}. +*/ + private static final class StubStateBackend implements StateBackend { Review comment: Let's also generate UUID for `StubStateBackend`, `MonotonicTTLTimeProvider`, `TtlVerifyUpdateFunction`, `UpdateStat`, `AggregateFunction`, `TtlUpdateContext`, `TtlVerificationContext`, `ValueWithTs` and `ValueWithTs.Serializer`. I would suggest to create `StateBackendWrapperAdaptor` in main code along with `StateBackend` as well which would always wrap `wrappedBackend` and forward all `StateBackend` interface methods. Here we would need to override only relevant methods. The adaptor could be reused in future for similar cases like we have here. If `StateBackend` interface gets more methods, they would need default forwarding only in `StateBackendWrapperAdaptor` and other extending classes can stay untouched. Otherwise, I would suggest to move this class at least to separate file. I leave these last ideas up to you. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16686867#comment-16686867 ] ASF GitHub Bot commented on FLINK-10531: kl0u commented on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-438735965 @azagrebin Please have another look and let me know what you think! This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16679947#comment-16679947 ] ASF GitHub Bot commented on FLINK-10531: kl0u edited a comment on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-437051981 Actually, now @StefanRRichter mentioned it, we have the `TtlTimeProvider` and the `ProcessingTimeService`. Both of them are used internally by Flink to let it know what is the "current" processing time, but each of them has its own implementation (which happens to be `currentTimeMillis` for both). I think that these two should be unified and the `TtlTimeProvider` should call the `ProcessingTimeService::getCurrentProcessingTime()`. This will allow to have a "single point of truth". What do you think @StefanRRichter and @azagrebin This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16679925#comment-16679925 ] ASF GitHub Bot commented on FLINK-10531: kl0u commented on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-437044814 @azagrebin I commented on https://issues.apache.org/jira/browse/FLINK-10830 about my concerns of allowing the user to specify his/her own processing time provider. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16679946#comment-16679946 ] ASF GitHub Bot commented on FLINK-10531: kl0u commented on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-437051981 Actually, now @StefanRRichter mentioned it, we have the `TtlTimeProvider` and the `ProcessingTimeService`. Both of them are used internally by Flink to let it know what is the "current" processing time, but each of them has its own implementation (which happens to be `currentTimeMillis` for both). I think that these two should be unified and the `TtlTimeProvider` should call the `ProcessingTimeService::getCurrentProcessingTime()`. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16679916#comment-16679916 ] ASF GitHub Bot commented on FLINK-10531: azagrebin edited a comment on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-437038969 I agree that configuring custom processing time provider on the API side is more robust approach for tests. The change might be a bit more involving. I have created issues to consider it: FLINK-10830, FLINK-10831. One more option is to check if `currentTimeMillis` jumped back in `TtlVerifyUpdateFunction.performUpdate` relatively to the latest consumed value. If jump happened then just wait until `currentTimeMillis` returns the next value greater than the latest consumed value. This way the update never needs to be rejected for this reason and test restarted. There is still potential subtle problem (could be not really practical) if: - `timestampBeforeUpdate` gets increased value - Flink gets jumped back `currentTimeMillis` - lag happens and - `timestampAfterUpdate` gets again increased value. In this case, the time queried by Flink is completely out of test control and without plugging the time there is no way to sync it in Flink and test. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16679915#comment-16679915 ] ASF GitHub Bot commented on FLINK-10531: azagrebin commented on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-437038969 I agree that configuring custom processing time provider on the API side is more robust approach for tests. The change might be a bit more involving. I have created issues to consider it: FLINK-10830, FLINK-10831. One more option is to check if `currentTimeMillis` jumped back in `TtlVerifyUpdateFunction.performUpdate` relatively to the latest consumed value. If jump happened then just wait until it returns the next value greater than the latest consumed value. This way the update never needs to be rejected for this reason and test restarted. There is still potential subtle problem (could be not really practical) if: - `timestampBeforeUpdate` gets increased value - Flink gets jumped back `currentTimeMillis` - lag happens and - `timestampAfterUpdate` gets again increased value. In this case, the time queried by Flink is completely out of test control and without plugging the time there is no way to sync it in Flink and test. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16679597#comment-16679597 ] ASF GitHub Bot commented on FLINK-10531: StefanRRichter commented on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-436954874 I wonder if it would not make more sense to use a monotonous time provider for a single machine, such as `System.nanoTime()`, or a wrapper around time millis that prevents falling back in time, or some test implementation that the test can control, for the test instead of ignoring elements and potentially missing out on errors. `TtlTimeProvider` is already an interface, and `TtlTimeProvider DEFAULT = System::currentTimeMillis;` is used in all places. Instead this could be the first case where we want to make the time provider configurable via a factor. The simplest case could just replace it when encountering a certain config entry or property. Another option would be that if we detect such time changes we exit the test with a certain exit code and rerun it again. What do you think? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16677872#comment-16677872 ] ASF GitHub Bot commented on FLINK-10531: zentol commented on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-436552626 `performUpdate()` is using `System.currentTimeMillis` which is not guaranteed to be monotonous. Could this be the underlying cause? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > Fix For: 1.7.0 > > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16677023#comment-16677023 ] ASF GitHub Bot commented on FLINK-10531: kl0u commented on issue #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036#issuecomment-436331588 R @azagrebin This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16677018#comment-16677018 ] ASF GitHub Bot commented on FLINK-10531: kl0u opened a new pull request #7036: [FLINK-10531][e2e] Fix unstable TTL end-to-end test. URL: https://github.com/apache/flink/pull/7036 As describe on the JIRA, the problem is that the clock on the machine on Travis seems to have jumped backwards. This resulted in a mismatch between the elements discarded as expired by Flink's internal TTL mechanism, and the "user-code" in the test that computes the expired elements based on the timestamp of the latest update. I repeat the explanation from the JIRA here for reference: --- The way the test works is the following. For a given state: 1) it keeps the state with TTL 2) and keeps all the updates to that state irrespective of their timestamp (e.g. all the elements added in a ListState) To verify correctness: 1) it fetches the state from Flink, as cleaned up by the internal TTL mechanism 2) it reconstructs the expected state from the stored updates by taking the timestamp of the latest update (ts) discarding elements with timestamp ts-ttl As you can see from the stacktrace in the error from Travis, the latest update has timestamp ts=1538918066021 while there are elements in the list with timestamps greater than ts (e.g. 1538918066136). This means that the internal clock on that machine went backwards, so Flink's TTL may have removed elements that appear in the expected state of the test, as it takes as current timestamp the ts=1538918066021. The fix is simply to assume that (for the test), processing time increases monotonically and ignore "updates from the past". This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: pull-request-available, test-stability > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10531) State TTL RocksDb backend end-to-end test failed on Travis
[ https://issues.apache.org/jira/browse/FLINK-10531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16676957#comment-16676957 ] Kostas Kloudas commented on FLINK-10531: It is a test instability. The way the test works is the following. For a given state: 1) it keeps the state with TTL 2) and keeps all the updates to that state irrespective of their timestamp (e.g. all the elements added in a ListState) To verify correctness: 1) it fetches the state from Flink, as cleaned up by the internal TTL mechanism 2) it reconstructs the expected state from the stored updates by taking the timestamp of the latest update (ts) discarding elements with timestamp ts-ttl As you can see from the stacktrace in the error from Travis, the latest update has timestamp ts=1538918066021 while there are elements in the list with timestamps greater than ts (e.g. 1538918066136). This means that the internal clock on that machine went backwards, so Flink's TTL may have removed elements that appear in the expected state of the test, as it takes as current timestamp the ts=1538918066021. The fix is simply to assume that (for the test), processing time increases monotonically and ignore "updates from the past". > State TTL RocksDb backend end-to-end test failed on Travis > -- > > Key: FLINK-10531 > URL: https://issues.apache.org/jira/browse/FLINK-10531 > Project: Flink > Issue Type: Bug > Components: Tests >Affects Versions: 1.6.1 >Reporter: Till Rohrmann >Assignee: Kostas Kloudas >Priority: Critical > Labels: test-stability > > The {{State TTL RocksDb backend end-to-end test}} end-to-end test failed on > Travis. > https://travis-ci.org/apache/flink/jobs/438226190 > https://api.travis-ci.org/v3/job/438226190/log.txt -- This message was sent by Atlassian JIRA (v7.6.3#76005)