You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by se...@apache.org on 2015/10/14 23:07:59 UTC

incubator-reef git commit: [REEF-757] Add submit application support in Yarn Rest client * Create datamodels for submission request * Plumb the REST API * Add unittest and YARN test

Repository: incubator-reef
Updated Branches:
  refs/heads/master 8b7189bd7 -> b6aaed0f5


[REEF-757] Add submit application support in Yarn Rest client
 * Create datamodels for submission request
 * Plumb the REST API
 * Add unittest and YARN test

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


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

Branch: refs/heads/master
Commit: b6aaed0f5b65e9d71113aef37f2ce6d0703f1f3a
Parents: 8b7189b
Author: Anupam <an...@gmail.com>
Authored: Tue Oct 6 19:17:58 2015 -0700
Committer: Beysim Sezgin <be...@hotmail.com>
Committed: Wed Oct 14 14:06:24 2015 -0700

----------------------------------------------------------------------
 .../WindowsHadoopEmulatorYarnClientTests.cs     |  89 +++++++--
 .../YarnClientTests.cs                          | 194 ++++++++++++++++++-
 .../Org.Apache.REEF.Client.csproj               |  16 +-
 .../YARN/RESTClient/DataModel/Acls.cs           |  49 +++++
 .../RESTClient/DataModel/AmContainerSpec.cs     |  44 +++++
 .../YARN/RESTClient/DataModel/ApplicationTag.cs |  24 +++
 .../YARN/RESTClient/DataModel/Commands.cs       |  25 +++
 .../YARN/RESTClient/DataModel/Credentials.cs    |  31 +++
 .../YARN/RESTClient/DataModel/Environment.cs    |  33 ++++
 .../YARN/RESTClient/DataModel/LocalResources.cs |  73 +++++++
 .../DataModel/MaximumResourceCapability.cs      |  31 ---
 .../YARN/RESTClient/DataModel/NewApplication.cs |   7 +-
 .../YARN/RESTClient/DataModel/Resouce.cs        |  36 ++++
 .../YARN/RESTClient/DataModel/Secrets.cs        |  33 ++++
 .../YARN/RESTClient/DataModel/ServiceData.cs    |  31 +++
 .../RESTClient/DataModel/SubmitApplication.cs   |  63 ++++++
 .../YARN/RESTClient/DataModel/Tokens.cs         |  33 ++++
 .../YARN/RESTClient/IRestClientFactory.cs       |   5 +-
 .../YARN/RESTClient/IRestRequestExecutor.cs     |  39 ++++
 .../YARN/RESTClient/IUrlProvider.cs             |  29 +++
 .../YARN/RESTClient/IYarnRMClient.cs            |  11 +-
 .../YARN/RESTClient/RestJsonSerializer.cs       |  49 +++++
 .../YARN/RESTClient/RestRequestExecutor.cs      |  31 +--
 .../YARN/RESTClient/YarnClient.cs               |  72 +++++--
 .../RESTClient/YarnClientNoCancellationToken.cs |   6 +
 25 files changed, 956 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 a9b493c..d802a16 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/WindowsHadoopEmulatorYarnClientTests.cs
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/WindowsHadoopEmulatorYarnClientTests.cs
@@ -19,8 +19,10 @@ using System;
 using System.Collections.Generic;
 using System.Linq;
 using System.ServiceProcess;
+using System.Threading.Tasks;
 using Microsoft.VisualStudio.TestTools.UnitTesting;
 using Org.Apache.REEF.Client.Yarn.RestClient;
+using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 
 namespace Org.Apache.REEF.Client.Tests
@@ -66,11 +68,11 @@ namespace Org.Apache.REEF.Client.Tests
 
         [TestMethod]
         [TestCategory("Functional")]
