You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by af...@apache.org on 2015/11/13 23:30:37 UTC

incubator-reef git commit: [REEF-937] Fix/Clean up YARN .NET REST client

Repository: incubator-reef
Updated Branches:
  refs/heads/master d0d9bbc7e -> 9fedfbc09


[REEF-937] Fix/Clean up YARN .NET REST client

This addressed the issue by:
 * Fix casing of property decoration in data models as YARNRM sometimes
 * silently fails on case mismatch
 * Use same library for both JSON serialization and deserialization
 * Cleanup some strings which were enums
 * Fix unittests

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

Pull Request:
  Closes #630


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

Branch: refs/heads/master
Commit: 9fedfbc09f9ee287fc9439c1feef00002e741eca
Parents: d0d9bbc
Author: Anupam <an...@gmail.com>
Authored: Wed Nov 11 15:52:45 2015 -0800
Committer: Andrew Chung <af...@gmail.com>
Committed: Fri Nov 13 14:30:00 2015 -0800

----------------------------------------------------------------------
 .../WindowsHadoopEmulatorYarnClientTests.cs     | 22 ++++---
 .../YarnClientTests.cs                          | 46 +++++++-------
 .../Org.Apache.REEF.Client.csproj               |  4 ++
 .../YARN/RESTClient/DataModel/Acls.cs           |  4 +-
 .../RESTClient/DataModel/AmContainerSpec.cs     |  3 +
 .../YARN/RESTClient/DataModel/Application.cs    | 24 ++++++++
 .../YARN/RESTClient/DataModel/ApplicationTag.cs |  3 +
 .../YARN/RESTClient/DataModel/ClusterHaState.cs | 35 +++++++++++
 .../YARN/RESTClient/DataModel/ClusterInfo.cs    | 17 ++++-
 .../YARN/RESTClient/DataModel/ClusterMetrics.cs | 25 ++++++++
 .../YARN/RESTClient/DataModel/ClusterState.cs   | 35 +++++++++++
 .../YARN/RESTClient/DataModel/Commands.cs       |  3 +
 .../YARN/RESTClient/DataModel/Credentials.cs    |  4 ++
 .../YARN/RESTClient/DataModel/KeyValuePair.cs   | 35 +++++++++++
 .../YARN/RESTClient/DataModel/LocalResources.cs | 10 ++-
 .../YARN/RESTClient/DataModel/NewApplication.cs |  1 -
 .../YARN/RESTClient/DataModel/Resouce.cs        |  3 +-
 .../YARN/RESTClient/DataModel/ServiceData.cs    |  3 +-
 .../RESTClient/DataModel/SubmitApplication.cs   |  4 +-
 .../YARN/RESTClient/IRestClientFactory.cs       |  5 +-
 .../YARN/RESTClient/RestJsonDeserializer.cs     | 65 ++++++++++++++++++++
 .../YARN/RESTClient/RestJsonSerializer.cs       |  5 +-
 .../YARN/RESTClient/RestRequestExecutor.cs      |  3 +
 23 files changed, 311 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client.Tests/WindowsHadoopEmulatorYarnClientTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/WindowsHadoopEmulatorYarnClientTests.cs b/lang/cs/Org.Apache.REEF.Client.Tests/WindowsHadoopEmulatorYarnClientTests.cs
index d802a16..31b7382 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/WindowsHadoopEmulatorYarnClientTests.cs
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/WindowsHadoopEmulatorYarnClientTests.cs
@@ -75,8 +75,7 @@ namespace Org.Apache.REEF.Client.Tests
             var clusterInfo = await client.GetClusterInfoAsync();
 
             Assert.IsNotNull(clusterInfo);
-            Assert.AreEqual("STARTED", clusterInfo.State);
-            Assert.IsFalse(string.IsNullOrEmpty(clusterInfo.HaState));
+            Assert.AreEqual(ClusterState.STARTED, clusterInfo.State);
             Assert.IsTrue(clusterInfo.StartedOn > 0);
         }
 
