You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by ma...@apache.org on 2016/07/08 18:39:25 UTC

reef git commit: [REEF-1433] Validate Context Stop failure => FailedContext Event

Repository: reef
Updated Branches:
  refs/heads/master 8557f0f2a -> 2de3998cb


[REEF-1433] Validate Context Stop failure => FailedContext Event

This addressed the issue by
  * Rearranging the Exception structure of ContextManager.
  * Adding a test for an Exception in ContextStopHandler.

JIRA:
  [REEF-1433](https://issues.apache.org/jira/browse/REEF-1433)

Pull request:
  This closes #1063


Project: http://git-wip-us.apache.org/repos/asf/reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/reef/commit/2de3998c
Tree: http://git-wip-us.apache.org/repos/asf/reef/tree/2de3998c
Diff: http://git-wip-us.apache.org/repos/asf/reef/diff/2de3998c

Branch: refs/heads/master
Commit: 2de3998cbb77f5a05759d962034b080ec4d092ed
Parents: 8557f0f
Author: Andrew Chung <af...@gmail.com>
Authored: Thu Jun 30 14:29:44 2016 -0700
Committer: Mariia Mykhailova <ma...@apache.org>
Committed: Fri Jul 8 11:35:53 2016 -0700

----------------------------------------------------------------------
 .../Org.Apache.REEF.Common.csproj               |   1 +
 .../Runtime/Evaluator/Context/ContextManager.cs |  56 ++++-
 .../Runtime/Evaluator/Context/ContextRuntime.cs |  41 +--
 .../Context/ContextStopHandlerException.cs      |  34 +++
 .../Functional/Common/Task/NullTask.cs          |  42 ++++
 .../Failure/User/ContextStartExceptionTest.cs   |  20 +-
 .../Failure/User/ContextStopExceptionTest.cs    | 248 +++++++++++++++++++
 .../Org.Apache.REEF.Tests.csproj                |   4 +-
 8 files changed, 399 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/2de3998c/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj b/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj
index 29955d8..cc14799 100644
--- a/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj
+++ b/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj
@@ -148,6 +148,7 @@ under the License.
     <Compile Include="Runtime\Evaluator\Context\ContextRuntime.cs" />
     <Compile Include="Runtime\Evaluator\Context\ContextStartHandlerException.cs" />
     <Compile Include="Runtime\Evaluator\Context\ContextStartImpl.cs" />
+    <Compile Include="Runtime\Evaluator\Context\ContextStopHandlerException.cs" />
     <Compile Include="Runtime\Evaluator\Context\ContextStopImpl.cs" />
     <Compile Include="Runtime\Evaluator\Context\RootContextLauncher.cs" />
     <Compile Include="Runtime\Evaluator\EvaluatorExitLogger.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/2de3998c/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextManager.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextManager.cs b/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextManager.cs
index 28d4e06..4cfee94 100644
--- a/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextManager.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextManager.cs
@@ -20,6 +20,7 @@ using System.Collections.Generic;
 using System.Collections.ObjectModel;
 using System.Globalization;
 using System.Linq;
+using System.Runtime.ExceptionServices;
 using System.Runtime.Serialization;
 using Org.Apache.REEF.Common.Avro;
 using Org.Apache.REEF.Common.Protobuf.ReefProtocol;
@@ -127,8 +128,8 @@ namespace Org.Apache.REEF.Common.Runtime.Evaluator.Context
                 else if (controlMessage.remove_context != null)
                 {
                     LOGGER.Log(Level.Info,
-                            "RemoveContext with id {0}",
-                            controlMessage.remove_context.context_id);
+                        "RemoveContext with id {0}",
+                        controlMessage.remove_context.context_id);
                     RemoveContext(controlMessage.remove_context.context_id);
                 }
                 else if (controlMessage.start_task != null)
