You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by sh...@apache.org on 2017/08/10 01:24:20 UTC

reef git commit: [REEF-1823] Support YARN API GetApplications() in .Net YarnREEFClient

Repository: reef
Updated Branches:
  refs/heads/master 721f8ca48 -> 47593a770


[REEF-1823] Support YARN API GetApplications() in .Net YarnREEFClient

This addressed the issue by
  * Exposing GetApplications in IYarnRMClient
  * Adding a IYarnREEFClient for Yarn specific functions
  * Implementing the new functionality in IYarnREEFClient
  * Updating YARNClientConfiguration to bind IYarnREEFClient to YarnREEFClient
  * Continuing to bind IREEFClient also to YarnREEFClient for backward compatibility
  * Adding test cases
  * Updating HelloREEFYarn to use the new API

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

Pull request:
  This closes #1361


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

Branch: refs/heads/master
Commit: 47593a77059e082b1e094af3b6811473cb5ac628
Parents: 721f8ca
Author: Julia Wang <ju...@apache.org>
Authored: Tue Aug 8 23:17:59 2017 -0700
Committer: Shravan Narayanamurthy <sh...@apache.org>
Committed: Wed Aug 9 18:23:03 2017 -0700

----------------------------------------------------------------------
 .../YarnClientTests.cs                          |  63 ++++++++++++
 .../Org.Apache.REEF.Client.csproj               |   3 +
 .../YARN/ApplicationReport.cs                   | 101 +++++++++++++++++++
 .../YARN/IApplicationReport.cs                  |  71 +++++++++++++
 .../YARN/IYarnREEFClient.cs                     |  37 +++++++
 .../YARN/RESTClient/DataModel/Application.cs    |  33 ++++++
 .../YARN/RESTClient/IYarnRMClient.cs            |  16 +++
 .../YARN/RESTClient/YarnClient.cs               |  19 ++++
 .../RESTClient/YarnClientNoCancellationToken.cs |  25 +++++
 .../YARN/YARNClientConfiguration.cs             |   1 +
 .../YARN/YARNREEFClient.cs                      |  47 ++++++++-
 .../YARN/YarnREEFDotNetClient.cs                |  13 ++-
 .../HelloREEFYarn.cs                            |  20 +++-
 13 files changed, 442 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs b/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs
index 3858b12..127ce52 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs
@@ -132,6 +132,69 @@ namespace Org.Apache.REEF.Client.Tests
         }
 
         [Fact]