@@ -132,16 +131,18 @@ namespace Org.Apache.REEF.Client.Tests
                     },
                     LocalResources = new LocalResources
                     {
-                        Entry = new List<KeyValuePair<string, LocalResourcesValue>>
+                        Entries = new List<YARN.RestClient.DataModel.KeyValuePair<string, LocalResourcesValue>>
                         {
-                            new KeyValuePair<string, LocalResourcesValue>(
-                                "APPLICATIONWILLFAILBUTWEDONTCAREHERE",
-                                new LocalResourcesValue
+                            new YARN.RestClient.DataModel.KeyValuePair<string, LocalResourcesValue>
+                            {
+                                Key = "APPLICATIONWILLFAILBUTWEDONTCAREHERE",
+                                Value = new LocalResourcesValue
                                 {
                                     Resource = "Foo",
                                     Type = ResourceType.FILE,
                                     Visibility = Visibility.APPLICATION
-                                })
+                                }
+                            }
                         }
                     }
                 }
@@ -166,17 +167,18 @@ namespace Org.Apache.REEF.Client.Tests
         [TestCategory("Functional")]
         public async Task TestErrorResponse()
         {
-            const string WrongApplicationName = @"Something";
+            const string wrongApplicationName = @"Something";
 
             var client = TangFactory.GetTang().NewInjector().GetInstance<IYarnRMClient>();
 
             try
             {
-                await client.GetApplicationAsync(WrongApplicationName);
+                await client.GetApplicationAsync(wrongApplicationName);
                 Assert.Fail("Should throw YarnRestAPIException");
             }
-            catch (YarnRestAPIException)
+            catch (AggregateException aggregateException)
             {
+                Assert.IsInstanceOfType(aggregateException.GetBaseException(), typeof(YarnRestAPIException));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/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 a5dc5f5..0af520b 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs
@@ -231,16 +231,18 @@ namespace Org.Apache.REEF.Client.Tests
                     },
                     LocalResources = new LocalResources
                     {
-                        Entry = new List<KeyValuePair<string, LocalResourcesValue>>
+                        Entries = new List<YARN.RestClient.DataModel.KeyValuePair<string, LocalResourcesValue>>
                         {
-                            new KeyValuePair<string, LocalResourcesValue>(
-                                "APPLICATIONWILLFAILBUTWEDONTCAREHERE",
-                                new LocalResourcesValue
+                            new YARN.RestClient.DataModel.KeyValuePair<string, LocalResourcesValue>
+                            {
+                                Key = "APPLICATIONWILLFAILBUTWEDONTCAREHERE",
+                                Value = new LocalResourcesValue
                                 {
                                     Resource = "Foo",
                                     Type = ResourceType.FILE,
                                     Visibility = Visibility.APPLICATION
-                                })
+                                }
+                            }
                         }
                     }
                 }
@@ -249,40 +251,40 @@ namespace Org.Apache.REEF.Client.Tests
             const string expectedJson = @"{" +
                                             @"""application-id"":""AnyApplicationId""," +
                                             @"""application-name"":""AnyAPP""," +
-                                            @"""Queue"":null,""Priority"":1," +
+                                            @"""queue"":null,""priority"":1," +
                                             @"""am-container-spec"":" +
                                             @"{" +
                                                 @"""local-resources"":" +
                                                 @"{" +
-                                                    @"""Entry"":" +
+                                                    @"""entry"":" +
                                                     @"[" +
                                                         @"{" +
-                                                            @"""Key"":""APPLICATIONWILLFAILBUTWEDONTCAREHERE""," +
-                                                            @"""Value"":" +
+                                                            @"""key"":""APPLICATIONWILLFAILBUTWEDONTCAREHERE""," +
+                                                            @"""value"":" +
                                                             @"{" +
-                                                                @"""Resource"":""Foo""," +
-                                                                @"""Type"":1," +
-                                                                @"""Visibility"":2," +
-                                                                @"""Size"":0," +
-                                                                @"""Timestamp"":0" +
+                                                                @"""resource"":""Foo""," +
+                                                                @"""type"":""FILE""," +
+                                                                @"""visibility"":""APPLICATION""," +
+                                                                @"""size"":0," +
+                                                                @"""timestamp"":0" +
                                                             @"}" +
                                                         @"}" +
                                                     @"]" +
                                                 @"}," + 
-                                            @"""Environment"":null," +
-                                            @"""Commands"":" +
+                                            @"""environment"":null," +
+                                            @"""commands"":" +
                                             @"{" +
-                                                @"""Command"":""DONTCARE""" +
+                                                @"""command"":""DONTCARE""" +
                                             @"}," +
                                             @"""service-data"":null," +
-                                            @"""Credentials"":null," +
+                                            @"""credentials"":null," +
                                             @"""application-acls"":null}," +