@@ -170,7 +171,7 @@ namespace Org.Apache.REEF.Common.Runtime.Evaluator.Context
                         if (_topContext != null)
                         {
                             context = _topContext.GetContextStack()
-                                    .FirstOrDefault(ctx => ctx.Id.Equals(contextMessageProto.context_id));
+                                .FirstOrDefault(ctx => ctx.Id.Equals(contextMessageProto.context_id));
                         }
                     }
 
@@ -202,11 +203,47 @@ namespace Org.Apache.REEF.Common.Runtime.Evaluator.Context
             }
             catch (ContextClientCodeException e)
             {
-                HandleContextException(e, e.ContextId, e.ParentId);
+                if (!e.ParentId.IsPresent())
+                {
+                    // Crash the Evaluator if an error occurs in the root context.
+                    throw;
+                }
+
+                HandleContextException(e, e.ContextId, e.ParentId.Value);
+            }
+            catch (ContextStartHandlerException e)
+            {
+                if (!e.ParentId.IsPresent())
+                {
+                    // Crash the Evaluator if an error occurs in the root context.
+                    ExceptionDispatchInfo.Capture(e.InnerException).Throw();
+                }
+
+                // Send back the InnerException to the Driver.
+                HandleContextException(e.InnerException, e.ContextId, e.ParentId.Value);
             }
             catch (ContextException e)
             {
-                HandleContextException(e.InnerException, e.ContextId, e.ParentId);
+                if (!e.ParentId.IsPresent())
+                {
+                    // Crash the Evaluator if an error occurs in the root context.
+                    ExceptionDispatchInfo.Capture(e.InnerException).Throw();
+                }
+                else
+                {
+                    // Remove the top context.
+                    // We do not need to do this for ContextStartHandlerException or ContextClientCodeException
+                    // since the child Context has not been spawned those Exceptions were thrown.
+                    if (_topContext == null || !_topContext.ParentContext.IsPresent())
+                    {
+                        throw new InvalidOperationException("Top context cannot be null if Parent ID is present.");
+                    }
+
+                    _topContext = _topContext.ParentContext.Value;
+                }
+
+                // Send back the InnerException to the Driver.
+                HandleContextException(e.InnerException, e.ContextId, e.ParentId.Value);
             }
         }
 
@@ -393,24 +430,21 @@ namespace Org.Apache.REEF.Common.Runtime.Evaluator.Context
             _heartBeatManager.OnNext(taskStatus);
         }
 
-        private void HandleContextException(Exception e, string contextId, Optional<string> parentContextId)
+        private void HandleContextException(Exception e, string contextId, string parentContextId)
         {
             lock (_heartBeatManager)
             {
                 LOGGER.Log(Level.Warning, "ContextException", e);
+
                 byte[] exception = ByteUtilities.StringToByteArrays(e.ToString());
                 var contextStatusProto = new ContextStatusProto
                 {
                     context_id = contextId,
                     context_state = ContextStatusProto.State.FAIL,
+                    parent_id = parentContextId,
                     error = exception
                 };
 
-                if (parentContextId.IsPresent())
-                {
-                    contextStatusProto.parent_id = parentContextId.Value;
-                }
-
                 LOGGER.Log(Level.Error, "Sending Heartbeat for a failed context: {0}", contextStatusProto);
                 _heartBeatManager.OnNext(contextStatusProto);
             }

http://git-wip-us.apache.org/repos/asf/reef/blob/2de3998c/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextRuntime.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextRuntime.cs b/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextRuntime.cs
index 953aa54..e05335a 100644
--- a/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextRuntime.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextRuntime.cs
@@ -245,18 +245,7 @@ namespace Org.Apache.REEF.Common.Runtime.Evaluator.Context
                 catch (Exception e)
                 {
                     Utilities.Diagnostics.Exceptions.Caught(e, Level.Error, LOGGER);
-
-                    var childContextId = string.Empty;
-                    try
-                    {
-                        var injector = TangFactory.GetTang().NewInjector(childContextConfiguration);
-                        childContextId = injector.GetNamedInstance<ContextConfigurationOptions.ContextIdentifier, string>();
-                    }
-                    catch (InjectionException)
-                    {
-                        Utilities.Diagnostics.Exceptions.Caught(
-                            e, Level.Error, "Unable to get Context ID from child ContextConfiguration. Using empty string.", LOGGER);
-                    }
+                    var childContextId = GetChildContextId(childContextConfiguration);
 
                     throw new ContextClientCodeException(childContextId, Optional<string>.Of(Id), "Unable to spawn context", e);
                 }
@@ -360,15 +349,33 @@ namespace Org.Apache.REEF.Common.Runtime.Evaluator.Context
                     LOGGER.Log(Level.Warning, "Closing a context because its parent context is being closed.");
                     _childContext.Value.Dispose();
                 }