-        public void TestGetClusterInfo()
+        public async Task TestGetClusterInfo()
         {
             var client = TangFactory.GetTang().NewInjector().GetInstance<IYarnRMClient>();
 
-            var clusterInfo = client.GetClusterInfoAsync().GetAwaiter().GetResult();
+            var clusterInfo = await client.GetClusterInfoAsync();
 
             Assert.IsNotNull(clusterInfo);
             Assert.AreEqual("STARTED", clusterInfo.State);
@@ -80,34 +82,89 @@ namespace Org.Apache.REEF.Client.Tests
 
         [TestMethod]
         [TestCategory("Functional")]
-        public void TestGetClusterMetrics()
+        public async Task TestGetClusterMetrics()
         {
             var client = TangFactory.GetTang().NewInjector().GetInstance<IYarnRMClient>();
 
-            var clusterMetrics = client.GetClusterMetricsAsync().GetAwaiter().GetResult();
+            var clusterMetrics = await client.GetClusterMetricsAsync();
 
             Assert.IsNotNull(clusterMetrics);
             Assert.IsTrue(clusterMetrics.TotalMB > 0);
             Assert.IsTrue(clusterMetrics.ActiveNodes > 0);
         }
 
-        //// TODO: [REEF-757] Once submit API is in place, submit an app then get the details
-        ////[TestMethod]
-        ////[TestCategory("Functional")]
-        ////public void TestGetApplication()
-        ////{
-        ////    const string applicationName = @"application_1440795762187_0001";
+        [TestMethod]
+        [TestCategory("Functional")]
+        public async Task TestApplicationSubmissionAndQuery()
+        {
+            var client = TangFactory.GetTang().NewInjector().GetInstance<IYarnRMClient>();
+
+            var newApplication = await client.CreateNewApplicationAsync();
 
-        ////    var client = TangFactory.GetTang().NewInjector().GetInstance<IYarnRMClient>();
+            Assert.IsNotNull(newApplication);
+            Assert.IsFalse(string.IsNullOrEmpty(newApplication.ApplicationId));
+            Assert.IsTrue(newApplication.MaximumResourceCapability.MemoryMB > 0);
+            Assert.IsTrue(newApplication.MaximumResourceCapability.VCores > 0);
 
-        ////    var application = client.GetApplicationAsync(applicationName).GetAwaiter().GetResult();
+            string applicationName = "REEFTEST_APPLICATION_" + Guid.NewGuid();
+            Console.WriteLine(applicationName);
 
-        ////    Assert.IsNotNull(application);
-        ////}
+            const string anyApplicationType = "REEFTest";
+            var submitApplicationRequest = new SubmitApplication
+            {
+                ApplicationId = newApplication.ApplicationId,
+                AmResource = new Resouce
+                {
+                    MemoryMB = 500,
+                    VCores = 1
+                },
+                ApplicationType = anyApplicationType,
+                ApplicationName = applicationName,
+                KeepContainersAcrossApplicationAttempts = false,
+                MaxAppAttempts = 1,
+                Priority = 1,
+                UnmanagedAM = false,
+                AmContainerSpec = new AmContainerSpec
+                {
+                    Commands = new Commands
+                    {
+                        Command = @"DONTCARE"
+                    },
+                    LocalResources = new LocalResources
+                    {
+                        Entry = new List<KeyValuePair<string, LocalResourcesValue>>
+                        {
+                            new KeyValuePair<string, LocalResourcesValue>(
+                                "APPLICATIONWILLFAILBUTWEDONTCAREHERE",
+                                new LocalResourcesValue
+                                {
+                                    Resource = "Foo",
+                                    Type = ResourceType.FILE,
+                                    Visibility = Visibility.APPLICATION
+                                })
+                        }
+                    }
+                }
+            };
+
+            var application = await client.SubmitApplicationAsync(submitApplicationRequest);
+
+            Assert.IsNotNull(application);
+            Assert.AreEqual(newApplication.ApplicationId, application.Id);
+            Assert.AreEqual(applicationName, application.Name);
+            Assert.AreEqual(anyApplicationType, application.ApplicationType);
+
+            var getApplicationResult = client.GetApplicationAsync(newApplication.ApplicationId).GetAwaiter().GetResult();
+
+            Assert.IsNotNull(getApplicationResult);
+            Assert.AreEqual(newApplication.ApplicationId, getApplicationResult.Id);
+            Assert.AreEqual(applicationName, getApplicationResult.Name);
+            Assert.AreEqual(anyApplicationType, getApplicationResult.ApplicationType);
+        }
 
         [TestMethod]
         [TestCategory("Functional")]
-        public void TestErrorResponse()
+        public async Task TestErrorResponse()
         {
             const string WrongApplicationName = @"Something";
 
@@ -115,7 +172,7 @@ namespace Org.Apache.REEF.Client.Tests
 
             try
             {
-                client.GetApplicationAsync(WrongApplicationName).GetAwaiter().GetResult();
+                await client.GetApplicationAsync(WrongApplicationName);
                 Assert.Fail("Should throw YarnRestAPIException");
             }
             catch (YarnRestAPIException)

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 fa56d4d..8e83ae4 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/YarnClientTests.cs
@@ -16,11 +16,15 @@
 // under the License.
 
 using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Net;
 using System.Threading;
 using System.Threading.Tasks;
 using Microsoft.VisualStudio.TestTools.UnitTesting;
 using NSubstitute;
 using Org.Apache.REEF.Client.Yarn.RestClient;
+using Org.Apache.REEF.Client.YARN.RestClient;
 using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Tang.Util;
@@ -32,7 +36,7 @@ namespace Org.Apache.REEF.Client.Tests
     public class YarnClientTests
     {
         [TestMethod]
-        public void TestGetClusterInfo()
+        public async Task TestGetClusterInfo()
         {
             // arrange
             var ctx = new TestContext();
@@ -56,15 +60,15 @@ namespace Org.Apache.REEF.Client.Tests
 
             // act
             var yarnClient = ctx.GetClient();
-            ClusterInfo actualClusterInfo = yarnClient.GetClusterInfoAsync().GetAwaiter().GetResult();
+            ClusterInfo actualClusterInfo = await yarnClient.GetClusterInfoAsync();
 
             // assert
             Assert.AreEqual(anyClusterInfo, actualClusterInfo);
-            urlProvider.Received(1).GetUrlAsync();
+            var unused = urlProvider.Received(1).GetUrlAsync();
         }
 
         [TestMethod]
-        public void TestGetClusterMetrics()
+        public async Task TestGetClusterMetrics()
         {
             var ctx = new TestContext();
             var urlProvider = ctx.UrlProviderFake;
@@ -87,14 +91,14 @@ namespace Org.Apache.REEF.Client.Tests
                 CancellationToken.None).Returns(Task.FromResult(anyClusterMetrics));
 
             var yarnClient = ctx.GetClient();
-            ClusterMetrics actualClusterMetrics = yarnClient.GetClusterMetricsAsync().GetAwaiter().GetResult();
+            ClusterMetrics actualClusterMetrics = await yarnClient.GetClusterMetricsAsync();
 
             Assert.AreEqual(anyClusterMetrics, actualClusterMetrics);
-            urlProvider.Received(1).GetUrlAsync();
+            var unused = urlProvider.Received(1).GetUrlAsync();
         }
 
         [TestMethod]
-        public void TestGetApplication()
+        public async Task TestGetApplication()
         {
             var ctx = new TestContext();
             var urlProvider = ctx.UrlProviderFake;
@@ -122,10 +126,182 @@ namespace Org.Apache.REEF.Client.Tests
                 CancellationToken.None).Returns(Task.FromResult(anyApplication));
 
             var yarnClient = ctx.GetClient();
-            Application actualApplication = yarnClient.GetApplicationAsync(applicationId).GetAwaiter().GetResult();
+            Application actualApplication = await yarnClient.GetApplicationAsync(applicationId);
 
             Assert.AreEqual(anyApplication, actualApplication);
-            urlProvider.Received(1).GetUrlAsync();
+            var unused = urlProvider.Received(1).GetUrlAsync();
+        }
+
+        [TestMethod]
+        public async Task TestCreateNewApplication()
+        {
+            var ctx = new TestContext();
+            var urlProvider = ctx.UrlProviderFake;
+            var restReqExecutor = ctx.RestRequestExecutorFake;
+            Uri anyUri = new Uri("anyscheme://anypath");
+            const string applicationId = "AnyApplicationId";
+            urlProvider.GetUrlAsync().Returns(Task.FromResult(anyUri));
+            var anyNewApplication = new NewApplication
+            {
+                ApplicationId = applicationId
+            };
+            restReqExecutor.ExecuteAsync<NewApplication>(
+                Arg.Is<IRestRequest>(
+                    req =>
+                        req.Resource == "ws/v1/cluster/apps/new-application"
+                        && req.Method == Method.POST),
+                anyUri,
+                CancellationToken.None).Returns(Task.FromResult(anyNewApplication));
+
+            var yarnClient = ctx.GetClient();
+            NewApplication actualNewApplication = await yarnClient.CreateNewApplicationAsync();
+
+            Assert.AreEqual(anyNewApplication, actualNewApplication);
+            var unused = urlProvider.Received(1).GetUrlAsync();
+        }
+
+        [TestMethod]
+        public async Task TestSubmitNewApplication()
+        {
+            var ctx = new TestContext();
+            var urlProvider = ctx.UrlProviderFake;
+            var restReqExecutor = ctx.RestRequestExecutorFake;
+            Uri anyUri = new Uri("anyscheme://anypath");
+            const string applicationId = "AnyApplicationId";
+            const string anyApplicationType = "REEFTest";
+            const string anyApplicationName = "AnyAPP";
+            urlProvider.GetUrlAsync().Returns(Task.FromResult(anyUri));
+            var anySubmitApplication = new SubmitApplication
+            {
+                ApplicationId = applicationId,
+                AmResource = new Resouce
+                {
+                    MemoryMB = 500,
+                    VCores = 1
+                },
+                ApplicationType = anyApplicationType,
+                ApplicationName = anyApplicationName,
+                KeepContainersAcrossApplicationAttempts = false,
+                MaxAppAttempts = 1,
+                Priority = 1,
+                UnmanagedAM = false,
+                AmContainerSpec = new AmContainerSpec
+                {
+                    Commands = new Commands
+                    {
+                        Command = @"DONTCARE"
+                    },
+                    LocalResources = new LocalResources
+                    {
+                        Entry = new List<KeyValuePair<string, LocalResourcesValue>>
+                        {
+                            new KeyValuePair<string, LocalResourcesValue>(
+                                "APPLICATIONWILLFAILBUTWEDONTCAREHERE",
+                                new LocalResourcesValue
+                                {
+                                    Resource = "Foo",
+                                    Type = ResourceType.FILE,
+                                    Visibility = Visibility.APPLICATION
+                                })
+                        }
+                    }
+                }
+            };
+
+            const string expectedJson = @"{" +
+                                            @"""application-id"":""AnyApplicationId""," +
+                                            @"""application-name"":""AnyAPP""," +
+                                            @"""Queue"":null,""Priority"":1," +
+                                            @"""am-container-spec"":" +
+                                            @"{" +
+                                                @"""local-resources"":" +
+                                                @"{" +
+                                                    @"""Entry"":" +
+                                                    @"[" +
+                                                        @"{" +
+                                                            @"""Key"":""APPLICATIONWILLFAILBUTWEDONTCAREHERE""," +
+                                                            @"""Value"":" +
+                                                            @"{" +
+                                                                @"""Resource"":""Foo""," +
+                                                                @"""Type"":1," +
+                                                                @"""Visibility"":2," +
+                                                                @"""Size"":0," +
+                                                                @"""Timestamp"":0" +
+                                                            @"}" +
+                                                        @"}" +
+                                                    @"]" +
+                                                @"}," + 
+                                            @"""Environment"":null," +
+                                            @"""Commands"":" +
+                                            @"{" +
+                                                @"""Command"":""DONTCARE""" +
+                                            @"}," +
+                                            @"""service-data"":null," +
+                                            @"""Credentials"":null," +
+                                            @"""application-acls"":null}," +
+                                            @"""unmanaged-am"":false," +
+                                            @"""max-app-attempts"":1," +
+                                            @"""resource"":" +
+                                            @"{" +
+                                                @"""memory"":500," +
+                                                @"""VCores"":1" +
+                                            @"},""application-type"":""REEFTest""," +
+                                            @"""keep-containers-across-application-attempts"":false," +
+                                            @"""application-tags"":null" +
+                                        @"}";
+
+            var thisApplication = new Application
+            {
+                AllocatedMB = 100,
+                AmHostHttpAddress = "http://anyhttpaddress",
+                AmContainerLogs = "SomeLogs",
+                ApplicationType = "AnyYarnApplicationType",
+                State = State.FINISHED,
+                Name = "AnyApplicationName",
+                RunningContainers = 0
+            };
+
+            var response = Substitute.For<IRestResponse>();
+            response.Headers.Returns(new List<Parameter>
+            {
+                new Parameter
+                {
+                    Name = "Location",
+                    Value = "http://somelocation"
+                }
+            });
+            response.StatusCode.Returns(HttpStatusCode.Accepted);
+
+            restReqExecutor.ExecuteAsync(
+                Arg.Is<IRestRequest>(
+                    req =>
+                        req.Resource == "ws/v1/cluster/apps"
+                        && req.Method == Method.POST
+                        && req.JsonSerializer is RestJsonSerializer
+                        && req.Parameters.First().Name == "application/json"
+                        && B(req, expectedJson)),
+                anyUri,
+                CancellationToken.None).Returns(Task.FromResult(response));
+
+            restReqExecutor.ExecuteAsync<Application>(
+                Arg.Is<IRestRequest>(
+                    req =>
+                        req.Resource == "ws/v1/cluster/apps/" + applicationId
+                        && req.RootElement == "app"
+                        && req.Method == Method.GET),
+                anyUri,
+                CancellationToken.None).Returns(Task.FromResult(thisApplication));
+
+            var yarnClient = ctx.GetClient();
+            Application actualApplication = await yarnClient.SubmitApplicationAsync(anySubmitApplication);
+
+            Assert.AreEqual(thisApplication, actualApplication);
+            var unused = urlProvider.Received(2).GetUrlAsync();
+        }
+
+        private static bool B(IRestRequest req, string expectedJson)
+        {
+            return (string)req.Parameters.First().Value == expectedJson;
         }
 
         private class TestContext

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 c261dfe..5f31ded 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
@@ -78,6 +78,20 @@ under the License.
     <Compile Include="Properties\AssemblyInfo.cs" />
     <Compile Include="YARN\Parameters\JobSubmissionDirectoryPrefixParameter.cs" />
     <Compile Include="YARN\Parameters\SecurityTokenParameters.cs" />
+    <Compile Include="YARN\RestClient\DataModel\Acls.cs" />
+    <Compile Include="YARN\RestClient\DataModel\AmContainerSpec.cs" />
+    <Compile Include="YARN\RestClient\DataModel\ApplicationTag.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\LocalResources.cs" />
+    <Compile Include="YARN\RestClient\DataModel\Secrets.cs" />
+    <Compile Include="YARN\RestClient\DataModel\ServiceData.cs" />
+    <Compile Include="YARN\RestClient\DataModel\SubmitApplication.cs" />
+    <Compile Include="YARN\RestClient\DataModel\Tokens.cs" />
+    <Compile Include="YARN\RestClient\IRestRequestExecutor.cs" />
+    <Compile Include="YARN\RestClient\IUrlProvider.cs" />
+    <Compile Include="YARN\RestClient\RestJsonSerializer.cs" />
     <Compile Include="YARN\YARNREEFClient.cs" />
     <Compile Include="YARN\RestClient\IRestClientFactory.cs" />
     <Compile Include="YARN\RestClient\RestRequestExecutor.cs" />
@@ -88,7 +102,7 @@ under the License.
     <Compile Include="YARN\RestClient\DataModel\ClusterInfo.cs" />
     <Compile Include="YARN\RestClient\DataModel\ClusterMetrics.cs" />
     <Compile Include="YARN\RestClient\DataModel\Error.cs" />
-    <Compile Include="YARN\RestClient\DataModel\MaximumResourceCapability.cs" />
+    <Compile Include="YARN\RestClient\DataModel\Resouce.cs" />
     <Compile Include="YARN\RestClient\DataModel\NewApplication.cs" />
     <Compile Include="YARN\RestClient\YarnClient.cs" />
     <Compile Include="YARN\RestClient\YarnClientNoCancellationToken.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..a6194d4
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Acls.cs
@@ -0,0 +1,49 @@
+// 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 Newtonsoft.Json;
+using Newtonsoft.Json.Converters;
+
+// ReSharper disable InconsistentNaming; Name kept in sync with YARN 
+// API documentation
+
+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 sealed class Acls
+    {
+        [JsonProperty(ItemConverterType = typeof(StringEnumConverter))]
+        public IList<KeyValuePair<AclsType, string>> Entry { get; set; }
+    }
+
+    /// <summary>
+    /// Enum 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 AclsType
+    {
+        VIEW_APP,
+
+        MODIFY_APP
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..074b823
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/AmContainerSpec.cs
@@ -0,0 +1,44 @@
+// 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>
+    /// 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 sealed class AmContainerSpec
+    {
+        [JsonProperty(PropertyName = "local-resources")]
+        public LocalResources LocalResources { get; set; }
+
+        public Environment Environment { get; set; }
+
+        public Commands Commands { get; set; }
+
+        [JsonProperty(PropertyName = "service-data")]
+        public ServiceData ServiceData { get; set; }
+
+        public Credentials Credentials { get; set; }
+
+        [JsonProperty(PropertyName = "application-acls")]
+        public Acls ApplicationAcls { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..c9a4e98
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ApplicationTag.cs
@@ -0,0 +1,24 @@
+// 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
+{
+    internal class ApplicationTag
+    {
+        public string Tag { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..edabb6d
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Commands.cs
@@ -0,0 +1,25 @@
+// 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
+{
+    internal class Commands
+    {
+        public string Command { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..b843236
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Credentials.cs
@@ -0,0 +1,31 @@
+// 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 sealed class Credentials
+    {
+        public Tokens Tokens;
+
+        public Secrets Secrets;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Environment.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Environment.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Environment.cs
new file mode 100644
index 0000000..4c92fe7
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Environment.cs
@@ -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.
+
+using System.Collections.Generic;
+using Newtonsoft.Json;
+
+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 sealed class Environment
+    {
+        [JsonProperty(PropertyName = "entry")]
+        public IList<KeyValuePair<string, string>> Entries { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..c479978
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/LocalResources.cs
@@ -0,0 +1,73 @@
+// 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 Newtonsoft.Json;
+using Newtonsoft.Json.Converters;
+
+// ReSharper disable InconsistentNaming; Name replicates YARN REST API
+
+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 sealed class LocalResources
+    {
+        public IList<KeyValuePair<string, LocalResourcesValue>> Entry { get; set; }
+    }
+
+    /// <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 sealed class LocalResourcesValue
+    {
+        public string Resource { get; set; }
+
+        [JsonProperty(ItemConverterType = typeof(StringEnumConverter))]
+        public ResourceType Type { get; set; }
+
+        [JsonProperty(ItemConverterType = typeof(StringEnumConverter))]
+        public Visibility Visibility { get; set; }
+
+        public long Size { get; set; }
+
+        public long Timestamp { get; set; }
+    }
+
+    internal enum ResourceType
+    {
+        ARCHIVE,
+
+        FILE,
+
+        PATTERN
+    }
+
+    internal enum Visibility
+    {
+        PUBLIC,
+
+        PRIVATE,
+
+        APPLICATION
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/MaximumResourceCapability.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/MaximumResourceCapability.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/MaximumResourceCapability.cs
deleted file mode 100644
index 1555328..0000000
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/MaximumResourceCapability.cs
+++ /dev/null
@@ -1,31 +0,0 @@
-// 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 sealed class MaximumResourceCapability
-    {
-        public int Memory { get; set; }
-
-        public int VCores { get; set; }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 481f66c..0468cc0 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
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+using Newtonsoft.Json;
 using RestSharp.Deserializers;
 
 namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
@@ -28,10 +29,10 @@ namespace Org.Apache.REEF.Client.YARN.RestClient.DataModel
     {
         internal static readonly string Resource = @"cluster/apps/new-application";
 
-        [DeserializeAs(Name = "application-id")]
+        [JsonProperty(PropertyName = "application-id")]
         public string ApplicationId { get; set; }
 
-        [DeserializeAs(Name = "maximum-resource-capability")]
-        public MaximumResourceCapability MaximumResourceCapability { get; set; }
+        [JsonProperty(PropertyName = "maximum-resource-capability")]
+        public Resouce MaximumResourceCapability { get; set; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..c5d98b7
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Resouce.cs
@@ -0,0 +1,36 @@
+// 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 RestSharp.Deserializers;
+using RestSharp.Serializers;
+
+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 sealed class Resouce
+    {
+        [JsonProperty(PropertyName = "memory")]
+        public int MemoryMB { get; set; }
+
+        public int VCores { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Secrets.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Secrets.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Secrets.cs
new file mode 100644
index 0000000..2410ca3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Secrets.cs
@@ -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.
+
+using System.Collections.Generic;
+using Newtonsoft.Json;
+
+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 sealed class Secrets
+    {
+        [JsonProperty(PropertyName = "entry")]
+        public IList<KeyValuePair<string, string>> Entries { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..66145ab
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/ServiceData.cs
@@ -0,0 +1,31 @@
+// 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;
+
+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 sealed class ServiceData
+    {
+        public KeyValuePair<string, string> Entry { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..8493187
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/SubmitApplication.cs
@@ -0,0 +1,63 @@
+// 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 Newtonsoft.Json;
+
+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 sealed class SubmitApplication
+    {
+        internal static readonly string Resource = @"cluster/apps";
+
+        [JsonProperty(PropertyName = "application-id")]
+        public string ApplicationId { get; set; }
+
+        [JsonProperty(PropertyName = "application-name")]
+        public string ApplicationName { get; set; }
+
+        public string Queue { get; set; }
+
+        public int Priority { get; set; }
+
+        [JsonProperty(PropertyName = "am-container-spec")]
+        public AmContainerSpec AmContainerSpec { get; set; }
+
+        [JsonProperty(PropertyName = "unmanaged-am")]
+        public bool UnmanagedAM { get; set; }
+
+        [JsonProperty(PropertyName = "max-app-attempts")]
+        public int MaxAppAttempts { get; set; }
+
+        [JsonProperty(PropertyName = "resource")]
+        public Resouce AmResource { get; set; }
+
+        [JsonProperty(PropertyName = "application-type")]
+        public string ApplicationType { get; set; }
+
+        [JsonProperty(PropertyName = "keep-containers-across-application-attempts")]
+        public bool KeepContainersAcrossApplicationAttempts { get; set; }
+
+        [JsonProperty(PropertyName = "application-tags")]
+        public IList<ApplicationTag> ApplicationTags { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Tokens.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Tokens.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Tokens.cs
new file mode 100644
index 0000000..d76816f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/DataModel/Tokens.cs
@@ -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.
+
+using System.Collections.Generic;
+using Newtonsoft.Json;
+
+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 sealed class Tokens
+    {
+        [JsonProperty(PropertyName = "entry")]
+        public IList<KeyValuePair<string, string>> Entries { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 32e40dd..dbcb8b9 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestClientFactory.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestClientFactory.cs
@@ -40,7 +40,10 @@ 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);
+            return new RestSharp.RestClient(baseUri)
+            {
+                FollowRedirects = true
+            };
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestRequestExecutor.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestRequestExecutor.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestRequestExecutor.cs
new file mode 100644
index 0000000..7bb492a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IRestRequestExecutor.cs
@@ -0,0 +1,39 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+// 
+//   http://www.apache.org/licenses/LICENSE-2.0
+// 
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using System;
+using System.Threading;
+using System.Threading.Tasks;
+using Org.Apache.REEF.Tang.Annotations;
+using RestSharp;
+
+namespace Org.Apache.REEF.Client.Yarn.RestClient
+{
+    [DefaultImplementation(typeof(RestRequestExecutor))]
+    internal interface IRestRequestExecutor
+    {
+        Task<T> ExecuteAsync<T>(
+            IRestRequest request,
+            Uri uri,
+            CancellationToken cancellationToken) where T : new();
+
+        Task<IRestResponse> ExecuteAsync(
+            IRestRequest request,
+            Uri uri,
+            CancellationToken cancellationToken);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IUrlProvider.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IUrlProvider.cs b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IUrlProvider.cs
new file mode 100644
index 0000000..85126d3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IUrlProvider.cs
@@ -0,0 +1,29 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using System;
+using System.Threading.Tasks;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.Yarn.RestClient
+{
+    [DefaultImplementation(typeof(YarnConfigurationUrlProvider))]
+    internal interface IUrlProvider
+    {
+        Task<Uri> GetUrlAsync();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 92e35c7..6a69374 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IYarnRMClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/IYarnRMClient.cs
@@ -15,7 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-using System;
 using System.Threading;
 using System.Threading.Tasks;
 using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
@@ -41,11 +40,11 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
         Task<NewApplication> CreateNewApplicationAsync();
 
         Task<NewApplication> CreateNewApplicationAsync(CancellationToken cancellationToken);
-    }
 
-    [DefaultImplementation(typeof(YarnConfigurationUrlProvider))]
-    internal interface IUrlProvider
-    {
-        Task<Uri> GetUrlAsync();
+        Task<Application> SubmitApplicationAsync(SubmitApplication submitApplicationRequest);
+
+        Task<Application> SubmitApplicationAsync(
+            SubmitApplication submitApplicationRequest,
+            CancellationToken cancellationToken);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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
new file mode 100644
index 0000000..f8a09ac
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestJsonSerializer.cs
@@ -0,0 +1,49 @@
+// 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 RestSharp.Serializers;
+
+namespace Org.Apache.REEF.Client.YARN.RestClient
+{
+    /// <summary>
+    /// RestSharp by default uses SimpleJsonSerializer which 
+    /// does not understand property renaming. Here we create a
+    /// simple ISerializer implementation that uses Newtonsoft.Json
+    /// for performing serialization
+    /// </summary>
+    internal class RestJsonSerializer : ISerializer
+    {
+        public RestJsonSerializer()
+        {
+            ContentType = "application/json";
+        }
+
+        public string RootElement { get; set; }
+
+        public string Namespace { get; set; }
+
+        public string DateFormat { get; set; }
+
+        public string ContentType { get; set; }
+
+        public string Serialize(object obj)
+        {
+            return JsonConvert.SerializeObject(obj);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 b9555a7..206c608 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestRequestExecutor.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/RestRequestExecutor.cs
@@ -26,15 +26,6 @@ using RestSharp;
 
 namespace Org.Apache.REEF.Client.Yarn.RestClient
 {
-    [DefaultImplementation(typeof(RestRequestExecutor))]
-    internal interface IRestRequestExecutor
-    {
-        Task<T> ExecuteAsync<T>(
-            IRestRequest request,
-            Uri uri,
-            CancellationToken cancellationToken) where T : new();
-    }
-
     internal class RestRequestExecutor : IRestRequestExecutor
     {
         private readonly IRestClientFactory _clientFactory;
@@ -56,14 +47,18 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
             var response =
                 await
                     client.ExecuteTaskAsync<T>(request, cancellationToken);
+
             if (response.ErrorException != null)
             {
-                throw new Exception("Executing REST API failed", response.ErrorException);
+                throw new YarnRestAPIException("Executing REST API failed", response.ErrorException);
             }
 
             try
             {
-                if (response.StatusCode != HttpStatusCode.OK)
+                // HTTP status code greater than 300 is unexpected here.
+                // See if the server sent a error response and throw suitable
+                // exception to user.
+                if (response.StatusCode >= HttpStatusCode.Ambiguous)
                 {
                     var errorResponse = JsonConvert.DeserializeObject<Error>(response.Content);
                     throw new YarnRestAPIException { Error = errorResponse };
@@ -76,5 +71,19 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
 
             return response.Data;
         }
+
+        public async Task<IRestResponse> ExecuteAsync(IRestRequest request, Uri uri, CancellationToken cancellationToken)
+        {
+            var client = _clientFactory.CreateRestClient(uri);
+
+            try
+            {
+                return await client.ExecuteTaskAsync(request, cancellationToken);
+            }
+            catch (Exception exception)
+            {
+                throw new YarnRestAPIException("Unhandled exception in executing REST request.", exception);
+            }
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 ca26ed3..f54c5f3 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClient.cs
@@ -16,8 +16,10 @@
 // under the License.
 
 using System;
+using System.Net;
 using System.Threading;
 using System.Threading.Tasks;
+using Org.Apache.REEF.Client.YARN.RestClient;
 using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
 using Org.Apache.REEF.Tang.Annotations;
 using Org.Apache.REEF.Utilities.AsyncUtils;
@@ -49,11 +51,8 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
         {
             await new RemoveSynchronizationContextAwaiter();
 
-            IRestRequest request = new RestRequest
-            {
-                Resource = _baseResourceString + ClusterInfo.Resource,
-                RootElement = ClusterInfo.RootElement
-            };
+            IRestRequest request = CreateRestRequest( ClusterInfo.Resource, Method.GET, ClusterInfo.RootElement
+            );
 
             return
                 await GenerateUrlAndExecuteRequestAsync<ClusterInfo>(request, cancellationToken);
@@ -63,11 +62,7 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
         {
             await new RemoveSynchronizationContextAwaiter();
 
-            var request = new RestRequest
-            {
-                Resource = _baseResourceString + ClusterMetrics.Resource,
-                RootElement = ClusterMetrics.RootElement
-            };
+            var request = CreateRestRequest(ClusterMetrics.Resource, Method.GET, ClusterMetrics.RootElement);
 
             return
                 await
@@ -78,11 +73,7 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
         {
             await new RemoveSynchronizationContextAwaiter();
 
-            var request = new RestRequest
-            {
-                Resource = _baseResourceString + Application.Resource + appId,
-                RootElement = Application.RootElement
-            };
+            var request = CreateRestRequest(Application.Resource + appId, Method.GET, Application.RootElement);
 
             return
                 await GenerateUrlAndExecuteRequestAsync<Application>(request, cancellationToken);
@@ -92,17 +83,49 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
         {
             await new RemoveSynchronizationContextAwaiter();
 
-            var request = new RestRequest
-            {
-                Resource = _baseResourceString + NewApplication.Resource,
-                Method = Method.POST
-            };
+            var request = CreateRestRequest(NewApplication.Resource, Method.POST);
 
             return
                 await
                     GenerateUrlAndExecuteRequestAsync<NewApplication>(request, cancellationToken);
         }
 
+        public async Task<Application> SubmitApplicationAsync(
+            SubmitApplication submitApplication,
+            CancellationToken cancellationToken)
+        {
+            await new RemoveSynchronizationContextAwaiter();
+
+            var request = CreateRestRequest(SubmitApplication.Resource, Method.POST);
+
+            request.AddBody(submitApplication);
+            var submitResponse = await GenerateUrlAndExecuteRequestAsync(request, cancellationToken);
+
+            if (submitResponse.StatusCode != HttpStatusCode.Accepted)
+            {
+                throw new YarnRestAPIException(
+                    string.Format("Application submission failed with HTTP STATUS {0}",
+                    submitResponse.StatusCode));
+            }
+
+            return await GetApplicationAsync(submitApplication.ApplicationId, cancellationToken);
+        }
+
+        private RestRequest CreateRestRequest(string resourcePath, Method method, string rootElement = null)
+        {
+            var request = new RestRequest
+            {
+                Resource = _baseResourceString + resourcePath,
+                RootElement = rootElement,
+                Method = method,
+                RequestFormat = DataFormat.Json,
+                JsonSerializer = new RestJsonSerializer()
+            };
+
+            return request;
+        }
+
+
         private async Task<T> GenerateUrlAndExecuteRequestAsync<T>(IRestRequest request,
             CancellationToken cancellationToken)
             where T : new()
@@ -112,5 +135,14 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
                 await
                     _restRequestExecutor.ExecuteAsync<T>(request, yarnRmUri, cancellationToken);
         }
+
+        private async Task<IRestResponse> GenerateUrlAndExecuteRequestAsync(IRestRequest request,
+            CancellationToken cancellationToken)
+        {
+            Uri yarnRmUri = await _yarnRmUrlProviderUri.GetUrlAsync();
+            return
+                await
+                    _restRequestExecutor.ExecuteAsync(request, yarnRmUri, cancellationToken);
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/b6aaed0f/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 ee7628f..707a3a0 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClientNoCancellationToken.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/RESTClient/YarnClientNoCancellationToken.cs
@@ -47,5 +47,11 @@ namespace Org.Apache.REEF.Client.Yarn.RestClient
             await new RemoveSynchronizationContextAwaiter();
             return await CreateNewApplicationAsync(CancellationToken.None);
         }
+
+        public async Task<Application> SubmitApplicationAsync(SubmitApplication submitApplicationRequest)
+        {
+            await new RemoveSynchronizationContextAwaiter();
+            return await SubmitApplicationAsync(submitApplicationRequest, CancellationToken.None);
+        }
     }
 }
\ No newline at end of file