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

Cleanup StreamTaskStates if a checkpoint operation of a subsequent operator fails

    Details

      Description

      Due to chaining, a StreamTask needs to checkpoint multiple operators. If the first operators succeed in creating a checkpoint but a subsequent operator in the chain fails, the StreamTask has to clean up the already completed checkpoints. Otherwise we might end up with orphaned state data.

        Issue Links

          Activity

          Hide
          aljoscha Aljoscha Krettek added a comment -

          @Till, this is already covered in this PR: https://github.com/apache/flink/pull/2857. It's not marked as this because it came up while fixing another thing. Could you maybe have a look at the StreamTask changes in there, I think this completely covers this issue.

          When merging the PR I will split those changes into a separate commit and tag it with this issue number.

          Show
          aljoscha Aljoscha Krettek added a comment - @Till, this is already covered in this PR: https://github.com/apache/flink/pull/2857 . It's not marked as this because it came up while fixing another thing. Could you maybe have a look at the StreamTask changes in there, I think this completely covers this issue. When merging the PR I will split those changes into a separate commit and tag it with this issue number.
          Hide
          aljoscha Aljoscha Krettek added a comment -

          Stefan Richter, please correct me if I'm wrong.

          Show
          aljoscha Aljoscha Krettek added a comment - Stefan Richter , please correct me if I'm wrong.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user tillrohrmann opened a pull request:

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

          FLINK-5229 [tm] Discard completed checkpoints if a subsequent operator's checkpoint fails

          In case of a failure of any StreamOperator#snapshotState method, all up to this point created
          StreamTaskStates will be discarded. This ensures that a failing checkpoint operation of a chained
          operator won't leave orphaned checkpoint data behind.

          cc @uce, @StephanEwen.

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

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

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

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


          commit 3ce8cec2cb3dc0c4979ab2cbe30dc1db8d988111
          Author: Till Rohrmann <trohrmann@apache.org>
          Date: 2016-12-02T14:33:06Z

          FLINK-5229 [tm] Discard completed checkpoints if a subsequent operator's checkpoint fails

          In case of a failure of any StreamOperator#snapshotState method, all up to this point created
          StreamTaskStates are discarded. This ensures that a failing checkpoint operation of a chained
          operator won't leave orphaned checkpoint data behind.


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tillrohrmann opened a pull request: https://github.com/apache/flink/pull/2924 FLINK-5229 [tm] Discard completed checkpoints if a subsequent operator's checkpoint fails In case of a failure of any StreamOperator#snapshotState method, all up to this point created StreamTaskStates will be discarded. This ensures that a failing checkpoint operation of a chained operator won't leave orphaned checkpoint data behind. cc @uce, @StephanEwen. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tillrohrmann/flink fixCheckpointCleanup Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2924.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 #2924 commit 3ce8cec2cb3dc0c4979ab2cbe30dc1db8d988111 Author: Till Rohrmann <trohrmann@apache.org> Date: 2016-12-02T14:33:06Z FLINK-5229 [tm] Discard completed checkpoints if a subsequent operator's checkpoint fails In case of a failure of any StreamOperator#snapshotState method, all up to this point created StreamTaskStates are discarded. This ensures that a failing checkpoint operation of a chained operator won't leave orphaned checkpoint data behind.
          Hide
          till.rohrmann Till Rohrmann added a comment -

          Thanks for the pointer Aljoscha Krettek. The PR I've opened is for the release-1.1 branch.

          I've checked the PR and it looks as if some of the work has been done. However, it is not yet exhaustive (e.g. the non partitionable state handles are not discarded properly and the error reporting can be improved). I'll base my work in this branch then. So please don't split the commit up.

          Show
          till.rohrmann Till Rohrmann added a comment - Thanks for the pointer Aljoscha Krettek . The PR I've opened is for the release-1.1 branch. I've checked the PR and it looks as if some of the work has been done. However, it is not yet exhaustive (e.g. the non partitionable state handles are not discarded properly and the error reporting can be improved). I'll base my work in this branch then. So please don't split the commit up.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user uce commented on the issue:

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

          Tested this with a job that fails during checkpointing and it works like a charm for both chained and non chained state, but I found another issue with discarding pending checkpoints. The task states map is cleared although it is given the async discarder. +1 to fix this related issue and merge this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user uce commented on the issue: https://github.com/apache/flink/pull/2924 Tested this with a job that fails during checkpointing and it works like a charm for both chained and non chained state, but I found another issue with discarding pending checkpoints. The task states map is cleared although it is given the async discarder. +1 to fix this related issue and merge this.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann commented on the issue:

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

          Thanks for testing it @uce and @StephanEwen. Really good finding with the bug. I'll fix it and then I will merge the branch.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/2924 Thanks for testing it @uce and @StephanEwen. Really good finding with the bug. I'll fix it and then I will merge the branch.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

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

          GitHub user tillrohrmann opened a pull request:

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

          FLINK-5229 [state] Cleanup of operator snapshots if subsequent operator snapshots fail

          This PR is based on #3178.

          This PR adds operator state cleanup to the StreamTask class. If a stream task contains multiple
          stream operators, then every operator is checkpointed. In case that a snapshot operation fails
          all state handles and OperatorSnapshotResults belonging to previous operators have to be freed.

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

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

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

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


          commit c1a597320dbe3c7f4514297d5ad7f2b8f416e287
          Author: Till Rohrmann <trohrmann@apache.org>
          Date: 2016-12-01T12:25:05Z

          FLINK-5214 Clean up checkpoint data in case of a failing checkpoint operation

          Adds exception handling to the stream operators for the snapshotState method. A failing
          snapshot operation will trigger the clean up of all so far generated state resources.
          This will avoid that in case of a failing snapshot operation resources (e.g. files) are
          left behind.

          Add test case for OperatorSnapshotResult

          Add StateSnapshotContextSynchronousImplTest

          Add AbstractStreamOperator failing snapshot tests

          commit b407e812ccb9c727ee0df8d130b54a50abee5b06
          Author: Till Rohrmann <trohrmann@apache.org>
          Date: 2017-01-20T13:28:44Z

          FLINK-5229 [state] Cleanup of operator snapshots if subsequent operator snapshots fail

          This PR adds operator state cleanup to the StreamTask class. If a stream task contains multiple
          stream operators, then every operator is checkpointed. In case that a snapshot operation fails
          all state handles and OperatorSnapshotResults belonging to previous operators have to be freed.

          Add test cases for failing checkpoint operations in StreamTask


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tillrohrmann opened a pull request: https://github.com/apache/flink/pull/3179 FLINK-5229 [state] Cleanup of operator snapshots if subsequent operator snapshots fail This PR is based on #3178. This PR adds operator state cleanup to the StreamTask class. If a stream task contains multiple stream operators, then every operator is checkpointed. In case that a snapshot operation fails all state handles and OperatorSnapshotResults belonging to previous operators have to be freed. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tillrohrmann/flink streamTaskCheckpointCleanup Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3179.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 #3179 commit c1a597320dbe3c7f4514297d5ad7f2b8f416e287 Author: Till Rohrmann <trohrmann@apache.org> Date: 2016-12-01T12:25:05Z FLINK-5214 Clean up checkpoint data in case of a failing checkpoint operation Adds exception handling to the stream operators for the snapshotState method. A failing snapshot operation will trigger the clean up of all so far generated state resources. This will avoid that in case of a failing snapshot operation resources (e.g. files) are left behind. Add test case for OperatorSnapshotResult Add StateSnapshotContextSynchronousImplTest Add AbstractStreamOperator failing snapshot tests commit b407e812ccb9c727ee0df8d130b54a50abee5b06 Author: Till Rohrmann <trohrmann@apache.org> Date: 2017-01-20T13:28:44Z FLINK-5229 [state] Cleanup of operator snapshots if subsequent operator snapshots fail This PR adds operator state cleanup to the StreamTask class. If a stream task contains multiple stream operators, then every operator is checkpointed. In case that a snapshot operation fails all state handles and OperatorSnapshotResults belonging to previous operators have to be freed. Add test cases for failing checkpoint operations in StreamTask
          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/3179#discussion_r97285679

          — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java —
          @@ -954,6 +954,27 @@ public void run()

          { owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); }

          } catch (Exception e) {
          + // clean up ongoing operator snapshot results and non partitioned state handles
          + for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) {
          + if (operatorSnapshotResult != null) {
          + try

          { + operatorSnapshotResult.cancel(); + }

          catch (Exception cancelException)

          { + e.addSuppressed(cancelException); + }

          + }
          + }
          +
          + for (StreamStateHandle nonPartitionedStateHandle : nonPartitionedStateHandles) {
          — End diff –

          You could use the utility method `StateUtil.bestEffortDiscardAllStateObjects(...)` to reduce code duplication a bit.

          I think we should also discard other state objects that completed fuctures could have created, e.g. `StateUtil.bestEffortDiscardAllStateObjects(operatorStatesBackend);` etc. , what do you think?

          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/3179#discussion_r97285679 — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java — @@ -954,6 +954,27 @@ public void run() { owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); } } catch (Exception e) { + // clean up ongoing operator snapshot results and non partitioned state handles + for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) { + if (operatorSnapshotResult != null) { + try { + operatorSnapshotResult.cancel(); + } catch (Exception cancelException) { + e.addSuppressed(cancelException); + } + } + } + + for (StreamStateHandle nonPartitionedStateHandle : nonPartitionedStateHandles) { — End diff – You could use the utility method `StateUtil.bestEffortDiscardAllStateObjects(...)` to reduce code duplication a bit. I think we should also discard other state objects that completed fuctures could have created, e.g. `StateUtil.bestEffortDiscardAllStateObjects(operatorStatesBackend);` etc. , what do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3179#discussion_r97315623

          — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java —
          @@ -954,6 +954,27 @@ public void run()

          { owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); }

          } catch (Exception e) {
          + // clean up ongoing operator snapshot results and non partitioned state handles
          + for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) {
          + if (operatorSnapshotResult != null) {
          + try

          { + operatorSnapshotResult.cancel(); + }

          catch (Exception cancelException)

          { + e.addSuppressed(cancelException); + }

          + }
          + }
          +
          + for (StreamStateHandle nonPartitionedStateHandle : nonPartitionedStateHandles) {
          — End diff –

          Yes you're absolutely right. Will address your comments.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/3179#discussion_r97315623 — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java — @@ -954,6 +954,27 @@ public void run() { owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); } } catch (Exception e) { + // clean up ongoing operator snapshot results and non partitioned state handles + for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) { + if (operatorSnapshotResult != null) { + try { + operatorSnapshotResult.cancel(); + } catch (Exception cancelException) { + e.addSuppressed(cancelException); + } + } + } + + for (StreamStateHandle nonPartitionedStateHandle : nonPartitionedStateHandles) { — End diff – Yes you're absolutely right. Will address your comments.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann commented on the issue:

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

          Thanks for the review @StefanRRichter. Will address your comments.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/3179 Thanks for the review @StefanRRichter. Will address your comments.
          Hide
          githubbot ASF GitHub Bot added a comment -

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

          https://github.com/apache/flink/pull/3179#discussion_r97321764

          — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java —
          @@ -954,6 +954,27 @@ public void run()

          { owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); }

          } catch (Exception e) {
          + // clean up ongoing operator snapshot results and non partitioned state handles
          + for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) {
          + if (operatorSnapshotResult != null) {
          + try

          { + operatorSnapshotResult.cancel(); + }

          catch (Exception cancelException)

          { + e.addSuppressed(cancelException); + }

          + }
          + }
          +
          + for (StreamStateHandle nonPartitionedStateHandle : nonPartitionedStateHandles) {
          — End diff –

          Having a second look, I think the other state objects don't have to be cleaned up, because they are still owned by the corresponding `OperatorSnapshotResult` which is cleaned up in case of a failure or `close` operation.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/3179#discussion_r97321764 — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java — @@ -954,6 +954,27 @@ public void run() { owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); } } catch (Exception e) { + // clean up ongoing operator snapshot results and non partitioned state handles + for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) { + if (operatorSnapshotResult != null) { + try { + operatorSnapshotResult.cancel(); + } catch (Exception cancelException) { + e.addSuppressed(cancelException); + } + } + } + + for (StreamStateHandle nonPartitionedStateHandle : nonPartitionedStateHandles) { — End diff – Having a second look, I think the other state objects don't have to be cleaned up, because they are still owned by the corresponding `OperatorSnapshotResult` which is cleaned up in case of a failure or `close` operation.
          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/3179#discussion_r97322137

          — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java —
          @@ -954,6 +954,27 @@ public void run()

          { owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); }

          } catch (Exception e) {
          + // clean up ongoing operator snapshot results and non partitioned state handles
          + for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) {
          + if (operatorSnapshotResult != null) {
          + try

          { + operatorSnapshotResult.cancel(); + }

          catch (Exception cancelException)

          { + e.addSuppressed(cancelException); + }

          + }
          + }
          +
          + for (StreamStateHandle nonPartitionedStateHandle : nonPartitionedStateHandles) {
          — End diff –

          Agreed

          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/3179#discussion_r97322137 — Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java — @@ -954,6 +954,27 @@ public void run() { owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); } } catch (Exception e) { + // clean up ongoing operator snapshot results and non partitioned state handles + for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) { + if (operatorSnapshotResult != null) { + try { + operatorSnapshotResult.cancel(); + } catch (Exception cancelException) { + e.addSuppressed(cancelException); + } + } + } + + for (StreamStateHandle nonPartitionedStateHandle : nonPartitionedStateHandles) { — End diff – Agreed
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tillrohrmann commented on the issue:

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

          I've addressed your comments @StefanRRichter. Note that I did not add the additional state object cleanup, because it was already done by cancelling the `OperatorSnapshotResults`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/3179 I've addressed your comments @StefanRRichter. Note that I did not add the additional state object cleanup, because it was already done by cancelling the `OperatorSnapshotResults`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

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

          Till Rohrmann can this be closed now or are you waiting until it's merged on release-1.2?

          Show
          aljoscha Aljoscha Krettek added a comment - Till Rohrmann can this be closed now or are you waiting until it's merged on release-1.2?
          Hide
          till.rohrmann Till Rohrmann added a comment -

          1.3.0: cfb95b9074a05686f5ad290b2aaa4be89536a35b
          1.2.0: 840b779c542462cf7bd4bed40620dd68e90ec6bd
          1.1.4: 020da2ce1c8be83789252d0db959896a761d7513

          Show
          till.rohrmann Till Rohrmann added a comment - 1.3.0: cfb95b9074a05686f5ad290b2aaa4be89536a35b 1.2.0: 840b779c542462cf7bd4bed40620dd68e90ec6bd 1.1.4: 020da2ce1c8be83789252d0db959896a761d7513
          Hide
          rmetzger Robert Metzger added a comment -

          I'm seeing the following exceptions in my 1.2.0 RC1 code

          2017-01-25 12:47:48,775 WARN  org.apache.flink.streaming.runtime.tasks.StreamTask           - Could not properly clean up the async checkpoint runnable.
          java.lang.Exception: Could not properly cancel managed operator state future.
                  at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:99)
                  at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:992)
                  at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.close(StreamTask.java:979)
                  at org.apache.flink.util.IOUtils.closeQuietly(IOUtils.java:224)
                  at org.apache.flink.util.AbstractCloseableRegistry.close(AbstractCloseableRegistry.java:92)
                  at org.apache.flink.streaming.runtime.tasks.StreamTask.cancel(StreamTask.java:363)
                  at org.apache.flink.runtime.taskmanager.Task$TaskCanceler.run(Task.java:1384)
                  at java.lang.Thread.run(Thread.java:745)
          Caused by: java.lang.NullPointerException
                  at org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:81)
                  at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:97)
                  ... 7 more
          

          I'm asking here because its code that has been touched while addressing this JIRA.

          How critical is this?

          Show
          rmetzger Robert Metzger added a comment - I'm seeing the following exceptions in my 1.2.0 RC1 code 2017-01-25 12:47:48,775 WARN org.apache.flink.streaming.runtime.tasks.StreamTask - Could not properly clean up the async checkpoint runnable. java.lang.Exception: Could not properly cancel managed operator state future . at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:99) at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:992) at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.close(StreamTask.java:979) at org.apache.flink.util.IOUtils.closeQuietly(IOUtils.java:224) at org.apache.flink.util.AbstractCloseableRegistry.close(AbstractCloseableRegistry.java:92) at org.apache.flink.streaming.runtime.tasks.StreamTask.cancel(StreamTask.java:363) at org.apache.flink.runtime.taskmanager.Task$TaskCanceler.run(Task.java:1384) at java.lang. Thread .run( Thread .java:745) Caused by: java.lang.NullPointerException at org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:81) at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:97) ... 7 more I'm asking here because its code that has been touched while addressing this JIRA. How critical is this?
          Hide
          till.rohrmann Till Rohrmann added a comment -

          Should be fixed. Apparently, there are state handle futures which contain null values.

          Show
          till.rohrmann Till Rohrmann added a comment - Should be fixed. Apparently, there are state handle futures which contain null values.
          Hide
          till.rohrmann Till Rohrmann added a comment -

          Will be addressed with FLINK-5643.

          Show
          till.rohrmann Till Rohrmann added a comment - Will be addressed with FLINK-5643 .
          Hide
          rmetzger Robert Metzger added a comment -

          Thank you!

          Show
          rmetzger Robert Metzger added a comment - Thank you!

            People

            • Assignee:
              till.rohrmann Till Rohrmann
              Reporter:
              till.rohrmann Till Rohrmann
            • Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development