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

Flink treats every task as stateful (making topology changes impossible)

    Details

      Description

      It seems that Flink treats every Task as stateful so changing the topology is not possible without setting uid on every single operator.

      If the topology has an iteration this is virtually impossible (or at least gets super hacky)

        Issue Links

          Activity

          Hide
          uce Ufuk Celebi added a comment -

          It should be possible to change the topology if you set the UIDs only for the stateful operators, because these are the only ones that are part of the savepoint state.

          If you don't set any UIDs, changing the topology will result in a changed auto generated UID for the stateful operators as well. That's the situation you are describing here, right?

          Show
          uce Ufuk Celebi added a comment - It should be possible to change the topology if you set the UIDs only for the stateful operators, because these are the only ones that are part of the savepoint state. If you don't set any UIDs, changing the topology will result in a changed auto generated UID for the stateful operators as well. That's the situation you are describing here, right?
          Hide
          gyfora Gyula Fora added a comment -

          It should be possible to set it for only the stateful ones but it doesn work.

          Show
          gyfora Gyula Fora added a comment - It should be possible to set it for only the stateful ones but it doesn work.
          Hide
          gyfora Gyula Fora added a comment -

          even something as simple as stream.filter(event -> true) requires a uid. Also the iteration tasks which are impossible to assign a uid for in the API

          Show
          gyfora Gyula Fora added a comment - even something as simple as stream.filter(event -> true) requires a uid. Also the iteration tasks which are impossible to assign a uid for in the API
          Hide
          uce Ufuk Celebi added a comment -

          I fully agree that this whole UID business is very tricky and intransparent right now.

          Could you:
          1) Post the Exception you get
          2) Run the job before the savepoint with DEBUG logging for org.apache.flink.streaming.api.graph
          3) Run the job with which you want to restore with DEBUG logging for org.apache.flink.streaming.api.graph?

          Either this is a bug or we are overlooking a stateful task that does not have a UID set. You can set a UID after the fact with 1.2 via `setUIDHash` to the String of the JobVertexID.

          Show
          uce Ufuk Celebi added a comment - I fully agree that this whole UID business is very tricky and intransparent right now. Could you: 1) Post the Exception you get 2) Run the job before the savepoint with DEBUG logging for org.apache.flink.streaming.api.graph 3) Run the job with which you want to restore with DEBUG logging for org.apache.flink.streaming.api.graph? Either this is a bug or we are overlooking a stateful task that does not have a UID set. You can set a UID after the fact with 1.2 via `setUIDHash` to the String of the JobVertexID.
          Hide
          gyfora Gyula Fora added a comment -

          Hi Ufuk,

          I did exactly this yesterday:

          First got an error like: Cannot map savepoint state for operator xxx to the new program, because the operator is not available in the new program. Then I added the debugging to see which actual operator was misbehaving.

          It showed a stateless map, so I was like whatever and set the uidHash(xxx) and I tried again. Now it gave the same error for a different stateless operator (lets say a filter). I kept adding the hashes for the stateless operators until I reached to the point where it complained about not being able to map back the Iteration sink, that's when I gave up.

          Show
          gyfora Gyula Fora added a comment - Hi Ufuk, I did exactly this yesterday: First got an error like: Cannot map savepoint state for operator xxx to the new program, because the operator is not available in the new program. Then I added the debugging to see which actual operator was misbehaving. It showed a stateless map, so I was like whatever and set the uidHash(xxx) and I tried again. Now it gave the same error for a different stateless operator (lets say a filter). I kept adding the hashes for the stateless operators until I reached to the point where it complained about not being able to map back the Iteration sink, that's when I gave up.
          Hide
          uce Ufuk Celebi added a comment -

          Is it possible to share the driver program (privately works as well)? I could then try to reproduce the issue with dummy data.

          Show
          uce Ufuk Celebi added a comment - Is it possible to share the driver program (privately works as well)? I could then try to reproduce the issue with dummy data.
          Hide
          gyfora Gyula Fora added a comment -

          I was somehow assuming that this affects all jobs. Unfortunately I cannot send the program but I can try to reproduce it in a minimal example tomorrow. I know this used to work in 1.1 with pretty much the same job.

          Show
          gyfora Gyula Fora added a comment - I was somehow assuming that this affects all jobs. Unfortunately I cannot send the program but I can try to reproduce it in a minimal example tomorrow. I know this used to work in 1.1 with pretty much the same job.
          Hide
          gyfora Gyula Fora added a comment -

          You can easily reproduce the bug locally by running the following simple program:

          https://gist.github.com/gyfora/b5ccf836dc9e95bab2b1f5f483cb8bf6

          Take a savepoint and uncomment the stateless map before the sink to change the sink hash then restore.

          Show
          gyfora Gyula Fora added a comment - You can easily reproduce the bug locally by running the following simple program: https://gist.github.com/gyfora/b5ccf836dc9e95bab2b1f5f483cb8bf6 Take a savepoint and uncomment the stateless map before the sink to change the sink hash then restore.
          Hide
          uce Ufuk Celebi added a comment -

          Thanks for taking the time to create the example, Gyula.

          Show
          uce Ufuk Celebi added a comment - Thanks for taking the time to create the example, Gyula.
          Hide
          uce Ufuk Celebi added a comment -

          Thanks again for reporting this. You are right, we accidentally changed the behaviour between 1.1 and 1.2. This is a critical issue.

          In 1.1 we did not serialize stateless tasks as part of the savepoint and therefore never took them into account on savepoint loading. With the recent refactorings in 1.2 we now serialize stateless tasks this as "zero length state". This makes the stateless operators part of the savepoint we try to load the state back to the new job.

          I think there a couple of easy ways to fix this, but Stefan Richter probably has the definitive answer here.

          I think this issue warrants a 1.2.1 release asap as users cannot change their topologies if they didn't specify a UID for each operator.

          Show
          uce Ufuk Celebi added a comment - Thanks again for reporting this. You are right, we accidentally changed the behaviour between 1.1 and 1.2. This is a critical issue. In 1.1 we did not serialize stateless tasks as part of the savepoint and therefore never took them into account on savepoint loading. With the recent refactorings in 1.2 we now serialize stateless tasks this as "zero length state". This makes the stateless operators part of the savepoint we try to load the state back to the new job. I think there a couple of easy ways to fix this, but Stefan Richter probably has the definitive answer here. I think this issue warrants a 1.2.1 release asap as users cannot change their topologies if they didn't specify a UID for each operator.
          Hide
          srichter Stefan Richter added a comment -

          I think the proper solution is based on preventing those empty files from being written, but there must be some differentiation between stateless tasks and stateful tasks with (currently) empty state. For the later ones, the empty state indicates restores which is an exposed signal for user code.

          Show
          srichter Stefan Richter added a comment - I think the proper solution is based on preventing those empty files from being written, but there must be some differentiation between stateless tasks and stateful tasks with (currently) empty state. For the later ones, the empty state indicates restores which is an exposed signal for user code.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user StefanRRichter opened a pull request:

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

          FLINK-5985 Report no task states for stateless tasks in checkpointing

          This PR fixes FLINK-5985. The solution is based on acknowledging `null` instead of some empty SubtaskState to `CheckpointCoordinator#acknowledgeCheckpoint(...)`, so that no `TaskState` is registered under the `JobVertexID` of a stateless task in the checkpoint.

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

          $ git pull https://github.com/StefanRRichter/flink no-taskstate-for-stateless

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

          https://github.com/apache/flink/pull/3523.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 #3523


          commit af45e89d011033b98ceca455725c7f167052ec74
          Author: Stefan Richter <s.richter@data-artisans.com>
          Date: 2017-03-10T16:55:45Z

          FLINK-5985 Report no task states for stateless tasks in checkpointing


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user StefanRRichter opened a pull request: https://github.com/apache/flink/pull/3523 FLINK-5985 Report no task states for stateless tasks in checkpointing This PR fixes FLINK-5985 . The solution is based on acknowledging `null` instead of some empty SubtaskState to `CheckpointCoordinator#acknowledgeCheckpoint(...)`, so that no `TaskState` is registered under the `JobVertexID` of a stateless task in the checkpoint. You can merge this pull request into a Git repository by running: $ git pull https://github.com/StefanRRichter/flink no-taskstate-for-stateless Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3523.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 #3523 commit af45e89d011033b98ceca455725c7f167052ec74 Author: Stefan Richter <s.richter@data-artisans.com> Date: 2017-03-10T16:55:45Z FLINK-5985 Report no task states for stateless tasks in checkpointing
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          CC @gyfora @uce

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3523 CC @gyfora @uce
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user gyfora commented on the issue:

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

          The changes look reasonable

          Show
          githubbot ASF GitHub Bot added a comment - Github user gyfora commented on the issue: https://github.com/apache/flink/pull/3523 The changes look reasonable
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user gyfora commented on the issue:

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

          I could only try the backported version on the topology that caused the problem initally (that is running 1.2.0)

          Show
          githubbot ASF GitHub Bot added a comment - Github user gyfora commented on the issue: https://github.com/apache/flink/pull/3523 I could only try the backported version on the topology that caused the problem initally (that is running 1.2.0)
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          @gyfora if the effort is reasonable, it would be great to try this out on your topology. As soon as you give your +1, I could merge this change

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3523 @gyfora if the effort is reasonable, it would be great to try this out on your topology. As soon as you give your +1, I could merge this change
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user gyfora commented on the issue:

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

          Im gonna try to cherry-pick this on 1.2 and run it today

          Show
          githubbot ASF GitHub Bot added a comment - Github user gyfora commented on the issue: https://github.com/apache/flink/pull/3523 Im gonna try to cherry-pick this on 1.2 and run it today
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          Great, thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3523 Great, thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user gyfora commented on the issue:

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

          There seems to have been some changes in the StreamTask and some tests so I couldn't rebase this nicely. Do you have a minute to take a look and maybe push a branch with the backport please? That would help me a lot.

          Show
          githubbot ASF GitHub Bot added a comment - Github user gyfora commented on the issue: https://github.com/apache/flink/pull/3523 There seems to have been some changes in the StreamTask and some tests so I couldn't rebase this nicely. Do you have a minute to take a look and maybe push a branch with the backport please? That would help me a lot.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          Sure, I just quickly prepared a backport here:

          https://github.com/StefanRRichter/flink/tree/FLINK-5985-backport-to-1.2

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3523 Sure, I just quickly prepared a backport here: https://github.com/StefanRRichter/flink/tree/FLINK-5985-backport-to-1.2
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user gyfora commented on the issue:

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

          Hm, doesnt seem to work for the first try. What I did is I updated the client with the new jar based on your backport branch. Redeployed the job with a savepoint (to get the new Flink version), took a savepoint and tried to redeploy with the changed topology.

          I still seem to get the same error.

          Is it possible that the previous checkpoints have an effect on this? In any case I will double check tomorrow morning and try to do the test again.

          Show
          githubbot ASF GitHub Bot added a comment - Github user gyfora commented on the issue: https://github.com/apache/flink/pull/3523 Hm, doesnt seem to work for the first try. What I did is I updated the client with the new jar based on your backport branch. Redeployed the job with a savepoint (to get the new Flink version), took a savepoint and tried to redeploy with the changed topology. I still seem to get the same error. Is it possible that the previous checkpoints have an effect on this? In any case I will double check tomorrow morning and try to do the test again.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user gyfora commented on the issue:

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

          It also doesnt seem to work starting from a clean state and then savepoint redeploy with changed topology so maybe I am really screwing up something

          Show
          githubbot ASF GitHub Bot added a comment - Github user gyfora commented on the issue: https://github.com/apache/flink/pull/3523 It also doesnt seem to work starting from a clean state and then savepoint redeploy with changed topology so maybe I am really screwing up something
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user gyfora commented on the issue:

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

          @StefanRRichter It seems to work correctly locally, I am trying to see what went wrong with my yarn tests, but this shouldnt block you

          Show
          githubbot ASF GitHub Bot added a comment - Github user gyfora commented on the issue: https://github.com/apache/flink/pull/3523 @StefanRRichter It seems to work correctly locally, I am trying to see what went wrong with my yarn tests, but this shouldnt block you
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user gyfora commented on the issue:

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

          Ah, the reason is probably that I didnt change my job jar, and this relies on changes in the rocks backend

          Show
          githubbot ASF GitHub Bot added a comment - Github user gyfora commented on the issue: https://github.com/apache/flink/pull/3523 Ah, the reason is probably that I didnt change my job jar, and this relies on changes in the rocks backend
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          Ok, then the mystery is finally solved Thanks again for reporting this problem and your additional testing efforts!

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3523 Ok, then the mystery is finally solved Thanks again for reporting this problem and your additional testing efforts!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter closed the pull request at:

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

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

          GitHub user StefanRRichter opened a pull request:

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

          FLINK-5985 [Backport for 1.2] Report no task states for stateless tasks on checkpointing

          This PR is a backport of #3523 to Flink 1.2.

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

          $ git pull https://github.com/StefanRRichter/flink FLINK-5985-backport-to-1.2

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

          https://github.com/apache/flink/pull/3543.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 #3543


          commit 0eb568152583a328c8e1342c94dd6dd2068bacbb
          Author: Stefan Richter <s.richter@data-artisans.com>
          Date: 2017-03-14T15:37:02Z

          FLINK-5985 Report no task states for stateless tasks on checkpointing


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user StefanRRichter opened a pull request: https://github.com/apache/flink/pull/3543 FLINK-5985 [Backport for 1.2] Report no task states for stateless tasks on checkpointing This PR is a backport of #3523 to Flink 1.2. You can merge this pull request into a Git repository by running: $ git pull https://github.com/StefanRRichter/flink FLINK-5985 -backport-to-1.2 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3543.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 #3543 commit 0eb568152583a328c8e1342c94dd6dd2068bacbb Author: Stefan Richter <s.richter@data-artisans.com> Date: 2017-03-14T15:37:02Z FLINK-5985 Report no task states for stateless tasks on checkpointing
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          Yes, it should still work because the changes on `RocksDBKeyedStateBackend` are purely cosmetical without changing any functionality.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3523 Yes, it should still work because the changes on `RocksDBKeyedStateBackend` are purely cosmetical without changing any functionality.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          Hm, one potential pitfall that I see is operator chaining, in case your stateless operators are chained together with stateful ones. But then again, you said it works locally?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3523 Hm, one potential pitfall that I see is operator chaining, in case your stateless operators are chained together with stateful ones. But then again, you said it works locally?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3523#discussion_r106161900

          — Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java —
          @@ -640,6 +640,74 @@ public SubtaskState answer(InvocationOnMock invocation) throws Throwable

          { verify(rawOperatorStateHandle).discardState(); }

          + /**
          + * FLINK-5985
          + *
          + * This test ensures that empty snapshots (no op/keyed stated whatsoever) will be reported as stateless tasks. This
          + * happens by translating an empty

          {@link SubtaskState}

          into reporting 'null' to #acknowledgeCheckpoint.
          + */
          + @Test
          + public void testEmptySubtaskStateLeadsToStatelessAcknowledgment() throws Exception {
          — End diff –

          I think the fact that this test requires extensive whiteboxing means we should move the whole `CheckpointOperation` to a separate class and make it work independent of `StreamTask`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3523#discussion_r106161900 — Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java — @@ -640,6 +640,74 @@ public SubtaskState answer(InvocationOnMock invocation) throws Throwable { verify(rawOperatorStateHandle).discardState(); } + /** + * FLINK-5985 + * + * This test ensures that empty snapshots (no op/keyed stated whatsoever) will be reported as stateless tasks. This + * happens by translating an empty {@link SubtaskState} into reporting 'null' to #acknowledgeCheckpoint. + */ + @Test + public void testEmptySubtaskStateLeadsToStatelessAcknowledgment() throws Exception { — End diff – I think the fact that this test requires extensive whiteboxing means we should move the whole `CheckpointOperation` to a separate class and make it work independent of `StreamTask`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

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

          Good fix!

          I think that verifying the possibility to reconfigure a job with respect to stateless operators warrants an ITCase. Can we extend the `SavepointITCase` for that?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3543 Good fix! I think that verifying the possibility to reconfigure a job with respect to stateless operators warrants an ITCase. Can we extend the `SavepointITCase` for that?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

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

          I think that verifying the possibility to reconfigure a job with respect to stateless operators warrants an ITCase. Can we extend the `SavepointITCase` for that?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3523 I think that verifying the possibility to reconfigure a job with respect to stateless operators warrants an ITCase. Can we extend the `SavepointITCase` for that?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on a diff in the pull request:

          https://github.com/apache/flink/pull/3523#discussion_r106185325

          — Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java —
          @@ -640,6 +640,74 @@ public SubtaskState answer(InvocationOnMock invocation) throws Throwable

          { verify(rawOperatorStateHandle).discardState(); }

          + /**
          + * FLINK-5985
          + *
          + * This test ensures that empty snapshots (no op/keyed stated whatsoever) will be reported as stateless tasks. This
          + * happens by translating an empty

          {@link SubtaskState}

          into reporting 'null' to #acknowledgeCheckpoint.
          + */
          + @Test
          + public void testEmptySubtaskStateLeadsToStatelessAcknowledgment() throws Exception {
          — End diff –

          Yes, we could separate the classes. `CheckpointOperation` is already a static inner class anyways. I would suggest to do this in a followup.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on a diff in the pull request: https://github.com/apache/flink/pull/3523#discussion_r106185325 — Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java — @@ -640,6 +640,74 @@ public SubtaskState answer(InvocationOnMock invocation) throws Throwable { verify(rawOperatorStateHandle).discardState(); } + /** + * FLINK-5985 + * + * This test ensures that empty snapshots (no op/keyed stated whatsoever) will be reported as stateless tasks. This + * happens by translating an empty {@link SubtaskState} into reporting 'null' to #acknowledgeCheckpoint. + */ + @Test + public void testEmptySubtaskStateLeadsToStatelessAcknowledgment() throws Exception { — End diff – Yes, we could separate the classes. `CheckpointOperation` is already a static inner class anyways. I would suggest to do this in a followup.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

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

          Can we slightly adapt the test to target more the typical use case:

          • Original job has some stateless ops (no uid), and some stateful ones (with uid)
          • Create a modified job that has the same stateful ones (same uids) but different stateless ones

          Otherwise this looks good.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3523 Can we slightly adapt the test to target more the typical use case: Original job has some stateless ops (no uid), and some stateful ones (with uid) Create a modified job that has the same stateful ones (same uids) but different stateless ones Otherwise this looks good.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

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

          Looks good.

          +1 for merging this!

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3543 Looks good. +1 for merging this!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          Thanks for the review @StephanEwen. I updated the test as suggested. Merging this now.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3523 Thanks for the review @StephanEwen. I updated the test as suggested. Merging this now.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter closed the pull request at: https://github.com/apache/flink/pull/3523
          Hide
          srichter Stefan Richter added a comment -

          fixed in 20fff32.

          Show
          srichter Stefan Richter added a comment - fixed in 20fff32.
          Hide
          aljoscha Aljoscha Krettek added a comment -

          Stefan Richter Did you also put this onto the release-1.2 branch? I'm asking since it's marked as "fixed for 1.2.1".

          Show
          aljoscha Aljoscha Krettek added a comment - Stefan Richter Did you also put this onto the release-1.2 branch? I'm asking since it's marked as "fixed for 1.2.1".
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on the issue:

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

          Could you merge this @StefanRRichter? I think it is one of the last blockers for 1.2.1 and it's a pretty critical issue...

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on the issue: https://github.com/apache/flink/pull/3543 Could you merge this @StefanRRichter? I think it is one of the last blockers for 1.2.1 and it's a pretty critical issue...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StefanRRichter commented on the issue:

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

          Yes, will do today.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3543 Yes, will do today.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

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

          Github user StefanRRichter commented on the issue:

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

          Merging this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3543 Merging this.

            People

            • Assignee:
              srichter Stefan Richter
              Reporter:
              gyfora Gyula Fora
            • Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development