-                _contextLifeCycle.Close();
-                if (_parentContext.IsPresent())
+
+                try
                 {
-                    ParentContext.Value.ResetChildContext();
+                    _contextLifeCycle.Close();
                 }
+                catch (Exception e)
+                {
+                    const string message = "Encountered Exception in ContextStopHandler.";
+                    if (ParentContext.IsPresent())
+                    {
+                        throw new ContextStopHandlerException(
+                            Id, Optional<string>.Of(ParentContext.Value.Id), message, e);
+                    }
 
-                foreach (var injectedService in _injectedServices.OfType<IDisposable>())
+                    throw new ContextStopHandlerException(Id, Optional<string>.Empty(), message, e);
+                }
+                finally
                 {
-                    injectedService.Dispose();
+                    if (_parentContext.IsPresent())
+                    {
+                        ParentContext.Value.ResetChildContext();
+                    }
+
+                    foreach (var injectedService in _injectedServices.OfType<IDisposable>())
+                    {
+                        injectedService.Dispose();
+                    }
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/reef/blob/2de3998c/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextStopHandlerException.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextStopHandlerException.cs b/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextStopHandlerException.cs
new file mode 100644
index 0000000..1294a26
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Common/Runtime/Evaluator/Context/ContextStopHandlerException.cs
@@ -0,0 +1,34 @@
+\ufeff// 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 permissions and limitations
+// under the License.
+
+using System;
+using Org.Apache.REEF.Utilities;
+
+namespace Org.Apache.REEF.Common.Runtime.Evaluator.Context
+{
+    /// <summary>
+    /// Exception that is thrown when the ContextStopHandler encounters an Exception.
+    /// </summary>
+    internal sealed class ContextStopHandlerException : ContextException
+    {
+        internal ContextStopHandlerException(
+            string contextId, Optional<string> parentId, string message, Exception inner) :
+            base(contextId, parentId, message, inner)
+        {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/2de3998c/lang/cs/Org.Apache.REEF.Tests/Functional/Common/Task/NullTask.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/Common/Task/NullTask.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/Common/Task/NullTask.cs
new file mode 100644
index 0000000..6478f14
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/Common/Task/NullTask.cs
@@ -0,0 +1,42 @@
+\ufeff// 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 permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Common.Tasks;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Tests.Functional.Common.Task
+{
+    /// <summary>
+    /// A Task that doesn't do anything.
+    /// </summary>
+    public sealed class NullTask : ITask
+    {
+        [Inject]
+        private NullTask()
+        {
+        }
+
+        public byte[] Call(byte[] memento)
+        {
+            return null;
+        }
+
+        public void Dispose()
+        {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/2de3998c/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStartExceptionTest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStartExceptionTest.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStartExceptionTest.cs
index 931f3c2..d761596 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStartExceptionTest.cs
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStartExceptionTest.cs
@@ -27,6 +27,7 @@ using Org.Apache.REEF.Tang.Annotations;
 using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Tang.Util;
 using Org.Apache.REEF.Tests.Functional.Bridge.Exceptions;
+using Org.Apache.REEF.Tests.Functional.Common.Task;
 using Org.Apache.REEF.Utilities.Logging;
 using Xunit;
 
@@ -132,7 +133,7 @@ namespace Org.Apache.REEF.Tests.Functional.Failure.User
                 value.ParentContext.Value.SubmitTask(
                     TaskConfiguration.ConfigurationModule
                         .Set(TaskConfiguration.Identifier, TaskId)
-                        .Set(TaskConfiguration.Task, GenericType<ContextStartExceptionTask>.Class)
+                        .Set(TaskConfiguration.Task, GenericType<NullTask>.Class)
                         .Build());
             }
 
@@ -179,23 +180,6 @@ namespace Org.Apache.REEF.Tests.Functional.Failure.User
             }
         }
 
-        private sealed class ContextStartExceptionTask : ITask
-        {
-            [Inject]
-            private ContextStartExceptionTask()
-            {
-            }
-
-            public byte[] Call(byte[] memento)
-            {
-                return null;
-            }
-
-            public void Dispose()
-            {
-            }
-        }
-
         private sealed class ContextStartExceptionHandler : IObserver<IContextStart>
         {
             [Inject]

http://git-wip-us.apache.org/repos/asf/reef/blob/2de3998c/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStopExceptionTest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStopExceptionTest.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStopExceptionTest.cs
new file mode 100644
index 0000000..268da56
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/Failure/User/ContextStopExceptionTest.cs
@@ -0,0 +1,248 @@
+\ufeff// 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 permissions and limitations
+// under the License.
+
+using System;
+using System.Linq;
+using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Common.Events;
+using Org.Apache.REEF.Common.Tasks;
+using Org.Apache.REEF.Driver;
+using Org.Apache.REEF.Driver.Context;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Driver.Task;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Tests.Functional.Bridge.Exceptions;
+using Org.Apache.REEF.Tests.Functional.Common.Task;
+using Org.Apache.REEF.Utilities.Logging;
+using Xunit;
+
+namespace Org.Apache.REEF.Tests.Functional.Failure.User
+{
+    /// <summary>
+    /// This class contains a test that tests whether throwing an Exception in ContextStopHandler behaves correctly.
+    /// </summary>
+    [Collection("FunctionalTests")]
+    public sealed class ContextStopExceptionTest : ReefFunctionalTest
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(ContextStopExceptionTest));
+
+        private static readonly string ExpectedException = "ExpectedException";
+        private static readonly string FailEvaluatorContextId = "FailEvaluatorContextId";
+        private static readonly string ContextId0 = "ContextId0";
+        private static readonly string ContextId1 = "ContextId1";
+        private static readonly string TaskId = "TaskId";
+        private static readonly string CompletedTaskReceived = "CompletedTaskReceived";
+        private static readonly string FailedContextReceived = "FailedContextReceived";
+        private static readonly string FailedEvaluatorReceived = "FailedEvaluatorReceived";
+
+        /// <summary>
+        /// This test tests whether throwing an Exception in ContextStopHandler behaves correctly.
+        /// The test requests two Evaluators, the first Evaluator will only have a Root Context on which
+        /// will throw an Exception upon calling ActiveContext.Dispose. The second Evaluator will have two
+        /// stacked Contexts - ContextId1 will be stacked on top of ContextId0. ContextId1 will contain the
+        /// failing ContextStopHandler, in which the Evaluator will pop the Context and submit a Task to ContextId0,
+        /// verify that the Task completes successfully, and verify that the Context is disposed without an Exception.
+        /// </summary>
+        [Fact]
+        [Trait("Priority", "1")]
+        [Trait("Category", "FunctionalGated")]
+        [Trait("Description", "Test throwing an Exception in ContextStopHandler should cause the Driver to receive a ContextFailed event." +
+                              "In the case of the Root Context, the Driver should receive a FailedEvaluator event.")]
+        public void TestContextStopException()
+        {
+            string testFolder = DefaultRuntimeFolder + TestId;
+            TestRun(
+                DriverConfiguration.ConfigurationModule
+                    .Set(DriverConfiguration.OnDriverStarted, GenericType<ContextStopExceptionDriver>.Class)
+                    .Set(DriverConfiguration.OnEvaluatorAllocated, GenericType<ContextStopExceptionDriver>.Class)
+                    .Set(DriverConfiguration.OnEvaluatorFailed, GenericType<ContextStopExceptionDriver>.Class)
+                    .Set(DriverConfiguration.OnContextActive, GenericType<ContextStopExceptionDriver>.Class)
+                    .Set(DriverConfiguration.OnContextFailed, GenericType<ContextStopExceptionDriver>.Class)
+                    .Set(DriverConfiguration.OnTaskCompleted, GenericType<ContextStopExceptionDriver>.Class)
+                    .Build(),
+                typeof(ContextStopExceptionDriver), 1, "ContextStopExceptionTest", "local", testFolder);
+
+            ValidateSuccessForLocalRuntime(numberOfContextsToClose: 3, numberOfTasksToFail: 0, numberOfEvaluatorsToFail: 1, testFolder: testFolder);
+            var driverMessages = new[]
+            {
+                CompletedTaskReceived,
+                FailedContextReceived,
+                FailedEvaluatorReceived
+            };
+
+            ValidateMessagesSuccessfullyLoggedForDriver(driverMessages, testFolder);
+            CleanUp(testFolder);
+        }
+
+        private sealed class ContextStopExceptionDriver : 
+            IObserver<IDriverStarted>, 
+            IObserver<IAllocatedEvaluator>,
+            IObserver<IActiveContext>,
+            IObserver<IFailedContext>,
+            IObserver<IFailedEvaluator>,
+            IObserver<ICompletedTask>
+        {
+            private readonly IEvaluatorRequestor _requestor;
+            private readonly object _lock = new object();
+            private bool _shouldSubmitFailEvaluatorContext = true;
+
+            [Inject]
+            private ContextStopExceptionDriver(IEvaluatorRequestor requestor)
+            {
+                _requestor = requestor;
+            }
+
+            public void OnNext(IDriverStarted value)
+            {
+                _requestor.Submit(_requestor.NewBuilder().SetNumber(2).Build());
+            }
+
+            public void OnNext(IAllocatedEvaluator value)
+            {
+                lock (_lock)
+                {
+                    if (_shouldSubmitFailEvaluatorContext)
+                    {
+                        // This context should fail the Evaluator upon calling ActiveContext.Dispose().
+                        value.SubmitContext(
+                            GetContextStopExceptionContextConfiguration(FailEvaluatorContextId));
+                        _shouldSubmitFailEvaluatorContext = false;
+                    }
+                    else
+                    {
+                        // This is the Context that will be stacked upon by ContextId1.
+                        value.SubmitContext(
+                            ContextConfiguration.ConfigurationModule
+                                .Set(ContextConfiguration.Identifier, ContextId0)
+                                .Build());
+                    }
+                }
+            }
+
+            /// <summary>
+            /// This will be ContextId1.
+            /// It will submit a Task to the parent of Context with ContextId1.
+            /// </summary>
+            public void OnNext(IFailedContext value)
+            {
+                Assert.Equal(ContextId1, value.Id);
+                Assert.True(value.ParentContext.IsPresent());
+                Assert.Equal(ContextId0, value.ParentContext.Value.Id);
+
+                // TODO[JIRA REEF-1468]: Validate that Exception is properly serialized.
+                Logger.Log(Level.Info, FailedContextReceived);
+
+                value.ParentContext.Value.SubmitTask(
+                    TaskConfiguration.ConfigurationModule
+                        .Set(TaskConfiguration.Identifier, TaskId)
+                        .Set(TaskConfiguration.Task, GenericType<NullTask>.Class)
+                        .Build());
+            }
+
+            /// <summary>
+            /// This will be a FailedEvaluator generated by the Context with ID as FailEvaluatorContextId.
+            /// </summary>
+            public void OnNext(IFailedEvaluator value)
+            {
+                Assert.Equal(1, value.FailedContexts.Count);
+                Assert.Equal(FailEvaluatorContextId, value.FailedContexts.First().Id);
+                Assert.NotNull(value.EvaluatorException.InnerException);
+                Assert.True(value.EvaluatorException.InnerException is TestSerializableException);
+                Assert.Equal(ExpectedException, value.EvaluatorException.InnerException.Message);
+                Logger.Log(Level.Info, FailedEvaluatorReceived);
+            }
+
+            public void OnNext(IActiveContext value)
+            {
+                if (value.Id.Equals(FailEvaluatorContextId))
+                {
+                    // Close context and trigger failure immediately.
+                    value.Dispose();
+                }
+                else
+                {
+                    if (value.Id.Equals(ContextId0))
+                    {
+                        // Stack Context with ContextId1 on top of Context with ContextId0.
+                        value.SubmitContext(GetContextStopExceptionContextConfiguration(ContextId1));
+                    }
+                    else
+                    {
+                        // Verify the stacked Context and close it.
+                        Assert.Equal(ContextId1, value.Id);
+                        value.Dispose();
+                    }
+                }
+            }
+
+            public void OnNext(ICompletedTask value)
+            {
+                // Verify the completion of Task on Context with ContextId0.
+                Assert.Equal(TaskId, value.Id);
+                Assert.Equal(ContextId0, value.ActiveContext.Id);
+                Logger.Log(Level.Info, CompletedTaskReceived);
+                value.ActiveContext.Dispose();
+            }
+
+            public void OnError(Exception error)
+            {
+                throw new NotImplementedException();
+            }
+
+            public void OnCompleted()
+            {
+                throw new NotImplementedException();
+            }
+
+            private static IConfiguration GetContextStopExceptionContextConfiguration(string contextId)
+            {
+                return ContextConfiguration.ConfigurationModule
+                    .Set(ContextConfiguration.Identifier, contextId)
+                    .Set(ContextConfiguration.OnContextStop, GenericType<ContextStopExceptionHandler>.Class)
+                    .Build();
+            }
+        }
+
+        /// <summary>
+        /// A ContextStopHandler that throws a Serializable Exception.
+        /// </summary>
+        private sealed class ContextStopExceptionHandler : IObserver<IContextStop>
+        {
+            [Inject]
+            private ContextStopExceptionHandler()
+            {
+            }
+
+            public void OnNext(IContextStop value)
+            {
+                throw new TestSerializableException(ExpectedException);
+            }
+
+            public void OnError(Exception error)
+            {
+                throw new NotImplementedException();
+            }
+
+            public void OnCompleted()
+            {
+                throw new NotImplementedException();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/2de3998c/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
index 6db6210..64e6c89 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
@@ -79,13 +79,15 @@ under the License.
     <Compile Include="Functional\Bridge\TestCloseTask.cs" />
     <Compile Include="Functional\Bridge\TestContextStack.cs" />
     <Compile Include="Functional\Bridge\TestFailedEvaluatorEventHandler.cs" />
+    <Compile Include="Functional\Common\Task\ExceptionTask.cs" />
+    <Compile Include="Functional\Common\Task\NullTask.cs" />
+    <Compile Include="Functional\Failure\User\ContextStopExceptionTest.cs" />
     <Compile Include="Functional\Common\EventMonitor.cs" />
     <Compile Include="Functional\Failure\User\SendContextMessageExceptionTest.cs" />
     <Compile Include="Functional\Failure\User\ServiceConstructorExceptionTest.cs" />
     <Compile Include="Functional\Failure\User\ReceiveContextMessageExceptionTest.cs" />
     <Compile Include="Functional\Failure\User\ContextStartExceptionTest.cs" />
     <Compile Include="Functional\Failure\User\ReceiveTaskMessageExceptionTest.cs" />
-    <Compile Include="Functional\Common\Task\ExceptionTask.cs" />
     <Compile Include="Functional\Failure\User\TaskCallExceptionTest.cs" />
     <Compile Include="Functional\Bridge\Exceptions\TestNonSerializableException.cs" />
     <Compile Include="Functional\Bridge\Exceptions\TestSerializableException.cs" />