You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by do...@apache.org on 2018/03/10 01:33:23 UTC

[14/28] reef git commit: [REEF-1951] Make Driver Status more robust.

[REEF-1951] Make Driver Status more robust.

This change makes the communication of the Driver status between the
Driver and the .NET Client more robust. It gueards against the Client
calling the HTTP server to late by changes on the Java side. It guards
against calling the HTTP server to early by changes on the .NET side.

On the *Java side*, this introduces a new alarm on the clock that makes
sure that `DriverStatusHTTPHandler` is called at least once via HTTP in
2s after launching.

On the *.NET Side*, this introduces a retry loop around the first
attempt to connect to the Driver. This makes sure we don't call it too
early.

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

Pull Request:
  This closes #1413


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

Branch: refs/heads/REEF-335
Commit: 3d571e6a4bd85d095cca5bd2f903c2acf67a5f4f
Parents: e36ee92
Author: Markus Weimer <we...@apache.org>
Authored: Mon Nov 6 11:00:07 2017 -0800
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Tue Nov 7 18:16:25 2017 -0800

----------------------------------------------------------------------
 .../Org.Apache.REEF.Client/API/DriverStatus.cs  |  5 ++
 .../cs/Org.Apache.REEF.Client/API/Parameters.cs | 41 ++++++++++++
 .../Common/IJobSubmissionResult.cs              |  2 +-
 .../Common/JobSubmissionResult.cs               | 42 +++++++++++-
 .../Org.Apache.REEF.Client/Local/LocalClient.cs | 21 +++++-
 .../Local/LocalJobSubmissionResult.cs           |  7 +-
 .../Org.Apache.REEF.Client.csproj               |  1 +
 .../YARN/YARNREEFClient.cs                      | 21 +++++-
 .../YARN/YarnJobSubmissionResult.cs             |  7 +-
 .../bridge/client/DriverStatusHTTPHandler.java  | 70 +++++++++++++++++++-
 .../Parameters/HTTPStatusAlarmInterval.java     | 33 +++++++++
 .../Parameters/HTTPStatusNumberOfRetries.java   | 33 +++++++++
 .../bridge/client/Parameters/package-info.java  | 23 +++++++
 .../client/TestDriverStatusHTTPHandler.java     | 14 ++--
 14 files changed, 303 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs b/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
