Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-6918

Failing tests: ChainLengthDecreaseTest and ChainLengthIncreaseTest

    Details

    • Type: Bug
    • Status: Closed
    • Priority: Critical
    • Resolution: Fixed
    • Affects Version/s: 1.4.0
    • Fix Version/s: 1.4.0, 1.3.2
    • Component/s: Tests
    • Labels:

      Description

      While running mvn clean verify on Linux with commit 3bad77c0ae932a926260b769efb151a89fc309ab.

      Tests in error:
        ChainLengthDecreaseTest>AbstractOperatorRestoreTestBase.testMigrationAndRestore:164->AbstractOperatorRestoreTestBase.migrateJob:202 »
        ChainLengthIncreaseTest>AbstractOperatorRestoreTestBase.testMigrationAndRestore:164->AbstractOperatorRestoreTestBase.migrateJob:202 »
      
      Tests run: 2, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 6.497 sec <<< FAILURE! - in org.apache.flink.test.state.operator.restore.unkeyed.ChainLengthDecreaseTest
      testMigrationAndRestore[Migrate Savepoint: nonKeyed-flink1.3](org.apache.flink.test.state.operator.restore.unkeyed.ChainLengthDecreaseTest)  Time elapsed: 0.361 sec  <<< ERROR!
      java.lang.Exception: java.lang.Exception: Failed to trigger savepoint.
      	at org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase.migrateJob(AbstractOperatorRestoreTestBase.java:202)
      	at org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase.testMigrationAndRestore(AbstractOperatorRestoreTestBase.java:164)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
      	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
      	at org.junit.rules.RunRules.evaluate(RunRules.java:20)
      	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
      	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
      	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
      	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
      	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
      	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
      	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
      	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
      	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
      	at org.junit.runners.Suite.runChild(Suite.java:128)
      	at org.junit.runners.Suite.runChild(Suite.java:27)
      	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
      	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
      	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
      	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
      	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
      	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
      	at org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:283)
      	at org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:173)
      	at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:153)
      	at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:128)
      	at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:203)
      	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:155)
      	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:103)
      Caused by: java.lang.Exception: Failed to trigger savepoint.
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anon$6.apply(JobManager.scala:639)
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anon$6.apply(JobManager.scala:629)
      	at org.apache.flink.runtime.concurrent.impl.FlinkFuture$5.onComplete(FlinkFuture.java:272)
      	at akka.dispatch.OnComplete.internal(Future.scala:247)
      	at akka.dispatch.OnComplete.internal(Future.scala:245)
      	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:175)
      	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:172)
      	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
      	at akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
      	at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91)
      	at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
      	at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
      	at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
      	at akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90)
      	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
      	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
      	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
      	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
      	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
      	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
      Caused by: java.lang.Exception: Checkpoint was declined (tasks not ready)
      	at org.apache.flink.runtime.checkpoint.PendingCheckpoint.abortDeclined(PendingCheckpoint.java:510)
      	at org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveDeclineMessage(CheckpointCoordinator.java:698)
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$org$apache$flink$runtime$jobmanager$JobManager$$handleCheckpointMessage$2.apply$mcV$sp(JobManager.scala:1491)
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$org$apache$flink$runtime$jobmanager$JobManager$$handleCheckpointMessage$2.apply(JobManager.scala:1490)
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$org$apache$flink$runtime$jobmanager$JobManager$$handleCheckpointMessage$2.apply(JobManager.scala:1490)
      	at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
      	at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
      	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
      	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
      	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
      	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
      	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
      	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
      
      Tests run: 2, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 6.722 sec <<< FAILURE! - in org.apache.flink.test.state.operator.restore.unkeyed.ChainLengthIncreaseTest
      testMigrationAndRestore[Migrate Savepoint: nonKeyed-flink1.3](org.apache.flink.test.state.operator.restore.unkeyed.ChainLengthIncreaseTest)  Time elapsed: 0.458 sec  <<< ERROR!
      java.lang.Exception: java.lang.Exception: Failed to trigger savepoint.
      	at org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase.migrateJob(AbstractOperatorRestoreTestBase.java:202)
      	at org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase.testMigrationAndRestore(AbstractOperatorRestoreTestBase.java:164)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
      	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
      	at org.junit.rules.RunRules.evaluate(RunRules.java:20)
      	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
      	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
      	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
      	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
      	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
      	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
      	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
      	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
      	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
      	at org.junit.runners.Suite.runChild(Suite.java:128)
      	at org.junit.runners.Suite.runChild(Suite.java:27)
      	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
      	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
      	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
      	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
      	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
      	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
      	at org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:283)
      	at org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:173)
      	at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:153)
      	at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:128)
      	at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:203)
      	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:155)
      	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:103)
      Caused by: java.lang.Exception: Failed to trigger savepoint.
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anon$6.apply(JobManager.scala:639)
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anon$6.apply(JobManager.scala:629)
      	at org.apache.flink.runtime.concurrent.impl.FlinkFuture$5.onComplete(FlinkFuture.java:272)
      	at akka.dispatch.OnComplete.internal(Future.scala:247)
      	at akka.dispatch.OnComplete.internal(Future.scala:245)
      	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:175)
      	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:172)
      	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
      	at akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
      	at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91)
      	at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
      	at akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
      	at scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
      	at akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90)
      	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
      	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
      	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
      	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
      	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
      	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
      Caused by: java.lang.Exception: Checkpoint was declined (tasks not ready)
      	at org.apache.flink.runtime.checkpoint.PendingCheckpoint.abortDeclined(PendingCheckpoint.java:510)
      	at org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveDeclineMessage(CheckpointCoordinator.java:698)
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$org$apache$flink$runtime$jobmanager$JobManager$$handleCheckpointMessage$2.apply$mcV$sp(JobManager.scala:1491)
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$org$apache$flink$runtime$jobmanager$JobManager$$handleCheckpointMessage$2.apply(JobManager.scala:1490)
      	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$org$apache$flink$runtime$jobmanager$JobManager$$handleCheckpointMessage$2.apply(JobManager.scala:1490)
      	at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
      	at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
      	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
      	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
      	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
      	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
      	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
      	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
      

        Issue Links

          Activity

          Hide
          till.rohrmann Till Rohrmann added a comment -

          1.4.0: 47862afbef98faee61e07ca4a00f41f34a764cf5
          1.3.2: 70f33441f87e600f7b5197b87ac34c81604f8ee9

          Show
          till.rohrmann Till Rohrmann added a comment - 1.4.0: 47862afbef98faee61e07ca4a00f41f34a764cf5 1.3.2: 70f33441f87e600f7b5197b87ac34c81604f8ee9
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/4129

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/4129
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann commented on the issue:

          https://github.com/apache/flink/pull/4129

          Thanks for reviewing @zentol. Merging this PR.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/4129 Thanks for reviewing @zentol. Merging this PR.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user zentol commented on the issue:

          https://github.com/apache/flink/pull/4129

          +1 to merge.

          Show
          githubbot ASF GitHub Bot added a comment - Github user zentol commented on the issue: https://github.com/apache/flink/pull/4129 +1 to merge.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user tillrohrmann opened a pull request:

          https://github.com/apache/flink/pull/4129

          FLINK-6918 [tests] Harden AbstractOperatorRestoreTestBase by retrying CancelWithSavepoint messages

          The problem is that a StreamTask can be in state RUNNING without internally being running.
          As a consequence checkpoint message will be discarded. This problem will be solved once
          FLINK-4714 has been addressed. Until then, we harden the test case by retrying the
          CancelWithSavepoint message.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/tillrohrmann/flink hardenAbstractOperatorRestoreTestBase

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/4129.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #4129


          commit 7587fa23b3fb0f3ef8d799625ea78bf58046128f
          Author: Till Rohrmann <trohrmann@apache.org>
          Date: 2017-06-15T06:54:42Z

          FLINK-6918 [tests] Harden AbstractOperatorRestoreTestBase by retrying CancelWithSavepoint messages

          The problem is that a StreamTask can be in state RUNNING without internally being running.
          As a consequence checkpoint message will be discarded. This problem will be solved once
          FLINK-4714 has been addressed. Until then, we harden the test case by retrying the
          CancelWithSavepoint message.


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tillrohrmann opened a pull request: https://github.com/apache/flink/pull/4129 FLINK-6918 [tests] Harden AbstractOperatorRestoreTestBase by retrying CancelWithSavepoint messages The problem is that a StreamTask can be in state RUNNING without internally being running. As a consequence checkpoint message will be discarded. This problem will be solved once FLINK-4714 has been addressed. Until then, we harden the test case by retrying the CancelWithSavepoint message. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tillrohrmann/flink hardenAbstractOperatorRestoreTestBase Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/4129.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #4129 commit 7587fa23b3fb0f3ef8d799625ea78bf58046128f Author: Till Rohrmann <trohrmann@apache.org> Date: 2017-06-15T06:54:42Z FLINK-6918 [tests] Harden AbstractOperatorRestoreTestBase by retrying CancelWithSavepoint messages The problem is that a StreamTask can be in state RUNNING without internally being running. As a consequence checkpoint message will be discarded. This problem will be solved once FLINK-4714 has been addressed. Until then, we harden the test case by retrying the CancelWithSavepoint message.
          Hide
          till.rohrmann Till Rohrmann added a comment -

          Once FLINK-4714 has been fixed, this issue should resolve as well.

          Show
          till.rohrmann Till Rohrmann added a comment - Once FLINK-4714 has been fixed, this issue should resolve as well.
          Hide
          till.rohrmann Till Rohrmann added a comment -

          Another instance: This time for the KeyedComplexChainTest.

          https://s3.amazonaws.com/archive.travis-ci.org/jobs/243035919/log.txt
          https://s3.amazonaws.com/archive.travis-ci.org/jobs/243035922/log.txt

          Maybe as a quick fix, one could retry the CancelJobWithSavepoint messages until we have the proper lifecycle management for StreamTask in place.

          Show
          till.rohrmann Till Rohrmann added a comment - Another instance: This time for the KeyedComplexChainTest . https://s3.amazonaws.com/archive.travis-ci.org/jobs/243035919/log.txt https://s3.amazonaws.com/archive.travis-ci.org/jobs/243035922/log.txt Maybe as a quick fix, one could retry the CancelJobWithSavepoint messages until we have the proper lifecycle management for StreamTask in place.
          Hide
          Zentol Chesnay Schepler added a comment -

          So, i guess what happens is that the savepoint is triggered after the task went into the RUNNING state, but (ironically) before the StreamTask has called run().

          Show
          Zentol Chesnay Schepler added a comment - So, i guess what happens is that the savepoint is triggered after the task went into the RUNNING state, but (ironically) before the StreamTask has called run() .
          Hide
          Zentol Chesnay Schepler added a comment -

          oh, I just got it as well locally. Time to investigate.

          Show
          Zentol Chesnay Schepler added a comment - oh, I just got it as well locally. Time to investigate.
          Hide
          greghogan Greg Hogan added a comment - - edited

          Yes, also 1.3.

          testMigrationAndRestore[Migrate Savepoint: nonKeyed-flink1.3](org.apache.flink.test.state.operator.restore.unkeyed.ChainLengthIncreaseTest)  Time elapsed: 0.595 sec  <<< ERROR!
          java.lang.Exception: java.lang.Exception: Failed to trigger savepoint.
          	at org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase.migrateJob(AbstractOperatorRestoreTestBase.java:202)
          
          Show
          greghogan Greg Hogan added a comment - - edited Yes, also 1.3. testMigrationAndRestore[Migrate Savepoint: nonKeyed-flink1.3](org.apache.flink.test.state.operator.restore.unkeyed.ChainLengthIncreaseTest) Time elapsed: 0.595 sec <<< ERROR! java.lang.Exception: java.lang.Exception: Failed to trigger savepoint. at org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase.migrateJob(AbstractOperatorRestoreTestBase.java:202)
          Hide
          Zentol Chesnay Schepler added a comment -

          This test is run twice, once for 1.2 and 1.3
          testMigrationAndRestore[Migrate Savepoint: nonKeyed-flink1.3 ==> 1.3

          When the test failed the second time, was that for 1.3 as well?

          Show
          Zentol Chesnay Schepler added a comment - This test is run twice, once for 1.2 and 1.3 testMigrationAndRestore[Migrate Savepoint: nonKeyed-flink1.3 ==> 1.3 When the test failed the second time, was that for 1.3 as well?
          Hide
          greghogan Greg Hogan added a comment - - edited

          @zentol no, not a consistent error.

          Edit: third run failed, so 2/3 failures.

          Show
          greghogan Greg Hogan added a comment - - edited @zentol no, not a consistent error. Edit: third run failed, so 2/3 failures.
          Hide
          Zentol Chesnay Schepler added a comment -

          Does this happen all the time?

          The checkpoint is declined because tasks are not ready, which is odd since the test does verify that they are running before triggering the savepoint with

          msg = new TestingJobManagerMessages.WaitForAllVerticesToBeRunning(jobToMigrate.getJobID());
          		Await.result(jobManager.ask(msg, timeout), timeout);
          

          Since the test finished in 6 seconds we could not have hit the timeout.

          Show
          Zentol Chesnay Schepler added a comment - Does this happen all the time? The checkpoint is declined because tasks are not ready, which is odd since the test does verify that they are running before triggering the savepoint with msg = new TestingJobManagerMessages.WaitForAllVerticesToBeRunning(jobToMigrate.getJobID()); Await.result(jobManager.ask(msg, timeout), timeout); Since the test finished in 6 seconds we could not have hit the timeout.

            People

            • Assignee:
              till.rohrmann Till Rohrmann
              Reporter:
              greghogan Greg Hogan
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development