+        public async Task TestGetApplications()
+        {
+            var ctx = new TestContext();
+            var urlProvider = ctx.UrlProviderFake;
+            var restReqExecutor = ctx.RestRequestExecutorFake;
+            var anyUri = Enumerable.Repeat(new Uri("anyscheme://anypath"), 1);
+
+            urlProvider.GetUrlAsync().Returns(Task.FromResult(anyUri));
+            var anyApplications = new Applications();
+
+            var anyApplication1 = new Application
+            {
+                AllocatedMB = 100,
+                AmHostHttpAddress = "http://anyhttpaddress",
+                AmContainerLogs = "SomeLogs",
+                ApplicationType = "AnyYarnApplicationType",
+                State = State.FINISHED,
+                Name = "AnyApplicationName",
+                RunningContainers = 0
+            };
+
+            var anyApplication2 = new Application
+            {
+                AllocatedMB = 100,
+                AmHostHttpAddress = "http://anyhttpaddress",
+                AmContainerLogs = "SomeLogs",
+                ApplicationType = "AnyYarnApplicationType",
+                State = State.FINISHED,
+                Name = "AnyApplicationName",
+                RunningContainers = 0
+            };
+
+            anyApplications.App = new List<Application> { anyApplication1, anyApplication2 };
+
+            restReqExecutor.ExecuteAsync<Applications>(
+                Arg.Is<RestRequest>(
+                    req =>
+                        req.Resource == "ws/v1/cluster/apps"
+                        && req.RootElement == Applications.RootElement
+                        && req.Method == Method.GET),
+                anyUri.First(),
+                CancellationToken.None).Returns(Task.FromResult(anyApplications));
+
+            var yarnClient = ctx.GetClient();
+            Applications actualApplications = await yarnClient.GetApplicationsAsync();
+
+            Assert.NotEqual(actualApplications.App, null);
+            Assert.Equal(actualApplications.App.Count, 2);
+
+            int matchCount = 0;
+            foreach (var anyApplication in actualApplications.App)
+            {
+                if (anyApplication == anyApplication1 || anyApplication == anyApplication2)
+                {
+                    ++matchCount;
+                }
+            }
+
+            Assert.Equal(actualApplications.App.Count, matchCount);
+            var unused = urlProvider.Received(1).GetUrlAsync();
+        }
+
+        [Fact]
         public async Task TestGetApplicationFinalStatus()
         {
             var ctx = new TestContext();

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/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 c8df560..2c3df86 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
@@ -99,18 +99,21 @@ under the License.
     <Compile Include="Local\Parameters\LocalRuntimeDirectory.cs" />
     <Compile Include="Local\Parameters\NumberOfEvaluators.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="YARN\ApplicationReport.cs" />
     <Compile Include="YARN\HDI\HDInsightClientConfiguration.cs" />
     <Compile Include="YARN\HDI\HDInsightCommandLineEnvironment.cs" />
     <Compile Include="YARN\HDI\HDInsightCredential.cs" />
     <Compile Include="YARN\HDI\HDInsightResourceFileRemoteUrlToClusterUrlConverter.cs" />
     <Compile Include="YARN\HDI\HDInsightUrlProvider.cs" />
     <Compile Include="YARN\HDI\NamedParameters.cs" />
+    <Compile Include="YARN\IApplicationReport.cs" />
     <Compile Include="YARN\IdentityResourceFileRemoteUrlToClusterUrlConverter.cs" />
     <Compile Include="YARN\IResourceFileRemoteUrlToClusterUrlConverter.cs" />
     <Compile Include="YARN\IYarnCommandLineEnvironment.cs" />
     <Compile Include="YARN\IYarnJobCommandProvider.cs" />
     <Compile Include="YARN\IJobResourceUploader.cs" />
     <Compile Include="YARN\IJobSubmissionDirectoryProvider.cs" />
+    <Compile Include="YARN\IYarnREEFClient.cs" />
     <Compile Include="YARN\Parameters\DriverMaxMemoryAllicationPoolSizeMB.cs" />
     <Compile Include="YARN\Parameters\DriverMaxPermSizeMB.cs" />
     <Compile Include="YARN\Parameters\DriverStderrFilePath.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/ApplicationReport.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/ApplicationReport.cs b/lang/cs/Org.Apache.REEF.Client/YARN/ApplicationReport.cs
new file mode 100644
index 0000000..6cbd89c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/ApplicationReport.cs
@@ -0,0 +1,101 @@
+// 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.Client.YARN.RestClient.DataModel;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Client.YARN
+{
+    /// <summary>
+    /// This class represents application information maintained by YARN RM.
+    /// This class is modeled on Org.Apache.REEF.Client.YARN.RestClient.DataModel.Application. 
+    /// Documentation on the class Org.Apache.REEF.Client.YARN.RestClient.DataModel.Application
+    /// can be found here.
+    /// <a href="http://hadoop.apache.org/docs/r2.6.0/hadoop-yarn/hadoop-yarn-site/WebServicesIntro.html">
+    /// Hadoop RM REST API</a> documentation.
+    /// </summary>
+    internal sealed class ApplicationReport : IApplicationReport
+    {
+        /// <summary>
+        /// Application name.
+        /// </summary>
+        public string AppName { get; private set; }
+
+        /// <summary>
+        /// Application start time.
+        /// </summary>
+        public long StartedTime { get; private set; }
+
+        /// <summary>
+        /// Application finished time.
+        /// </summary>
+        public long FinishedTime { get; private set; }
+
+        /// <summary>
+        /// Number of running evaluators
+        /// </summary>
+        public int NumberOfRunningEvaluators { get; private set; }
+
+        /// <summary>
+        /// Driver url
+        /// </summary>
+        public Uri TrackingUrl { get; private set; }
+
+        /// <summary>
+        /// Application id
+        /// </summary>
+        public string AppId { get; private set; }
+
+        /// <summary>
+        /// Application final state
+        /// </summary>
+        public FinalState FinalState { get; private set; }
+
+        /// <summary>
+        /// Application report constructor
+        /// </summary>
+        internal ApplicationReport(string appId, string appName, string trackingUrl, 
+            long startedTime, long finishedTime, int numberOfRunningEvaluators, FinalState finalState)
+        {
+            AppId = appId;
+            AppName = appName;
+            TrackingUrl = trackingUrl == null ? null : new Uri(trackingUrl);
+            FinalState = finalState;
+            StartedTime = startedTime;
+            FinishedTime = finishedTime;
+            NumberOfRunningEvaluators = numberOfRunningEvaluators;
+        }
+
+        /// <summary>
+        /// To string for printing/log
+        /// </summary>
+        /// <returns></returns>
+        public override string ToString()
+        {
+            return string.Format("AppName: {0} StartedTime: {1}, FinishedTime: {2}," +
+                                    "NumberOfRunningEvaluators: {3}, TrackingUrl: {4}, AppId: {5}, FinalState: {6}",
+                AppName,
+                StartedTime,
+                FinishedTime,
+                NumberOfRunningEvaluators,
+                TrackingUrl,
+                AppId,
+                FinalState);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/IApplicationReport.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/IApplicationReport.cs b/lang/cs/Org.Apache.REEF.Client/YARN/IApplicationReport.cs
new file mode 100644
index 0000000..66bfe14
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/IApplicationReport.cs
@@ -0,0 +1,71 @@
+// 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.Client.YARN.RestClient.DataModel;
+using Org.Apache.REEF.Utilities.Attributes;
+
+namespace Org.Apache.REEF.Client.YARN
+{
+    /// <summary>
+    /// This interface represents application information maintained by YARN RM.
+    /// This interface is modeled on Org.Apache.REEF.Client.YARN.RestClient.DataModel.Application. 
+    /// Documentation on the class Org.Apache.REEF.Client.YARN.RestClient.DataModel.Application
+    /// can be found here.
+    /// <a href="http://hadoop.apache.org/docs/r2.6.0/hadoop-yarn/hadoop-yarn-site/WebServicesIntro.html">
+    /// Hadoop RM REST API</a> documentation.
+    /// </summary>
+    [Unstable("0.17", "Working in progress. For local runtime, some of the property, such as FinalState and AppId are not implemented yet.")]
+    public interface IApplicationReport
+    {
+        /// <summary>
+        /// Get YARN application name.
+        /// </summary>
+        string AppName { get; }
+        
+        /// <summary>
+        /// Get YARN application start time.
+        /// </summary>
+        long StartedTime { get; }
+
+        /// <summary>
+        /// Get YARN application finish time.
+        /// </summary>
+        long FinishedTime { get; }
+
+        /// <summary>
+        /// Get YARN application number of running containers.
+        /// </summary>
+        int NumberOfRunningEvaluators { get; }
+
+        /// <summary>
+        /// This method returns the url of http server running inside the driver.
+        /// e.g. http://hostname:port/
+        /// </summary>
+        Uri TrackingUrl { get; }
+
+        /// <summary>
+        /// Get YARN application id.
+        /// </summary>
+        string AppId { get; }
+
+        /// <summary>
+        /// Get Application final state.
+        /// </summary>
+        FinalState FinalState { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/IYarnREEFClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/IYarnREEFClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/IYarnREEFClient.cs
new file mode 100644
index 0000000..e208277
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/IYarnREEFClient.cs
@@ -0,0 +1,37 @@
+// 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.Collections.Generic;
+using System.Threading.Tasks;
+using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Utilities.Attributes;
+
+namespace Org.Apache.REEF.Client.YARN
+{
+    /// <summary>
+    /// Interface that defines Yarn client API
+    /// </summary>
+    public interface IYarnREEFClient : IREEFClient
+    {
+        /// <summary>
+        /// Returns all the application reports running in the cluster
+        /// </summary>
+        /// <returns></returns>
+        [Unstable("0.17", "Working in progress for rest API id returned")]
+        Task<IReadOnlyDictionary<string, IApplicationReport>> GetApplicationReports();
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Application.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Application.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Application.cs
index db7b0c7..ff09140 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Application.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Application.cs
@@ -94,5 +94,38 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 
         [JsonProperty("vcoreSeconds")]
         public long VcoreSeconds { get; set; }
+
+        public override string ToString()
+        {
+            return string.Format("Id: {0}, User: {1}, Name: {2}, ApplicationType: {3}, " +
+                                 "Queue: {4}, State: {5}, FinalStatus: {6}, Progress: {7}, " +
+                                 "ClusterId: {8}, StartedTime: {9}, FinishedTime: {10}, " +
+                                 "ElapsedTime: {11}, AmContainerLogs: {12}, AmHostHttpAddress: {13}, " +
+                                 "AllocatedMB: {14}, AllocatedVCores: {15}, RunningContainers: {16}, " +
+                                 "MemorySeconds: {17}, VcoreSeconds: {18}, TrackingUI: {19}, " +
+                                 "TrackingUrl: {20}, Diagnostics: {21} ",
+                Id,
+                User,
+                Name,
+                ApplicationType,
+                Queue,
+                State,
+                FinalStatus,
+                Progress,
+                ClusterId,
+                StartedTime,
+                FinishedTime,
+                ElapsedTime,
+                AmContainerLogs,
+                AmHostHttpAddress,
+                AllocatedMB,
+                AllocatedVCores,
+                RunningContainers,
+                MemorySeconds,
+                VcoreSeconds,
+                TrackingUI,
+                TrackingUrl,
+                Diagnostics);
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IYarnRMClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IYarnRMClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IYarnRMClient.cs
index 6a69374..ee63b60 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IYarnRMClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IYarnRMClient.cs
@@ -37,6 +37,22 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
 
         Task<Application> GetApplicationAsync(string appId, CancellationToken cancellationToken);
 
+        /// <summary>
+        /// This API returns information about all the applications maintained
+        /// by YARN RM in the cluster by invoking REST API.
+        /// </summary>
+        /// <returns></returns>
+        Task<Applications> GetApplicationsAsync();
+
+        /// <summary>
+        /// This API returns information about all the applications maintained
+        /// by YARN RM in the cluster by invoking REST API. This API also allow cooperative
+        /// cancellation in multi-threading scenarios.
+        /// </summary>
+        /// <param name="cancellationToken"></param>
+        /// <returns></returns>
+        Task<Applications> GetApplicationsAsync(CancellationToken cancellationToken);
+
         Task<NewApplication> CreateNewApplicationAsync();
 
         Task<NewApplication> CreateNewApplicationAsync(CancellationToken cancellationToken);

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClient.cs
index ec47d8b..6a29c49 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClient.cs
@@ -91,6 +91,25 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
                 await GenerateUrlAndExecuteRequestAsync<Application>(request, cancellationToken);
         }
 
+        /// <summary>
+        /// This API returns information about all the applications maintained
+        /// by YARN RM in the cluster by invoking REST API. This API also allow cooperative
+        /// cancellation in multi-threading scenarios.
+        /// </summary>
+        /// <param name="cancellationToken">cancellation token</param>
+        /// <returns>list of applications</returns>
+        public async Task<Applications> GetApplicationsAsync(CancellationToken cancellationToken)
+        {
+            await new RemoveSynchronizationContextAwaiter();
+
+            var request = _requestFactory.CreateRestRequest(Applications.Resource,
+                Method.GET,
+                Applications.RootElement);
+
+            return
+                await GenerateUrlAndExecuteRequestAsync<Applications>(request, cancellationToken);
+        }
+
         public async Task<NewApplication> CreateNewApplicationAsync(CancellationToken cancellationToken)
         {
             await new RemoveSynchronizationContextAwaiter();

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClientNoCancellationToken.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClientNoCancellationToken.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClientNoCancellationToken.cs
index 707a3a0..2d490f8 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClientNoCancellationToken.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClientNoCancellationToken.cs
@@ -24,30 +24,55 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
 {
     internal sealed partial class YarnClient
     {
+        /// <summary>
+        /// Get Cluster Info without cancellation token.
+        /// </summary>
         public async Task<ClusterInfo> GetClusterInfoAsync()
         {
             await new RemoveSynchronizationContextAwaiter();
             return await GetClusterInfoAsync(CancellationToken.None);
         }
 
+        /// <summary>
+        /// Get Cluster Metrics without cancellation token.
+        /// </summary>
         public async Task<ClusterMetrics> GetClusterMetricsAsync()
         {
             await new RemoveSynchronizationContextAwaiter();
             return await GetClusterMetricsAsync(CancellationToken.None);
         }
 
+        /// <summary>
+        /// Get Application without cancellation token.
+        /// </summary>
         public async Task<Application> GetApplicationAsync(string appId)
         {
             await new RemoveSynchronizationContextAwaiter();
             return await GetApplicationAsync(appId, CancellationToken.None);
         }
 
+        /// <summary>
+        /// Get Applications without cancellation token.
+        /// </summary>
+        public async Task<Applications> GetApplicationsAsync()
+        {
+            await new RemoveSynchronizationContextAwaiter();
+            return await GetApplicationsAsync(CancellationToken.None);
+        }
+
+        /// <summary>
+        /// Create a new Application without cancellation token.
+        /// </summary>
         public async Task<NewApplication> CreateNewApplicationAsync()
         {
             await new RemoveSynchronizationContextAwaiter();
             return await CreateNewApplicationAsync(CancellationToken.None);
         }
 
+        /// <summary>
+        /// Submit Application without cancellation token.
+        /// </summary>
+        /// <param name="submitApplicationRequest"></param>
         public async Task<Application> SubmitApplicationAsync(SubmitApplication submitApplicationRequest)
         {
             await new RemoveSynchronizationContextAwaiter();

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
index 20127c6..69d715e 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
@@ -43,6 +43,7 @@ namespace Org.Apache.REEF.Client.Yarn
         public static readonly OptionalParameter<string> FileSystemUrl = new OptionalParameter<string>();
 
         public static ConfigurationModule ConfigurationModule = new YARNClientConfiguration()
+            .BindImplementation(GenericType<IYarnREEFClient>.Class, GenericType<YarnREEFClient>.Class)
             .BindImplementation(GenericType<IREEFClient>.Class, GenericType<YarnREEFClient>.Class)
             .BindImplementation(GenericType<IYarnRestClientCredential>.Class, YarnRestClientCredential)
             .BindNamedParameter(GenericType<JobSubmissionDirectoryPrefixParameter>.Class, JobSubmissionFolderPrefix)

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/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 20c8389..c00be21 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
@@ -16,6 +16,8 @@
 // under the License.
 
 using System;
+using System.Collections.Generic;
+using System.Collections.ObjectModel;
 using System.Globalization;
 using System.IO;
 using System.Linq;
@@ -32,7 +34,7 @@ using Org.Apache.REEF.Utilities.Logging;
 
 namespace Org.Apache.REEF.Client.Yarn
 {
-    internal sealed class YarnREEFClient : IREEFClient
+    internal sealed class YarnREEFClient : IYarnREEFClient
     {
         /// <summary>
         /// The class name that contains the Java counterpart for this client.
@@ -98,13 +100,50 @@ namespace Org.Apache.REEF.Client.Yarn
         {
             var application = await _yarnClient.GetApplicationAsync(appId);
 
-            var msg = string.Format("application status {0}, Progress: {1}, trackingUri: {2}, Name: {3}, ApplicationId: {4}, State {5}.",
-                application.FinalStatus, application.Progress, application.TrackingUI, application.Name, application.Id, application.State);
-            Logger.Log(Level.Verbose, msg);
+            Logger.Log(Level.Verbose,
+               "application status {0}, Progress: {1}, uri: {2}, Name: {3}, ApplicationId: {4}, State {5}.",
+               application.FinalStatus,
+               application.Progress,
+               application.TrackingUI,
+               application.Name,
+               application.Id,
+               application.State);
 
             return application.FinalStatus;
         }
 
+        /// <summary>
+        /// Returns all the application reports running in the cluster.
+        /// GetApplicationReports call is very expensive as it is trying 
+        /// fetch information about all the applications in the cluster.
+        /// 
+        /// If this method is called right after submitting a new app then
+        /// that new app might not immediately result in this list until 
+        /// some number of retries. 
+        /// </summary>
+        /// <returns></returns>
+        public async Task<IReadOnlyDictionary<string, IApplicationReport>> GetApplicationReports()
+        {
+            var appReports = new Dictionary<string, IApplicationReport>();
+            var applications = await _yarnClient.GetApplicationsAsync();
+
+            foreach (var application in applications.App)
+            {
+                appReports.Add(application.Id, new ApplicationReport(application.Id,
+                    application.Name,
+                    application.TrackingUrl,
+                    application.StartedTime,
+                    application.FinishedTime,
+                    application.RunningContainers,
+                    application.FinalStatus));
+
+                Logger.Log(Level.Verbose,
+                    "Application report {0}: {1}",
+                    application.Id, application);
+            }
+            return new ReadOnlyDictionary<string, IApplicationReport>(appReports);
+        }
+
         private void Launch(JobRequest jobRequest, string driverFolderPath)
         {
             _driverFolderPreparationHelper.PrepareDriverFolder(jobRequest.AppParameters, driverFolderPath);

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs
index 18f8fbf..fcae2e4 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs
@@ -40,7 +40,7 @@ namespace Org.Apache.REEF.Client.YARN
     /// deprecated by final client.
     /// </summary>
     [Unstable("For security token support we still need to use YARNREEFClient until (REEF-875)")]
-    public sealed class YarnREEFDotNetClient : IREEFClient
+    public sealed class YarnREEFDotNetClient : IYarnREEFClient
     {
         private const string REEFApplicationType = @"REEF";
         private static readonly Logger Log = Logger.GetLogger(typeof(YarnREEFDotNetClient));
@@ -146,6 +146,17 @@ namespace Org.Apache.REEF.Client.YARN
             return application.FinalStatus;
         }
 
+        /// <summary>
+        /// Returns all the application reports running in the cluster
+        /// </summary>
+        /// <returns></returns>
+        /// TODO: [REEF-1825]: Implement GetApplicationReports to return the status of the applications.
+        public async Task<IReadOnlyDictionary<string, IApplicationReport>> GetApplicationReports()
+        {
+            await Task.Delay(0);
+            throw new NotImplementedException();
+        }
+
         private SubmitApplication CreateApplicationSubmissionRequest(
            JobParameters jobParameters,
            string appId,

http://git-wip-us.apache.org/repos/asf/reef/blob/47593a77/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
index fbccdec..6044569 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
@@ -22,6 +22,7 @@ using System.Threading;
 using Org.Apache.REEF.Client.API;
 using Org.Apache.REEF.Client.Common;
 using Org.Apache.REEF.Client.Yarn;
+using Org.Apache.REEF.Client.YARN;
 using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
 using Org.Apache.REEF.Driver;
 using Org.Apache.REEF.Tang.Annotations;
@@ -46,7 +47,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         private const string SecurityTokenId = "SecurityTokenId";
         private const string SecurityTokenPwd = "SecurityTokenPwd";
 
-        private readonly IREEFClient _reefClient;
+        private readonly IYarnREEFClient _reefClient;
         private readonly JobRequestBuilder _jobRequestBuilder;
 
         private static readonly Logger Logger = Logger.GetLogger(typeof(HelloREEFYarn));
@@ -57,7 +58,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         private readonly IList<string> _nodeNames;
 
         [Inject]
-        private HelloREEFYarn(IREEFClient reefClient, 
+        private HelloREEFYarn(IYarnREEFClient reefClient, 
             JobRequestBuilder jobRequestBuilder,
             [Parameter(typeof(NodeNames))] ISet<string> nodeNames)
         {
@@ -94,11 +95,26 @@ namespace Org.Apache.REEF.Examples.HelloREEF
                 .Build();
 
             var result = _reefClient.SubmitAndGetJobStatus(helloJobRequest);
+
+            LogApplicationReport();
             var state = PullFinalJobStatus(result);
             Logger.Log(Level.Info, "Application state : {0}.", state);
         }
 
         /// <summary>
+        /// Get application report and log.
+        /// </summary>
+        private void LogApplicationReport()
+        {
+            Logger.Log(Level.Info, "Getting Application report...");
+            var apps = _reefClient.GetApplicationReports().Result;
+            foreach (var r in apps)
+            {
+                Logger.Log(Level.Info, "Application report -- AppId {0}: {1}.", r.Key, r.Value.ToString());
+            }
+        }
+
+        /// <summary>
         /// This is to pull job final status until the Job is done
         /// </summary>
         /// <param name="jobSubmitionResult"></param>