index d9934a9..2978382 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
@@ -23,6 +23,11 @@ namespace Org.Apache.REEF.Client.API
     internal enum DriverStatus
     {
         /// <summary>
+        /// Represents the fact that the Driver status hasn't been received yet.
+        /// </summary>
+        UNKNOWN,
+
+        /// <summary>
         /// Driver is initializing.
         /// </summary>
         INIT,

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/API/Parameters.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Parameters.cs b/lang/cs/Org.Apache.REEF.Client/API/Parameters.cs
new file mode 100644
index 0000000..0b6db60
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Parameters.cs
@@ -0,0 +1,41 @@
+// 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.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.API.Parameters
+{
+    /// <summary>
+    /// Interval ins ms between connection attempts to the Driver's HTTP Server to obtain Status information.
+    /// </summary>
+    [NamedParameter(DefaultValue = "500", Documentation = 
+        "Interval ins ms between connection attempts to the Driver's HTTP Server to obtain Status information.")]
+    internal sealed class DriverHTTPConnectionRetryInterval : Name<int>
+    {
+        // Intentionally empty
+    }
+
+    /// <summary>
+    /// Number of Retries when connecting to the Driver's HTTP server.
+    /// </summary>
+    [NamedParameter(DefaultValue = "10",
+        Documentation = "Number of Retries when connecting to the Driver's HTTP server.")]
+    internal sealed class DriverHTTPConnectionAttempts : Name<int>
+    {
+        // Intentionally empty
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
index 328677a..e3510ed 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
@@ -49,7 +49,7 @@ namespace Org.Apache.REEF.Client.Common
         /// <summary>
         /// Waits for the Driver to complete.
         /// </summary>
-        /// <exception cref="System.Net.WebException">If the Driver cannot be reached.</exception>
+        /// <exception cref="System.Net.WebException">If the Driver could be reached at least once.</exception>
         [Unstable("0.17", "Uses the HTTP server in the Java Driver. Might not work if that cannot be reached.")]
         void WaitForDriverToFinish();
     }

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
index 3880433..0649422 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
@@ -23,10 +23,17 @@ using System.Net.Http;
 using System.Net.Http.Headers;
 using System.Threading;
 using System.Threading.Tasks;
+#if DOTNET_BUILD
+using Microsoft.Practices.EnterpriseLibrary.TransientFaultHandling;
+#else
+using Microsoft.Practices.TransientFaultHandling;
+#endif
 using Newtonsoft.Json;
 using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.YARN.RestClient;
 using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
 using Org.Apache.REEF.Utilities.Logging;
+using HttpClient = System.Net.Http.HttpClient;
 
 namespace Org.Apache.REEF.Client.Common
 {
@@ -48,7 +55,17 @@ namespace Org.Apache.REEF.Client.Common
         private readonly HttpClient _client;
         private readonly IREEFClient _reefClient;
 
-        internal JobSubmissionResult(IREEFClient reefClient, string filePath)
+        /// <summary>
+        /// Number of retries when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _numberOfRetries;
+
+        /// <summary>
+        /// Retry interval in ms when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly TimeSpan _retryInterval;
+
+        internal JobSubmissionResult(IREEFClient reefClient, string filePath, int numberOfRetries, int retryInterval)
         {
             _reefClient = reefClient;
             _client = new HttpClient
@@ -58,6 +75,9 @@ namespace Org.Apache.REEF.Client.Common
             _client.DefaultRequestHeaders.Accept.Add(new MediaTypeWithQualityHeaderValue(AppJson));
 
             _driverUrl = GetDriverUrl(filePath);
+
+            _numberOfRetries = numberOfRetries;
+            _retryInterval = TimeSpan.FromMilliseconds(retryInterval);
         }
 
         /// <summary>
@@ -98,7 +118,13 @@ namespace Org.Apache.REEF.Client.Common
 
         public void WaitForDriverToFinish()
         {
-            DriverStatus status = FetchDriverStatus();
+            DriverStatus status = FetchFirstDriverStatus();
+
+            if (DriverStatus.UNKNOWN == status)
+            {
+                // We were unable to connect to the Driver at least once.
+                throw new WebException("Unable to connect to the Driver.");
+            }
             
             while (status.IsActive())
             {
@@ -106,7 +132,7 @@ namespace Org.Apache.REEF.Client.Common
                 {
                     status = FetchDriverStatus();
                 }
-                catch (System.Net.WebException)
+                catch (WebException)
                 {
                     // If we no longer can reach the Driver, it must have exited.
                     status = DriverStatus.UNKNOWN_EXITED;
@@ -126,6 +152,16 @@ namespace Org.Apache.REEF.Client.Common
             }
         }
 
+        /// <summary>
+        /// Fetches the Driver Status for the 1st time.
+        /// </summary>
+        /// <returns>The obtained Driver Status or DriverStatus.UNKNOWN, if the Driver was never reached.</returns>
+        private DriverStatus FetchFirstDriverStatus()
+        {
+            var policy = new RetryPolicy<AllErrorsTransientStrategy>(_numberOfRetries, _retryInterval);
+            return policy.ExecuteAction<DriverStatus>(FetchDriverStatus);
+        }
+
         protected abstract string GetDriverUrl(string filepath);
 
         enum UrlResultKind

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs b/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
index 722bf9b..9f45986 100644
--- a/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
@@ -37,6 +37,7 @@ using Org.Apache.REEF.Utilities.Logging;
 using Org.Apache.REEF.Wake.Remote;
 using Org.Apache.REEF.Wake.Remote.Impl;
 using Org.Apache.REEF.Wake.Remote.Parameters;
+using Org.Apache.REEF.Client.API.Parameters;
 
 namespace Org.Apache.REEF.Client.Local
 {
@@ -64,10 +65,22 @@ namespace Org.Apache.REEF.Client.Local
         private readonly IConfiguration _localConfigurationOnDriver;
         private readonly JobRequestBuilderFactory _jobRequestBuilderFactory;
 
+        /// <summary>
+        /// Number of retries when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _numberOfRetries;
+
+        /// <summary>
+        /// Retry interval in ms when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _retryInterval;
+
         [Inject]
         private LocalClient(DriverFolderPreparationHelper driverFolderPreparationHelper,
             [Parameter(typeof(LocalRuntimeDirectory))] string runtimeFolder,
             [Parameter(typeof(NumberOfEvaluators))] int maxNumberOfConcurrentEvaluators,
+            [Parameter(typeof(DriverHTTPConnectionRetryInterval))]int retryInterval,
+            [Parameter(typeof(DriverHTTPConnectionAttempts))] int numberOfRetries,
             IJavaClientLauncher javaClientLauncher,
             REEFFileNames fileNames,
             JobRequestBuilderFactory jobRequestBuilderFactory)
@@ -75,6 +88,8 @@ namespace Org.Apache.REEF.Client.Local
             _driverFolderPreparationHelper = driverFolderPreparationHelper;
             _runtimeFolder = runtimeFolder;
             _maxNumberOfConcurrentEvaluators = maxNumberOfConcurrentEvaluators;
+            _retryInterval = retryInterval;
+            _numberOfRetries = numberOfRetries;
             _javaClientLauncher = javaClientLauncher;
             _fileNames = fileNames;
             _jobRequestBuilderFactory = jobRequestBuilderFactory;
@@ -95,11 +110,13 @@ namespace Org.Apache.REEF.Client.Local
         private LocalClient(
             DriverFolderPreparationHelper driverFolderPreparationHelper,
             [Parameter(typeof(NumberOfEvaluators))] int numberOfEvaluators,
+            [Parameter(typeof(DriverHTTPConnectionRetryInterval))]int retryInterval,
+            [Parameter(typeof(DriverHTTPConnectionAttempts))] int numberOfRetries,
             IJavaClientLauncher javaClientLauncher,
             REEFFileNames fileNames,
             JobRequestBuilderFactory jobRequestBuilderFactory)
             : this(driverFolderPreparationHelper, Path.GetTempPath(),
-                numberOfEvaluators, javaClientLauncher, fileNames, jobRequestBuilderFactory)
+                numberOfEvaluators, retryInterval, numberOfRetries, javaClientLauncher, fileNames, jobRequestBuilderFactory)
         {
             // Intentionally left blank.
         }
@@ -193,7 +210,7 @@ namespace Org.Apache.REEF.Client.Local
                 .LogAndIgnoreExceptionIfAny(Logger, "Java launcher failed");
 
             var fileName = Path.Combine(driverFolder, _fileNames.DriverHttpEndpoint);
-            JobSubmissionResult result = new LocalJobSubmissionResult(this, fileName);
+            JobSubmissionResult result = new LocalJobSubmissionResult(this, fileName, _numberOfRetries, _retryInterval);
 
             var msg = string.Format(CultureInfo.CurrentCulture,
                 "Submitted the Driver for execution. Returned driverUrl is: {0}.", result.DriverUrl);

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs
index 51d7afd..f5f50c0 100644
--- a/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs
@@ -34,8 +34,11 @@ namespace Org.Apache.REEF.Client.Local
 
         private const string UriTemplate = @"http://{0}/";
 
-        internal LocalJobSubmissionResult(IREEFClient reefClient, string filePath) 
-            : base(reefClient, filePath)
+        internal LocalJobSubmissionResult(IREEFClient reefClient, 
+            string filePath, 
+            int numberOfRetries, 
+            int retryInterval) 
+            : base(reefClient, filePath, numberOfRetries, retryInterval)
         {
         }
 

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
index 80b8d90..879b8c3 100644
--- a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
+++ b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
@@ -74,6 +74,7 @@ under the License.
     <Compile Include="API\JobRequest.cs" />
     <Compile Include="API\JobRequestBuilder.cs" />
     <Compile Include="API\JobRequestBuilderFactory.cs" />
+    <Compile Include="API\Parameters.cs" />
     <Compile Include="API\TcpPortConfigurationModule.cs" />
     <Compile Include="Avro\AvroAppSubmissionParameters.cs" />
     <Compile Include="Avro\AvroJobSubmissionParameters.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
index 09434f0..554914a 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
@@ -31,6 +31,8 @@ using Org.Apache.REEF.Common.Files;
 using Org.Apache.REEF.Tang.Annotations;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Client.API.Parameters;
+using Org.Apache.REEF.Client.Local.Parameters;
 
 namespace Org.Apache.REEF.Client.Yarn
 {
@@ -49,6 +51,16 @@ namespace Org.Apache.REEF.Client.Yarn
         private readonly YarnREEFParamSerializer _paramSerializer;
         private readonly JobRequestBuilderFactory _jobRequestBuilderFactory;
 
+        /// <summary>
+        /// Number of retries when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _numberOfRetries;
+
+        /// <summary>
+        /// Retry interval in ms when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _retryInterval;
+
         [Inject]
         internal YarnREEFClient(IJavaClientLauncher javaClientLauncher,
             DriverFolderPreparationHelper driverFolderPreparationHelper,
@@ -56,7 +68,9 @@ namespace Org.Apache.REEF.Client.Yarn
             YarnCommandLineEnvironment yarn,
             IYarnRMClient yarnClient,
             YarnREEFParamSerializer paramSerializer,
-            JobRequestBuilderFactory jobRequestBuilderFactory)
+            JobRequestBuilderFactory jobRequestBuilderFactory,
+            [Parameter(typeof(DriverHTTPConnectionRetryInterval))]int retryInterval,
+            [Parameter(typeof(DriverHTTPConnectionAttempts))] int numberOfRetries)
         {
             _javaClientLauncher = javaClientLauncher;
             _javaClientLauncher.AddToClassPath(yarn.GetYarnClasspathList());
@@ -65,6 +79,8 @@ namespace Org.Apache.REEF.Client.Yarn
             _yarnClient = yarnClient;
             _paramSerializer = paramSerializer;
             _jobRequestBuilderFactory = jobRequestBuilderFactory;
+            _retryInterval = retryInterval;
+            _numberOfRetries = numberOfRetries;
         }
 
         public void Submit(JobRequest jobRequest)
@@ -90,7 +106,8 @@ namespace Org.Apache.REEF.Client.Yarn
             Launch(jobRequest, driverFolderPath);
 
             var pointerFileName = Path.Combine(driverFolderPath, _fileNames.DriverHttpEndpoint);
-            var jobSubmitionResultImpl = new YarnJobSubmissionResult(this, pointerFileName);
+            var jobSubmitionResultImpl = new YarnJobSubmissionResult(this, 
+                pointerFileName, _numberOfRetries, _retryInterval);
 
             var msg = string.Format(CultureInfo.CurrentCulture,
                 "Submitted the Driver for execution. Returned driverUrl is: {0}, appId is {1}.",

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs
index df9fd8d..617874a 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs
@@ -24,8 +24,11 @@ namespace Org.Apache.REEF.Client.YARN
 {
     internal class YarnJobSubmissionResult : JobSubmissionResult
     {
-        internal YarnJobSubmissionResult(IREEFClient reefClient, string filePath) 
-            : base(reefClient, filePath)
+        internal YarnJobSubmissionResult(IREEFClient reefClient, 
+            string filePath,
+            int numberOfRetries,
+            int retryInterval) 
+            : base(reefClient, filePath, numberOfRetries, retryInterval)
         {
         }
 

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
index 058f565..c2c2b87 100644
--- a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
@@ -18,8 +18,14 @@
  */
 package org.apache.reef.bridge.client;
 
+import org.apache.reef.bridge.client.Parameters.HTTPStatusAlarmInterval;
+import org.apache.reef.bridge.client.Parameters.HTTPStatusNumberOfRetries;
 import org.apache.reef.proto.ReefServiceProtos;
 import org.apache.reef.runtime.common.driver.client.JobStatusHandler;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.Clock;
+import org.apache.reef.wake.time.event.Alarm;
 import org.apache.reef.webserver.HttpHandler;
 import org.apache.reef.webserver.ParsedHttpRequest;
 
@@ -52,8 +58,49 @@ final class DriverStatusHTTPHandler implements HttpHandler, JobStatusHandler {
    */
   private ReefServiceProtos.JobStatusProto lastStatus = null;
 
+  /**
+   * The clock is used to schedule a check whether the handler has been called.
+   */
+  private final Clock clock;
+
+  /**
+   * The maximum number of times the AlarmHandler will be scheduled.
+   */
+  private final int maxNumberOfRetries;
+
+  /**
+   * The interval between alarms.
+   */
+  private final int alarmInterval;
+
+  /**
+   * The current retry.
+   */
+  private int retry = 0;
+
+  /**
+   * The alarm handler to keep the Clock alive until the status has been requested once.
+   */
+  private final EventHandler<Alarm> alarmHandler = new EventHandler<Alarm>() {
+    @Override
+    public void onNext(final Alarm value) {
+      scheduleAlarm();
+    }
+  };
+
+  /**
+   * Whether or not this handler was called at least once via HTTP.
+   */
+  private boolean wasCalledViaHTTP = false;
+
   @Inject
-  DriverStatusHTTPHandler(){
+  DriverStatusHTTPHandler(final Clock clock,
+                          @Parameter(HTTPStatusNumberOfRetries.class) final int maxNumberOfRetries,
+                          @Parameter(HTTPStatusAlarmInterval.class) final int alarmInterval) {
+    this.clock = clock;
+    this.maxNumberOfRetries = maxNumberOfRetries;
+    this.alarmInterval = alarmInterval;
+    scheduleAlarm();
   }
 
   @Override
@@ -71,6 +118,7 @@ final class DriverStatusHTTPHandler implements HttpHandler, JobStatusHandler {
       throws IOException, ServletException {
     try (final PrintWriter writer = response.getWriter()) {
       writer.write(waitAndGetMessage());
+      this.wasCalledViaHTTP = true;
     }
   }
 
@@ -126,4 +174,24 @@ final class DriverStatusHTTPHandler implements HttpHandler, JobStatusHandler {
   static String getMessageForStatus(final ReefServiceProtos.JobStatusProto status) {
     return status.getState().name();
   }
+
+  /**
+   * Schedules an alarm, if needed.
+   * <p>
+   * The alarm will prevent the Clock from going idle. This gives the .NET Client time to make a call to this HTTP
+   * handler.
+   */
+  private void scheduleAlarm() {
+    if (wasCalledViaHTTP || retry >= maxNumberOfRetries) {
+      // No alarm necessary anymore.
+      LOG.log(Level.INFO,
+          "Not scheduling additional alarms after {0} out of max {1} retries. The HTTP handles was called: ",
+          new Object[] {retry, maxNumberOfRetries, wasCalledViaHTTP});
+      return;
+    }
+
+    // Scheduling an alarm will prevent the clock from going idle.
+    ++retry;
+    clock.scheduleAlarm(alarmInterval, alarmHandler);
+  }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusAlarmInterval.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusAlarmInterval.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusAlarmInterval.java
new file mode 100644
index 0000000..9f600b6
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusAlarmInterval.java
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+package org.apache.reef.bridge.client.Parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * The interval between alarms in DriverStatusHTTPHandler.
+ */
+@NamedParameter(default_value = "200", doc = "The interval between alarms in DriverStatusHTTPHandler.")
+public final class HTTPStatusAlarmInterval implements Name<Integer> {
+
+  private HTTPStatusAlarmInterval() {
+    //intentionally empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusNumberOfRetries.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusNumberOfRetries.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusNumberOfRetries.java
new file mode 100644
index 0000000..6f43cb6
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusNumberOfRetries.java
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+package org.apache.reef.bridge.client.Parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Number of times the HTTPStatusHandler will advance its alarm.
+ */
+@NamedParameter(default_value = "10", doc = "Number of times the HTTPStatusHandler will advance its alarm.")
+public final class HTTPStatusNumberOfRetries implements Name<Integer> {
+
+  private HTTPStatusNumberOfRetries() {
+    // Intentionally empty.
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/package-info.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/package-info.java
new file mode 100644
index 0000000..48e9af2
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * Named Parameters of the Java side of the .NET Client.
+ */
+package org.apache.reef.bridge.client.Parameters;

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java b/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
index 222832a..8b1b126 100644
--- a/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
+++ b/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
@@ -19,6 +19,8 @@
 package org.apache.reef.bridge.client;
 
 import org.apache.reef.proto.ReefServiceProtos;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -71,8 +73,8 @@ public final class TestDriverStatusHTTPHandler {
    * {@link org.apache.reef.runtime.common.driver.client.JobStatusHandler}.
    */
   @Test
-  public void testLastStatus() {
-    final DriverStatusHTTPHandler tester = new DriverStatusHTTPHandler();
+  public void testLastStatus() throws InjectionException {
+    final DriverStatusHTTPHandler tester = getInstance();
 
     for (final ReefServiceProtos.JobStatusProto status : allStatuses) {
       tester.onNext(status);
@@ -84,8 +86,8 @@ public final class TestDriverStatusHTTPHandler {
    * Test the wait and notify for correctness.
    */
   @Test
-  public void testAsyncCalls() throws InterruptedException {
-    final DriverStatusHTTPHandler tester = new DriverStatusHTTPHandler();
+  public void testAsyncCalls() throws InterruptedException, InjectionException {
+    final DriverStatusHTTPHandler tester = getInstance();
 
     final WaitingRunnable waiter = new WaitingRunnable(tester);
 
@@ -100,6 +102,10 @@ public final class TestDriverStatusHTTPHandler {
     }
   }
 
+  private static DriverStatusHTTPHandler getInstance() throws InjectionException {
+    return Tang.Factory.getTang().newInjector().getInstance(DriverStatusHTTPHandler.class);
+  }
+
   private final class WaitingRunnable implements Runnable {
     private final DriverStatusHTTPHandler handler;
     private String result = null;