-                                            @"""unmanaged-am"":false," +
+                                            @"""unmanaged-AM"":false," +
                                             @"""max-app-attempts"":1," +
                                             @"""resource"":" +
                                             @"{" +
                                                 @"""memory"":500," +
-                                                @"""VCores"":1" +
+                                                @"""vCores"":1" +
                                             @"},""application-type"":""REEFTest""," +
                                             @"""keep-containers-across-application-attempts"":false," +
                                             @"""application-tags"":null" +
@@ -317,7 +319,7 @@ namespace Org.Apache.REEF.Client.Tests
                         && req.Method == Method.POST
                         && req.JsonSerializer is RestJsonSerializer
                         && req.Parameters.First().Name == "application/json"
-                        && B(req, expectedJson)),
+                        && IsExpectedJson(req, expectedJson)),
                 anyUri.First(),
                 CancellationToken.None).Returns(Task.FromResult(response));
 
@@ -337,7 +339,7 @@ namespace Org.Apache.REEF.Client.Tests
             var unused = urlProvider.Received(2).GetUrlAsync();
         }
 
-        private static bool B(IRestRequest req, string expectedJson)
+        private static bool IsExpectedJson(IRestRequest req, string expectedJson)
         {
             return (string)req.Parameters.First().Value == expectedJson;
         }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/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 117017a..e456716 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
@@ -100,9 +100,12 @@ under the License.
     <Compile Include="YARN\RestClient\DataModel\AmContainerSpec.cs" />
     <Compile Include="YARN\RestClient\DataModel\ApplicationFinalState.cs" />
     <Compile Include="YARN\RestClient\DataModel\ApplicationTag.cs" />
+    <Compile Include="YARN\RestClient\DataModel\ClusterHaState.cs" />
+    <Compile Include="YARN\RestClient\DataModel\ClusterState.cs" />
     <Compile Include="YARN\RestClient\DataModel\Commands.cs" />
     <Compile Include="YARN\RestClient\DataModel\Credentials.cs" />
     <Compile Include="YARN\RestClient\DataModel\Environment.cs" />
+    <Compile Include="YARN\RestClient\DataModel\KeyValuePair.cs" />
     <Compile Include="YARN\RestClient\DataModel\LocalResources.cs" />
     <Compile Include="YARN\RestClient\DataModel\Secrets.cs" />
     <Compile Include="YARN\RestClient\DataModel\ServiceData.cs" />
@@ -111,6 +114,7 @@ under the License.
     <Compile Include="YARN\RestClient\IRestRequestExecutor.cs" />
     <Compile Include="YARN\RestClient\IUrlProvider.cs" />
     <Compile Include="YARN\RestClient\MultipleRMUrlProvider.cs" />
+    <Compile Include="YARN\RestClient\RestJsonDeserializer.cs" />
     <Compile Include="YARN\RestClient\RestJsonSerializer.cs" />
     <Compile Include="YARN\YarnJobSubmissionResult.cs" />
     <Compile Include="YARN\YARNREEFClient.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Acls.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Acls.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Acls.cs
index a6194d4..59ce123 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Acls.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Acls.cs
@@ -31,8 +31,8 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
     /// </summary>
     internal sealed class Acls
     {
-        [JsonProperty(ItemConverterType = typeof(StringEnumConverter))]
-        public IList<KeyValuePair<AclsType, string>> Entry { get; set; }
+        [JsonProperty(ItemConverterType = typeof(StringEnumConverter), PropertyName = "entry")]
+        public IList<KeyValuePair<AclsType, string>> Entries { get; set; }
     }
 
     /// <summary>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/AmContainerSpec.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/AmContainerSpec.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/AmContainerSpec.cs
index 074b823..dd75bc0 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/AmContainerSpec.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/AmContainerSpec.cs
@@ -29,13 +29,16 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
         [JsonProperty(PropertyName = "local-resources")]
         public LocalResources LocalResources { get; set; }
 
+        [JsonProperty(PropertyName = "environment")]
         public Environment Environment { get; set; }
 
+        [JsonProperty(PropertyName = "commands")]
         public Commands Commands { get; set; }
 
         [JsonProperty(PropertyName = "service-data")]
         public ServiceData ServiceData { get; set; }
 
+        [JsonProperty(PropertyName = "credentials")]
         public Credentials Credentials { get; set; }
 
         [JsonProperty(PropertyName = "application-acls")]

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/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 d80c551..876bd03 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
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using Newtonsoft.Json;
+
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {
     /// <summary>
@@ -27,48 +29,70 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
         internal static readonly string Resource = @"cluster/apps/";
         internal static readonly string RootElement = @"app";
 
+        [JsonProperty("id")]
         public string Id { get; set; }
 
+        [JsonProperty("user")]
         public string User { get; set; }
 
+        [JsonProperty("name")]
         public string Name { get; set; }
 
+        [JsonProperty("applicationType")]
         public string ApplicationType { get; set; }
 
+        [JsonProperty("queue")]
         public string Queue { get; set; }
 
+        [JsonProperty("state")]
         public State State { get; set; }
 
+        [JsonProperty("finalStatus")]
         public FinalState FinalStatus { get; set; }
 
+        [JsonProperty("progress")]
         public float Progress { get; set; }
 
+        [JsonProperty("trackingUI")]
         public string TrackingUI { get; set; }
 
+        [JsonProperty("trackingUrl")]
         public string TrackingUrl { get; set; }
 
+        [JsonProperty("diagnostics")]
         public string Diagnostics { get; set; }
 
+        [JsonProperty("clusterId")]
         public long ClusterId { get; set; }
 
+        [JsonProperty("startedTime")]
         public long StartedTime { get; set; }
 
+        [JsonProperty("finishedTime")]
         public long FinishedTime { get; set; }
 
+        [JsonProperty("elapsedTime")]
         public long ElapsedTime { get; set; }
 
+        [JsonProperty("amContainerLogs")]
         public string AmContainerLogs { get; set; }
 
+        [JsonProperty("amHostHttpAddress")]
         public string AmHostHttpAddress { get; set; }
 
+        [JsonProperty("allocatedMB")]
         public int AllocatedMB { get; set; }
 
+        [JsonProperty("allocatedVCores")]
         public int AllocatedVCores { get; set; }
 
+        [JsonProperty("runningContainers")]
         public int RunningContainers { get; set; }
 
+        [JsonProperty("memorySeconds")]
         public long MemorySeconds { get; set; }
 
+        [JsonProperty("vcoreSeconds")]
         public long VcoreSeconds { get; set; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ApplicationTag.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ApplicationTag.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ApplicationTag.cs
index c9a4e98..6a94036 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ApplicationTag.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ApplicationTag.cs
@@ -15,10 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using Newtonsoft.Json;
+
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {
     internal class ApplicationTag
     {
+        [JsonProperty(PropertyName = "tag")]
         public string Tag { get; set; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterHaState.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterHaState.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterHaState.cs
new file mode 100644
index 0000000..461db21
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterHaState.cs
@@ -0,0 +1,35 @@
+// 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.
+
+namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
+{
+    /// <summary>
+    /// Class generated based on schema provided in
+    /// <see cref="!:http://hadoop.apache.org/docs/r2.6.0/hadoop-yarn/hadoop-yarn-site/WebServicesIntro.html">
+    /// Hadoop RM REST API </see> documentation.
+    /// </summary>
+    internal enum ClusterHaState
+    {
+        INITIALIZING,
+
+        ACTIVE,
+
+        STANDBY,
+
+        STOPPED
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterInfo.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterInfo.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterInfo.cs
index c5c1780..e592a84 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterInfo.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterInfo.cs
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using Newtonsoft.Json;
+
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {
     /// <summary>
@@ -27,25 +29,34 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
         internal static readonly string Resource = @"cluster/info";
         internal static readonly string RootElement = @"clusterInfo";
 
+        [JsonProperty("id")]
         public long Id { get; set; }
 
+        [JsonProperty("startedOn")]
         public long StartedOn { get; set; }
 
-        public string State { get; set; }
+        [JsonProperty("state")]
+        public ClusterState State { get; set; }
 
-        public string HaState { get; set; }
+        [JsonProperty("haState")]
+        public ClusterHaState HaState { get; set; }
 
+        [JsonProperty("resourceManagerVersion")]
         public string ResourceManagerVersion { get; set; }
 
+        [JsonProperty("resourceManagerBuildVersion")]
         public string ResourceManagerBuildVersion { get; set; }
 
+        [JsonProperty("resourceManagerVersionBuiltOn")]
         public string ResourceManagerVersionBuiltOn { get; set; }
 
+        [JsonProperty("hadoopVersion")]
         public string HadoopVersion { get; set; }
 
+        [JsonProperty("hadoopBuildVersion")]
         public string HadoopBuildVersion { get; set; }
 
+        [JsonProperty("hadoopVersionBuiltOn")]
         public string HadoopVersionBuiltOn { get; set; }
-
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterMetrics.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterMetrics.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterMetrics.cs
index 1958e54..04c4346 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterMetrics.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterMetrics.cs
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using Newtonsoft.Json;
+
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {
     /// <summary>
@@ -27,50 +29,73 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
         internal static readonly string Resource = @"cluster/metrics";
         internal static readonly string RootElement = @"clusterMetrics";
 
+        [JsonProperty("appsSubmitted")]
         public int AppsSubmitted { get; set; }
 
+        [JsonProperty("appsCompleted")]
         public int AppsCompleted { get; set; }
 
+        [JsonProperty("appsPending")]
         public int AppsPending { get; set; }
 
+        [JsonProperty("appsRunning")]
         public int AppsRunning { get; set; }
 
+        [JsonProperty("appsFailed")]
         public int AppsFailed { get; set; }
 
+        [JsonProperty("appsKilled")]
         public int AppsKilled { get; set; }
 
+        [JsonProperty("reservedMB")]
         public long ReservedMB { get; set; }
 
+        [JsonProperty("availableMB")]
         public long AvailableMB { get; set; }
 
+        [JsonProperty("allocatedMB")]
         public long AllocatedMB { get; set; }
 
+        [JsonProperty("totalMB")]
         public long TotalMB { get; set; }
 
+        [JsonProperty("reservedVirtualCores")]
         public long ReservedVirtualCores { get; set; }
 
+        [JsonProperty("availableVirtualCores")]
         public long AvailableVirtualCores { get; set; }
 
+        [JsonProperty("allocatedVirtualCores")]
         public long AllocatedVirtualCores { get; set; }
 
+        [JsonProperty("totalVirtualCores")]
         public long TotalVirtualCores { get; set; }
 
+        [JsonProperty("containersAllocated")]
         public int ContainersAllocated { get; set; }
 
+        [JsonProperty("containersReserved")]
         public int ContainersReserved { get; set; }
 
+        [JsonProperty("containersPending")]
         public int ContainersPending { get; set; }
 
+        [JsonProperty("totalNodes")]
         public int TotalNodes { get; set; }
 
+        [JsonProperty("activeNodes")]
         public int ActiveNodes { get; set; }
 
+        [JsonProperty("lostNodes")]
         public int LostNodes { get; set; }
 
+        [JsonProperty("unhealthyNodes")]
         public int UnhealthyNodes { get; set; }
 
+        [JsonProperty("decommissionedNodes")]
         public int DecommissionedNodes { get; set; }
 
+        [JsonProperty("rebootedNodes")]
         public int RebootedNodes { get; set; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterState.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterState.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterState.cs
new file mode 100644
index 0000000..bcb07e3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ClusterState.cs
@@ -0,0 +1,35 @@
+// 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.
+
+namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
+{
+    /// <summary>
+    /// Class generated based on schema provided in
+    /// <see cref="!:http://hadoop.apache.org/docs/r2.6.0/hadoop-yarn/hadoop-yarn-site/WebServicesIntro.html">
+    /// Hadoop RM REST API </see> documentation.
+    /// </summary>
+    internal enum ClusterState
+    {
+        NOTINITED,
+
+        INITED,
+
+        STARTED,
+
+        STOPPED
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Commands.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Commands.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Commands.cs
index edabb6d..f6d9c44 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Commands.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Commands.cs
@@ -16,10 +16,13 @@
 // under the License.
 
 
+using Newtonsoft.Json;
+
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {
     internal class Commands
     {
+        [JsonProperty(PropertyName = "command")]
         public string Command { get; set; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Credentials.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Credentials.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Credentials.cs
index b843236..781698d 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Credentials.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Credentials.cs
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using Newtonsoft.Json;
+
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {
     /// <summary>
@@ -24,8 +26,10 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
     /// </summary>
     internal sealed class Credentials
     {
+        [JsonProperty(PropertyName = "tokens")]
         public Tokens Tokens;
 
+        [JsonProperty(PropertyName = "secrets")]
         public Secrets Secrets;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/KeyValuePair.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/KeyValuePair.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/KeyValuePair.cs
new file mode 100644
index 0000000..74a929a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/KeyValuePair.cs
@@ -0,0 +1,35 @@
+// 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 Newtonsoft.Json;
+
+namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
+{
+    /// <summary>
+    /// YARNRM does not allow case mismatch in received JSON.
+    /// Hence we need our own implementation of KeyValuePair
+    /// where we can annotate PropertyName
+    /// </summary>
+    internal sealed class KeyValuePair<T1, T2>
+    {
+        [JsonProperty(PropertyName = "key")]
+        public T1 Key { get; set; }
+
+        [JsonProperty(PropertyName = "value")]
+        public T2 Value { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/LocalResources.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/LocalResources.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/LocalResources.cs
index c479978..ed3a677 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/LocalResources.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/LocalResources.cs
@@ -30,7 +30,8 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
     /// </summary>
     internal sealed class LocalResources
     {
-        public IList<KeyValuePair<string, LocalResourcesValue>> Entry { get; set; }
+        [JsonProperty(PropertyName = "entry")]
+        public IList<KeyValuePair<string, LocalResourcesValue>> Entries { get; set; }
     }
 
     /// <summary>
@@ -40,16 +41,19 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
     /// </summary>
     internal sealed class LocalResourcesValue
     {
+        [JsonProperty(PropertyName = "resource")]
         public string Resource { get; set; }
 
-        [JsonProperty(ItemConverterType = typeof(StringEnumConverter))]
+        [JsonProperty(ItemConverterType = typeof(StringEnumConverter), PropertyName = "type")]
         public ResourceType Type { get; set; }
 
-        [JsonProperty(ItemConverterType = typeof(StringEnumConverter))]
+        [JsonProperty(ItemConverterType = typeof(StringEnumConverter), PropertyName = "visibility")]
         public Visibility Visibility { get; set; }
 
+        [JsonProperty(PropertyName = "size")]
         public long Size { get; set; }
 
+        [JsonProperty(PropertyName = "timestamp")]
         public long Timestamp { get; set; }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/NewApplication.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/NewApplication.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/NewApplication.cs
index 0468cc0..dc5d8db 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/NewApplication.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/NewApplication.cs
@@ -16,7 +16,6 @@
 // under the License.
 
 using Newtonsoft.Json;
-using RestSharp.Deserializers;
 
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Resouce.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Resouce.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Resouce.cs
index c5d98b7..ae0f4ad 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Resouce.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Resouce.cs
@@ -16,8 +16,6 @@
 // under the License.
 
 using Newtonsoft.Json;
-using RestSharp.Deserializers;
-using RestSharp.Serializers;
 
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {
@@ -31,6 +29,7 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
         [JsonProperty(PropertyName = "memory")]
         public int MemoryMB { get; set; }
 
+        [JsonProperty(PropertyName = "vCores")]
         public int VCores { get; set; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ServiceData.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ServiceData.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ServiceData.cs
index 66145ab..6723aea 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ServiceData.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ServiceData.cs
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System.Collections.Generic;
+using Newtonsoft.Json;
 
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
 {
@@ -26,6 +26,7 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
     /// </summary>
     internal sealed class ServiceData
     {
+        [JsonProperty(PropertyName = "entry")]
         public KeyValuePair<string, string> Entry { get; set; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/SubmitApplication.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/SubmitApplication.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/SubmitApplication.cs
index 8493187..91e04cb 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/SubmitApplication.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/SubmitApplication.cs
@@ -35,14 +35,16 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
         [JsonProperty(PropertyName = "application-name")]
         public string ApplicationName { get; set; }
 
+        [JsonProperty(PropertyName = "queue")]
         public string Queue { get; set; }
 
+        [JsonProperty(PropertyName = "priority")]
         public int Priority { get; set; }
 
         [JsonProperty(PropertyName = "am-container-spec")]
         public AmContainerSpec AmContainerSpec { get; set; }
 
-        [JsonProperty(PropertyName = "unmanaged-am")]
+        [JsonProperty(PropertyName = "unmanaged-AM")]
         public bool UnmanagedAM { get; set; }
 
         [JsonProperty(PropertyName = "max-app-attempts")]

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestClientFactory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestClientFactory.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestClientFactory.cs
index dbcb8b9..77e813d 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestClientFactory.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestClientFactory.cs
@@ -16,6 +16,7 @@
 // under the License.
 
 using System;
+using Org.Apache.REEF.Client.YARN.RestClient;
 using Org.Apache.REEF.Tang.Annotations;
 using RestSharp;
 
@@ -40,10 +41,12 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
             // as one client can contact only one baseUri.
             // This is not very bad but it might still be worth
             // it to cache clients per baseUri in the future.
-            return new RestSharp.RestClient(baseUri)
+            var restClient = new RestSharp.RestClient(baseUri)
             {
                 FollowRedirects = true
             };
+            restClient.AddHandler("application/json", new RestJsonDeserializer());
+            return restClient;
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonDeserializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonDeserializer.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonDeserializer.cs
new file mode 100644
index 0000000..aa6fd7b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonDeserializer.cs
@@ -0,0 +1,65 @@
+// 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 Newtonsoft.Json;
+using Newtonsoft.Json.Converters;
+using Newtonsoft.Json.Linq;
+using RestSharp;
+using RestSharp.Deserializers;
+
+namespace Org.Apache.REEF.Client.YARN.RestClient
+{
+    internal sealed class RestJsonDeserializer : IDeserializer
+    {
+        public string RootElement { get; set; }
+        public string Namespace { get; set; }
+        public string DateFormat { get; set; }
+
+        public T Deserialize<T>(IRestResponse response)
+        {
+            /* If root element is not empty, then we want to 
+             * skip the top level token and parse only one level deeper
+             * For instance:
+             * {
+             *     "app" : {
+             *        "state" : "FINISHED",
+             *        "user" : "user1",
+             *     }
+             *  }
+             * when used without this code will need an extra wrapper class
+             * around a class that has `state` and `user` fields.
+             * 
+             * public class AppResponse
+             * {
+             *      public Application App { get;set }
+             * }
+             * 
+             * This logic helps us avoid such classes.
+            */ 
+            if (!string.IsNullOrEmpty(RootElement))
+            {
+                var jobject = JObject.Parse(response.Content);
+                var jtoken = jobject[RootElement];
+                var jsonSerializer = new JsonSerializer();
+                jsonSerializer.Converters.Add(new StringEnumConverter());
+                return jtoken.ToObject<T>(jsonSerializer);
+            }
+
+            return JsonConvert.DeserializeObject<T>(response.Content, new StringEnumConverter());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonSerializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonSerializer.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonSerializer.cs
index f8a09ac..abd88b9 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonSerializer.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonSerializer.cs
@@ -16,6 +16,7 @@
 // under the License.
 
 using Newtonsoft.Json;
+using Newtonsoft.Json.Converters;
 using RestSharp.Serializers;
 
 namespace Org.Apache.REEF.Client.YARN.RestClient
@@ -26,7 +27,7 @@ namespace Org.Apache.REEF.Client.YARN.RestClient
     /// simple ISerializer implementation that uses Newtonsoft.Json
     /// for performing serialization
     /// </summary>
-    internal class RestJsonSerializer : ISerializer
+    internal sealed class RestJsonSerializer : ISerializer
     {
         public RestJsonSerializer()
         {
@@ -43,7 +44,7 @@ namespace Org.Apache.REEF.Client.YARN.RestClient
 
         public string Serialize(object obj)
         {
-            return JsonConvert.SerializeObject(obj);
+            return JsonConvert.SerializeObject(obj, new StringEnumConverter());
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/9fedfbc0/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestRequestExecutor.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestRequestExecutor.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestRequestExecutor.cs
index 206c608..75187a5 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestRequestExecutor.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestRequestExecutor.cs
@@ -22,12 +22,14 @@ using System.Threading.Tasks;
 using Newtonsoft.Json;
 using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
 using RestSharp;
 
 namespace Org.Apache.REEF.Client.Yarn.RestClient
 {
     internal class RestRequestExecutor : IRestRequestExecutor
     {
+        private static readonly Logger Log = Logger.GetLogger(typeof(RestRequestExecutor));
         private readonly IRestClientFactory _clientFactory;
 
         [Inject]
@@ -60,6 +62,7 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
                 // exception to user.
                 if (response.StatusCode >= HttpStatusCode.Ambiguous)
                 {
+                    Log.Log(Level.Error, "RESTRequest failed. StatusCode: {0}; Response: {1}", response.StatusCode, response.Content);
                     var errorResponse = JsonConvert.DeserializeObject<Error>(response.Content);
                     throw new YarnRestAPIException { Error = errorResponse };
                 }