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

[06/16] reef git commit: [REEF-1965] Implement .NET REEF runtime for Azure Batch

[REEF-1965] Implement .NET REEF runtime for Azure Batch

This implements
    * Azure Batch DotNet client that submits REEF Job to Azure Batch pool.
    * HelloReef example runs in Azure Batch.

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

Closes #1441


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

Branch: refs/heads/REEF-335
Commit: 8d0bad248c5421d1d906ce5fc67b0863817c70b0
Parents: aa30e7f
Author: Chenxi Zhao <ch...@microsoft.com>
Authored: Thu Nov 30 17:48:31 2017 -0800
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Wed Apr 25 15:50:23 2018 -0700

----------------------------------------------------------------------
 .../AzureBatchCommandBuilderTests.cs            |  56 ++++++++
 .../Org.Apache.REEF.Client.Tests.csproj         |   1 +
 .../AvroAzureBatchJobSubmissionParameters.cs    | 107 ++++++++++++++
 .../AzureBatch/AzureBatchDotNetClient.cs        | 139 +++++++++++++++++++
 .../AzureBatchRuntimeClientConfiguration.cs     |  78 +++++++++++
 .../Parameters/AzureBatchAccountKey.cs          |  25 ++++
 .../Parameters/AzureBatchAccountName.cs         |  26 ++++
 .../Parameters/AzureBatchAccountUri.cs          |  26 ++++
 .../AzureBatch/Parameters/AzureBatchPoolId.cs   |  26 ++++
 .../Parameters/AzureStorageAccountKey.cs        |  26 ++++
 .../Parameters/AzureStorageAccountName.cs       |  26 ++++
 .../Parameters/AzureStorageContainerName.cs     |  26 ++++
 .../Parameters/DriverStderrFilePath.cs          |  29 ++++
 .../Parameters/DriverStdoutFilePath.cs          |  29 ++++
 .../AzureBatch/Parameters/EnableDebugLogging.cs |  28 ++++
 .../AzureBatch/Parameters/IsWindows.cs          |  26 ++++
 .../AzureBatch/Service/AzureBatchService.cs     | 138 ++++++++++++++++++
 .../AzureBatch/Storage/AzureStorageClient.cs    | 109 +++++++++++++++
 .../AzureBatch/Util/AbstractCommandBuilder.cs   |  72 ++++++++++
 .../AzureBatchCommandProviderConfiguration.cs   |  42 ++++++
 .../AzureBatch/Util/AzureBatchFileNames.cs      |  55 ++++++++
 .../AzureBatch/Util/ICommandBuilder.cs          |  35 +++++
 .../AzureBatch/Util/JobJarMaker.cs              | 108 ++++++++++++++
 .../AzureBatch/Util/LinuxCommandBuilder.cs      |  44 ++++++
 .../AzureBatch/Util/WindowsCommandBuilder.cs    |  52 +++++++
 .../Common/DriverFolderPreparationHelper.cs     |  46 ++++--
 .../Org.Apache.REEF.Client.DotNet.csproj        |   1 +
 .../Org.Apache.REEF.Client.csproj               |  60 +++++++-
 lang/cs/Org.Apache.REEF.Client/packages.config  |  13 ++
 .../Runtime/RuntimeName.cs                      |   5 +
 .../HelloREEF.cs                                |  25 +++-
 .../Org.Apache.REEF.Examples.HelloREEF/Run.cs   |   9 +-
 lang/cs/Org.Apache.REEF.sln                     | Bin 37102 -> 37230 bytes
 lang/java/reef-bridge-client/pom.xml            |   5 +
 .../src/main/avro/JobSubmissionParameters.avsc  |  17 +++
 ...zureBatchBootstrapDriverConfigGenerator.java |  85 ++++++++++++
 .../client/AzureBatchBootstrapREEFLauncher.java | 135 ++++++++++++++++++
 lang/java/reef-bridge-java/pom.xml              |   5 +
 38 files changed, 1712 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client.Tests/AzureBatchCommandBuilderTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/AzureBatchCommandBuilderTests.cs b/lang/cs/Org.Apache.REEF.Client.Tests/AzureBatchCommandBuilderTests.cs
new file mode 100644
index 0000000..489753b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/AzureBatchCommandBuilderTests.cs
@@ -0,0 +1,56 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Client.AzureBatch.Util;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Xunit;
+
+namespace Org.Apache.REEF.Client.Tests
+{
+    public class AzureBatchCommandBuilder
+    {
+        [Fact]
+        public void WindowsCommandBuilderDriverTest()
+        {
+            // Prepare
+            const int driverMemory = 100;
+            AbstractCommandBuilder builder = TestContext.GetWindowsCommandBuilder();
+            string expected = "powershell.exe /c \"Add-Type -AssemblyName System.IO.Compression.FileSystem; " +
+                            "[System.IO.Compression.ZipFile]::ExtractToDirectory(\\\"$env:AZ_BATCH_TASK_WORKING_DIR\\local.jar\\\", " +
+                            "\\\"$env:AZ_BATCH_TASK_WORKING_DIR\\reef\\\");reef\\Org.Apache.REEF.Bridge.exe " +
+                            "java -Xmx100m -XX:PermSize=128m -XX:MaxPermSize=128m -classpath 'reef/local/*;reef/global/*;' " +
+                            "-Dproc_reef org.apache.reef.bridge.client.AzureBatchBootstrapREEFLauncher " +
+                            "reef\\job-submission-params.json\";";
+
+            // Action
+            string actual = builder.BuildDriverCommand(driverMemory);
+
+            // Assert
+            Assert.Equal(expected, actual);
+        }
+
+        private class TestContext
+        {
+            public static AbstractCommandBuilder GetWindowsCommandBuilder()
+            {
+                IInjector injector = TangFactory.GetTang().NewInjector();
+                return injector.GetInstance<WindowsCommandBuilder>();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj b/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
index 384e178..0602f70 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
@@ -51,6 +51,7 @@ under the License.
     <Compile Include="$(SolutionDir)\SharedAssemblyInfo.cs">
       <Link>Properties\SharedAssemblyInfo.cs</Link>
     </Compile>
+    <Compile Include="AzureBatchCommandBuilderTests.cs" />
     <Compile Include="HDInsightYarnClientTests.cs" />
     <Compile Include="JobRequestBuilderTests.cs" />
     <Compile Include="JobResourceUploaderTests.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs b/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs
new file mode 100644
index 0000000..78bfe4a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Avro/AzureBatch/AvroAzureBatchJobSubmissionParameters.cs
@@ -0,0 +1,107 @@
+// 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.Runtime.Serialization;
+using Org.Apache.REEF.Utilities.Attributes;
+
+namespace Org.Apache.REEF.Client.Avro.AzureBatch
+{
+    /// <summary>
+    /// Used to serialize and deserialize Avro record 
+    /// org.apache.reef.reef.bridge.client.avro.AvroAzureBatchJobSubmissionParameters.
+    /// This is a (mostly) auto-generated class. 
+    /// For instructions on how to regenerate, please view the README.md in the same folder.
+    /// </summary>
+    [Private]
+    [DataContract(Namespace = "org.apache.reef.reef.bridge.client.avro")]
+    public sealed class AvroAzureBatchJobSubmissionParameters
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroAzureBatchJobSubmissionParameters"",""doc"":""Job submission parameters used by the Azure Batch runtime"",""fields"":[{""name"":""sharedJobSubmissionParameters"",""type"":{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroJobSubmissionParameters"",""doc"":""General cross-language job submission parameters shared by all runtimes"",""fields"":[{""name"":""jobId"",""type"":""string""},{""name"":""jobSubmissionFolder"",""type"":""string""}]}},{""name"":""AzureBatchAccountKey"",""type"":""string""},{""name"":""AzureBatchAccountName"",""type"":""string""},{""name"":""AzureBatchAccountUri"",""type"":""string""},{""name"":""AzureBatchPoolId"",""type"":""string""},{""name"":""AzureStorageAccountKey"",""type"":""string""},{""name"":""AzureStorageAccountName"",""type"":""string""},{""name"":""AzureStorageContainerName"",""type"":""string""},{""name"":
 ""AzureBatchIsWindows"",""type"":""boolean""}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+
+        /// <summary>
+        /// Gets or sets the sharedJobSubmissionParameters field.
+        /// </summary>
+        [DataMember]
+        public AvroJobSubmissionParameters sharedJobSubmissionParameters { get; set; }
+
+        /// <summary>
+        /// Gets or sets the AzureBatchAccountKey field.
+        /// </summary>
+        [DataMember]
+        public string AzureBatchAccountKey { get; set; }
+
+        /// <summary>
+        /// Gets or sets the AzureBatchAccountName field.
+        /// </summary>
+        [DataMember]
+        public string AzureBatchAccountName { get; set; }
+
+        /// <summary>
+        /// Gets or sets the AzureBatchAccountUri field.
+        /// </summary>
+        [DataMember]
+        public string AzureBatchAccountUri { get; set; }
+
+        /// <summary>
+        /// Gets or sets the AzureBatchPoolId field.
+        /// </summary>
+        [DataMember]
+        public string AzureBatchPoolId { get; set; }
+
+        /// <summary>
+        /// Gets or sets the AzureStorageAccountKey field.
+        /// </summary>
+        [DataMember]
+        public string AzureStorageAccountKey { get; set; }
+
+        /// <summary>
+        /// Gets or sets the AzureStorageAccountName field.
+        /// </summary>
+        [DataMember]
+        public string AzureStorageAccountName { get; set; }
+
+        /// <summary>
+        /// Gets or sets the AzureStorageContainerName field.
+        /// </summary>
+        [DataMember]
+        public string AzureStorageContainerName { get; set; }
+
+        /// <summary>
+        /// Gets or sets the AzureBatchIsWindows field.
+        /// </summary>
+        [DataMember]
+        public bool AzureBatchIsWindows { get; set; }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AvroAzureBatchJobSubmissionParameters"/> class.
+        /// </summary>
+        public AvroAzureBatchJobSubmissionParameters()
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/AzureBatchDotNetClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/AzureBatchDotNetClient.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/AzureBatchDotNetClient.cs
new file mode 100644
index 0000000..7f9bdb3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/AzureBatchDotNetClient.cs
@@ -0,0 +1,139 @@
+// 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.Client.API;
+using Org.Apache.REEF.Client.AzureBatch.Storage;
+using Org.Apache.REEF.Client.AzureBatch;
+using Org.Apache.REEF.Client.AzureBatch.Util;
+using Org.Apache.REEF.Client.Common;
+using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Client.DotNet.AzureBatch
+{
+    public sealed class AzureBatchDotNetClient : IREEFClient
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(AzureBatchDotNetClient));
+
+        /// Maximum number of characters allowed in Azure Batch job name. This limit is imposed by Azure Batch.
+        private const int AzureBatchMaxCharsJobName = 64;
+
+        private readonly IInjector _injector;
+        private readonly DriverFolderPreparationHelper _driverFolderPreparationHelper;
+        private readonly REEFFileNames _fileNames;
+        private readonly AzureStorageClient _azureStorageClient;
+        private readonly JobRequestBuilderFactory _jobRequestBuilderFactory;
+        private readonly AzureBatchService _batchService;
+        private readonly JobJarMaker _jobJarMaker;
+        private readonly AzureBatchFileNames _azbatchFileNames;
+
+        [Inject]
+        private AzureBatchDotNetClient(
+            IInjector injector,
+            IResourceArchiveFileGenerator resourceArchiveFileGenerator,
+            DriverFolderPreparationHelper driverFolderPreparationHelper,
+            AzureStorageClient azureStorageClient,
+            REEFFileNames fileNames,
+            AzureBatchFileNames azbatchFileNames,
+            JobRequestBuilderFactory jobRequestBuilderFactory,
+            AzureBatchService batchService,
+            JobJarMaker jobJarMaker)
+        {
+            _injector = injector;
+            _fileNames = fileNames;
+            _azbatchFileNames = azbatchFileNames;
+            _driverFolderPreparationHelper = driverFolderPreparationHelper;
+            _azureStorageClient = azureStorageClient;
+            _jobRequestBuilderFactory = jobRequestBuilderFactory;
+            _batchService = batchService;
+            _jobJarMaker = jobJarMaker;
+        }
+
+        public JobRequestBuilder NewJobRequestBuilder()
+        {
+            return _jobRequestBuilderFactory.NewInstance();
+        }
+
+        public Task<FinalState> GetJobFinalStatus(string appId)
+        {
+            // FinalState is DataModel in YARN. For Azure Batch runtime, this is not supported.
+            throw new NotImplementedException();
+        }
+
+        public void Submit(JobRequest jobRequest)
+        {
+            var configModule = AzureBatchRuntimeClientConfiguration.ConfigurationModule;
+            string jobId = jobRequest.JobIdentifier;
+            string azureBatchjobId = CreateAzureJobId(jobId);
+            string commandLine = GetCommand(jobRequest.JobParameters);
+            string jarPath = _jobJarMaker.CreateJobSubmissionJAR(jobRequest, azureBatchjobId);
+            string destination = _azbatchFileNames.GetStorageJobFolder(azureBatchjobId);
+            Uri blobUri = _azureStorageClient.UploadFile(destination, jarPath).Result;
+            string sasToken = _azureStorageClient.CreateContainerSharedAccessSignature();
+            _batchService.CreateJob(azureBatchjobId, blobUri, commandLine, sasToken);
+        }
+
+        private string GetCommand(JobParameters jobParameters)
+        {
+            var commandProviderConfigModule = AzureBatchCommandBuilderConfiguration.ConfigurationModule;
+            if (jobParameters.JavaLogLevel == JavaLoggingSetting.Verbose)
+            {
+                commandProviderConfigModule = commandProviderConfigModule
+                    .Set(AzureBatchCommandBuilderConfiguration.JavaDebugLogging, true.ToString().ToLowerInvariant());
+            }
+
+            if (jobParameters.StdoutFilePath.IsPresent())
+            {
+                commandProviderConfigModule = commandProviderConfigModule
+                    .Set(AzureBatchCommandBuilderConfiguration.DriverStdoutFilePath, jobParameters.StdoutFilePath.Value);
+            }
+
+            if (jobParameters.StderrFilePath.IsPresent())
+            {
+                commandProviderConfigModule = commandProviderConfigModule
+                    .Set(AzureBatchCommandBuilderConfiguration.DriverStderrFilePath, jobParameters.StderrFilePath.Value);
+            }
+
+            var azureBatchJobCommandBuilder = _injector.ForkInjector(commandProviderConfigModule.Build())
+                .GetInstance<ICommandBuilder>();
+
+            var command = azureBatchJobCommandBuilder.BuildDriverCommand(jobParameters.DriverMemoryInMB);
+
+            return command;
+        }
+
+        public IJobSubmissionResult SubmitAndGetJobStatus(JobRequest jobRequest)
+        {
+            Submit(jobRequest);
+            /// Azure Batch is not able to comminicate to client through driver end point. It behaves the same as Submit(JobRequest jobRequest).
+            return null;
+        }
+
+        private string CreateAzureJobId(string jobId)
+        {
+            string guid = Guid.NewGuid().ToString();
+            string jobNameShort = jobId.Length + 1 + guid.Length < AzureBatchMaxCharsJobName ?
+                jobId : jobId.Substring(0, AzureBatchMaxCharsJobName - guid.Length - 1);
+            return jobNameShort + "-" + guid;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/AzureBatchRuntimeClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/AzureBatchRuntimeClientConfiguration.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/AzureBatchRuntimeClientConfiguration.cs
new file mode 100644
index 0000000..31db860
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/AzureBatchRuntimeClientConfiguration.cs
@@ -0,0 +1,78 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.AzureBatch.Parameters;
+using Org.Apache.REEF.Client.DotNet.AzureBatch;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using System;
+using System.IO;
+
+namespace Org.Apache.REEF.Client.AzureBatch
+{
+    /// <summary>
+    /// The Configuration for the Azure Batch Client
+    /// </summary>
+    public sealed class AzureBatchRuntimeClientConfiguration : ConfigurationModuleBuilder
+    {
+        public static readonly string AzBatchConfigurationFileEnvironmentVariable = "REEF_AZBATCH_CONF";
+
+        public static readonly RequiredParameter<string> AzureBatchAccountUri = new RequiredParameter<string>();
+        public static readonly RequiredParameter<string> AzureBatchAccountName = new RequiredParameter<string>();
+        public static readonly RequiredParameter<string> AzureBatchAccountKey = new RequiredParameter<string>();
+        public static readonly RequiredParameter<string> AzureBatchPoolId = new RequiredParameter<string>();
+
+        public static readonly RequiredParameter<string> AzureStorageAccountName = new RequiredParameter<string>();
+        public static readonly RequiredParameter<string> AzureStorageAccountKey = new RequiredParameter<string>();
+        public static readonly RequiredParameter<string> AzureStorageContainerName = new RequiredParameter<string>();
+
+        public static ConfigurationModule ConfigurationModule = new AzureBatchRuntimeClientConfiguration()
+            .BindImplementation(GenericType<IREEFClient>.Class, GenericType<AzureBatchDotNetClient>.Class)
+            .BindNamedParameter(GenericType<AzureBatchAccountUri>.Class, AzureBatchAccountUri)
+            .BindNamedParameter(GenericType<AzureBatchAccountName>.Class, AzureBatchAccountName)
+            .BindNamedParameter(GenericType<AzureBatchAccountKey>.Class, AzureBatchAccountKey)
+            .BindNamedParameter(GenericType<AzureBatchPoolId>.Class, AzureBatchPoolId)
+            .BindNamedParameter(GenericType<AzureStorageAccountName>.Class, AzureStorageAccountName)
+            .BindNamedParameter(GenericType<AzureStorageAccountKey>.Class, AzureStorageAccountKey)
+            .BindNamedParameter(GenericType<AzureStorageContainerName>.Class, AzureStorageContainerName)
+            .Build();
+
+        public static IConfiguration FromTextFile(string file)
+        {
+            return new AvroConfigurationSerializer().FromFile(file);
+        }
+
+        public static IConfiguration FromEnvironment()
+        {
+            string configurationPath = Environment.GetEnvironmentVariable(AzBatchConfigurationFileEnvironmentVariable);
+
+            if (configurationPath == null)
+            {
+                throw new ArgumentException(@"Environment Variable {AzureBatchClientConfiguration.AzBatchConfigurationFileEnvironmentVariable} not set");
+            }
+
+            if (!File.Exists(configurationPath))
+            {
+                throw new ArgumentException(@"File located by Environment Variable {AzureBatchClientConfiguration.AzBatchConfigurationFileEnvironmentVariable} cannot be read.");
+            }
+
+            return FromTextFile(configurationPath);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountKey.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountKey.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountKey.cs
new file mode 100644
index 0000000..9d5aa71
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountKey.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.
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(Documentation = "The Azure Batch Account Key")]
+    public sealed class AzureBatchAccountKey : Name<string>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountName.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountName.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountName.cs
new file mode 100644
index 0000000..a0c1151
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountName.cs
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(Documentation = "The Azure Batch Account Name")]
+    public sealed class AzureBatchAccountName : Name<string>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountUri.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountUri.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountUri.cs
new file mode 100644
index 0000000..2a3466c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchAccountUri.cs
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(Documentation = "The Azure Batch account URI")]
+    public sealed class AzureBatchAccountUri : Name<string>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchPoolId.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchPoolId.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchPoolId.cs
new file mode 100644
index 0000000..da7766d
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureBatchPoolId.cs
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(Documentation = "The Azure Batch Pool Id")]
+    public sealed class AzureBatchPoolId : Name<string>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageAccountKey.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageAccountKey.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageAccountKey.cs
new file mode 100644
index 0000000..eef3f00
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageAccountKey.cs
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(Documentation = "The Azure Storage Account Key")]
+    public sealed class AzureStorageAccountKey : Name<string>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageAccountName.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageAccountName.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageAccountName.cs
new file mode 100644
index 0000000..0238feb
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageAccountName.cs
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(Documentation = "The Azure Storage Account Name")]
+    public sealed class AzureStorageAccountName : Name<string>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageContainerName.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageContainerName.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageContainerName.cs
new file mode 100644
index 0000000..bf93730
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/AzureStorageContainerName.cs
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(Documentation = "The Azure Storage Container Name")]
+    public sealed class AzureStorageContainerName : Name<string>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/DriverStderrFilePath.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/DriverStderrFilePath.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/DriverStderrFilePath.cs
new file mode 100644
index 0000000..2b96e33
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/DriverStderrFilePath.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 Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter("Driver stderr file path for Azure Batch.", defaultValue: "<LOG_DIR>/driver.stderr")]
+    internal sealed class DriverStderrFilePath : Name<string>
+    {
+        private DriverStderrFilePath()
+        {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/DriverStdoutFilePath.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/DriverStdoutFilePath.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/DriverStdoutFilePath.cs
new file mode 100644
index 0000000..1f2e546
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/DriverStdoutFilePath.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 Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter("Driver stdout file path for Azure Batch.", defaultValue: "<LOG_DIR>/driver.stdout")]
+    internal sealed class DriverStdoutFilePath : Name<string>
+    {
+        private DriverStdoutFilePath()
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/EnableDebugLogging.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/EnableDebugLogging.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/EnableDebugLogging.cs
new file mode 100644
index 0000000..ceb7f9e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/EnableDebugLogging.cs
@@ -0,0 +1,28 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+// 
+//   http://www.apache.org/licenses/LICENSE-2.0
+// 
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(
+        "Boolean indicating whether debug logging should be enabled in Java part of runtime.",
+        defaultValue: "false")]
+    public sealed class EnableDebugLogging : Name<bool>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/IsWindows.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/IsWindows.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/IsWindows.cs
new file mode 100644
index 0000000..3b16fee
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Parameters/IsWindows.cs
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Parameters
+{
+    [NamedParameter(Documentation = "Are the Azure Batch VMs linux or Windows based")]
+    public sealed class IsWindows : Name<bool>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Service/AzureBatchService.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Service/AzureBatchService.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Service/AzureBatchService.cs
new file mode 100644
index 0000000..b3e40da
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Service/AzureBatchService.cs
@@ -0,0 +1,138 @@
+// 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.Collections.Generic;
+using System.Threading.Tasks;
+using Microsoft.Azure.Batch;
+using Microsoft.Azure.Batch.Common;
+using Org.Apache.REEF.Client.AzureBatch.Parameters;
+using Org.Apache.REEF.Client.AzureBatch.Util;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using BatchSharedKeyCredential = Microsoft.Azure.Batch.Auth.BatchSharedKeyCredentials;
+
+namespace Org.Apache.REEF.Client.DotNet.AzureBatch
+{
+    public sealed class AzureBatchService : IDisposable
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(AzureBatchService));
+        private static readonly TimeSpan RetryDeltaBackOff = TimeSpan.FromSeconds(5);
+        private const string AzureStorageContainerSasToken = "AZURE_STORAGE_CONTAINER_SAS_TOKEN_ENV";
+        private const int MaxRetries = 3;
+
+        public BatchSharedKeyCredential Credentials { get; private set; }
+        public string PoolId { get; private set; }
+
+        private BatchClient Client { get; set; }
+        private bool disposed;
+
+        [Inject]
+        public AzureBatchService(
+            [Parameter(typeof(AzureBatchAccountUri))] string azureBatchAccountUri,
+            [Parameter(typeof(AzureBatchAccountName))] string azureBatchAccountName,
+            [Parameter(typeof(AzureBatchAccountKey))] string azureBatchAccountKey,
+            [Parameter(typeof(AzureBatchPoolId))] string azureBatchPoolId)
+        {
+            BatchSharedKeyCredential credentials = new BatchSharedKeyCredential(azureBatchAccountUri, azureBatchAccountName, azureBatchAccountKey);
+
+            this.Client = BatchClient.Open(credentials);
+            this.Credentials = credentials;
+            this.PoolId = azureBatchPoolId;
+            this.Client.CustomBehaviors.Add(new RetryPolicyProvider(new ExponentialRetry(RetryDeltaBackOff, MaxRetries)));
+        }
+
+        /// <summary>
+        /// Dispose of this object and all members
+        /// </summary>
+        public void Dispose()
+        {
+            this.Dispose(true);
+            GC.SuppressFinalize(this);
+        }
+
+        ~AzureBatchService()
+        {
+            this.Dispose(false);
+        }
+
+        /// <summary>
+        /// Disposes of this object
+        /// </summary>
+        private void Dispose(bool disposing)
+        {
+            if (this.disposed)
+            {
+                return;
+            }
+
+            if (disposing)
+            {
+                this.Client.Dispose();
+            }
+
+            this.disposed = true;
+        }
+
+        #region Job related operations
+
+        public void CreateJob(string jobId, Uri resourceFile, string commandLine, string storageContainerSAS)
+        {
+            CloudJob unboundJob = this.Client.JobOperations.CreateJob();
+            unboundJob.Id = jobId;
+            unboundJob.PoolInformation = new PoolInformation() { PoolId = this.PoolId };
+            unboundJob.JobManagerTask = new JobManagerTask()
+            {
+                Id = jobId,
+                CommandLine = commandLine,
+                RunExclusive = false,
+
+                ResourceFiles = resourceFile != null
+                    ? new List<ResourceFile>() { new ResourceFile(resourceFile.AbsoluteUri, AzureBatchFileNames.GetTaskJarFileName()) }
+                    : new List<ResourceFile>(),
+
+                EnvironmentSettings = new List<EnvironmentSetting> { new EnvironmentSetting(AzureStorageContainerSasToken, storageContainerSAS) },
+
+                // This setting will signal Batch to generate an access token and pass it
+                // to the Job Manager Task (aka the Driver) as an environment variable.
+                // For more info, see
+                // https://docs.microsoft.com/en-us/dotnet/api/microsoft.azure.batch.cloudtask.authenticationtokensettings
+                AuthenticationTokenSettings = new AuthenticationTokenSettings() { Access = AccessScope.Job }
+            };
+
+            unboundJob.Commit();
+
+            LOGGER.Log(Level.Info, "Submitted job {0}, commandLine {1} ", jobId, commandLine);
+        }
+
+        public CloudJob GetJob(string jobId, DetailLevel detailLevel)
+        {
+            using (Task<CloudJob> getJobTask = this.GetJobAsync(jobId, detailLevel))
+            {
+                getJobTask.Wait();
+                return getJobTask.Result;
+            }
+        }
+
+        public Task<CloudJob> GetJobAsync(string jobId, DetailLevel detailLevel)
+        {
+            return this.Client.JobOperations.GetJobAsync(jobId, detailLevel);
+        }
+
+        #endregion
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Storage/AzureStorageClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Storage/AzureStorageClient.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Storage/AzureStorageClient.cs
new file mode 100644
index 0000000..ce39f1e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Storage/AzureStorageClient.cs
@@ -0,0 +1,109 @@
+// 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 Microsoft.WindowsAzure.Storage;
+using Microsoft.WindowsAzure.Storage.Blob;
+using Org.Apache.REEF.Client.AzureBatch.Parameters;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using System;
+using System.IO;
+using System.Threading.Tasks;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Storage
+{
+    internal sealed class AzureStorageClient
+    {
+        private static readonly Logger LOGGER = Logger.GetLogger(typeof(AzureStorageClient));
+        private const string StorageConnectionStringFormat = "DefaultEndpointsProtocol=https;AccountName={0};AccountKey={1}";
+        private const int SASTokenValidityMinutes = 60;
+
+        private readonly string _storageAccountName;
+        private readonly string _storageAccountKey;
+        private readonly string _storageContainerName;
+
+        private readonly string _storageConnectionString;
+
+        [Inject]
+        AzureStorageClient(
+            [Parameter(typeof(AzureStorageAccountName))] string storageAccountName,
+            [Parameter(typeof(AzureStorageAccountKey))] string storageAccountKey,
+            [Parameter(typeof(AzureStorageContainerName))] string storageContainerName)
+        {
+            this._storageAccountName = storageAccountName;
+            this._storageAccountKey = storageAccountKey;
+            this._storageContainerName = storageContainerName;
+
+            this._storageConnectionString = string.Format(StorageConnectionStringFormat,
+                new object[] { storageAccountName, storageAccountKey });
+        }
+
+        /// <summary>
+        /// Uploads a given file to the given destination folder in Azure Storage.
+        /// </summary>
+        /// <param name="destination">Destination in Azure Storage where given file will be uploaded.</param>
+        /// <param name="filePath">Path to the file to be uploaded.</param>
+        /// <returns>Storage SAS URI for uploaded file.</returns>
+        public async Task<Uri> UploadFile(string destination, string filePath)
+        {
+            CloudBlobContainer blobContainer = await this.GetOrCreateCloudBlobContainer();
+            CloudBlobDirectory directory = blobContainer.GetDirectoryReference(destination);
+
+            string fileName = Path.GetFileName(filePath);
+            CloudBlockBlob blob = directory.GetBlockBlobReference(fileName);
+            await blob.UploadFromFileAsync(filePath);
+
+            string sas = blob.GetSharedAccessSignature(CreateSASPolicy());
+            string uri = blob.Uri.AbsoluteUri;
+            Uri uploadedFile = new Uri(uri + sas);
+            LOGGER.Log(Level.Info, "Uploaded {0} jar file to {1}", filePath, uploadedFile);
+            return uploadedFile;
+        }
+
+        public string CreateContainerSharedAccessSignature()
+        {
+            CloudBlobClient cloudBlobClient = CloudStorageAccount.Parse(this._storageConnectionString).CreateCloudBlobClient();
+            CloudBlobContainer cloudBlobContainer = cloudBlobClient.GetContainerReference(this._storageContainerName);
+            cloudBlobContainer.CreateIfNotExists();
+            return cloudBlobContainer.GetSharedAccessSignature(CreateSASPolicy());
+        }
+
+        private CloudBlobClient GetCloudBlobClient()
+        {
+            return CloudStorageAccount.Parse(this._storageConnectionString).CreateCloudBlobClient();
+        }
+
+        private async Task<CloudBlobContainer> GetOrCreateCloudBlobContainer()
+        {
+            CloudBlobClient blobClient = this.GetCloudBlobClient();
+            CloudBlobContainer blobContainer = blobClient.GetContainerReference(this._storageContainerName);
+            await blobContainer.CreateIfNotExistsAsync();
+
+            return blobContainer;
+        }
+
+        private SharedAccessBlobPolicy CreateSASPolicy()
+        {
+            return new SharedAccessBlobPolicy()
+            {
+                SharedAccessStartTime = DateTime.UtcNow,
+                SharedAccessExpiryTime = DateTime.UtcNow.AddMinutes(SASTokenValidityMinutes),
+                Permissions = SharedAccessBlobPermissions.Read | SharedAccessBlobPermissions.Write
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AbstractCommandBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AbstractCommandBuilder.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AbstractCommandBuilder.cs
new file mode 100644
index 0000000..4363bd7
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AbstractCommandBuilder.cs
@@ -0,0 +1,72 @@
+// 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.IO;
+using System.Text;
+using Org.Apache.REEF.Common.Files;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Util
+{
+    internal abstract class AbstractCommandBuilder : ICommandBuilder
+    {
+        private const string JavaExe = @"java";
+        private const string JvmOptionsPermSize = @"-XX:PermSize=128m";
+        private const string JvmOptionsMaxPermSizeFormat = @"-XX:MaxPermSize=128m";
+        private const string JvmOptionsMaxMemoryAllocationPoolSizeFormat = @"-Xmx{0}m";
+        private const string ClassPathToken = @"-classpath";
+        private const string ProcReefProperty = @"-Dproc_reef";
+        private const string LauncherClassName = @"org.apache.reef.bridge.client.AzureBatchBootstrapREEFLauncher";
+        protected readonly REEFFileNames _fileNames;
+        protected readonly string _osCommandFormat;
+        protected readonly string _commandPrefix;
+        protected readonly AzureBatchFileNames _azureBatchFileNames;
+
+        protected AbstractCommandBuilder(
+            REEFFileNames fileNames,
+            AzureBatchFileNames azureBatchFileNames,
+            string commandPrefix,
+            string osCommandFormat)
+        {
+            _fileNames = fileNames;
+            _osCommandFormat = osCommandFormat;
+            _commandPrefix = commandPrefix;
+            _azureBatchFileNames = azureBatchFileNames;
+        }
+
+        public string BuildDriverCommand(int driverMemory)
+        {
+            var sb = new StringBuilder();
+            sb.Append(_fileNames.GetBridgeExePath())
+              .Append(" " + JavaExe)
+              .Append(" " + string.Format(JvmOptionsMaxMemoryAllocationPoolSizeFormat, driverMemory))
+              .Append(" " + JvmOptionsPermSize)
+              .Append(" " + JvmOptionsMaxPermSizeFormat)
+              .Append(" " + ClassPathToken)
+              .Append(" " + GetDriverClasspath())
+              .Append(" " + ProcReefProperty)
+              .Append(" " + LauncherClassName)
+              .Append(" " + Path.Combine(_fileNames.GetReefFolderName(), _fileNames.GetJobSubmissionParametersFile()));
+            return string.Format(_osCommandFormat, _commandPrefix + sb.ToString());
+        }
+
+        /// <summary>
+        /// Returns the driver classpath string which is compatible with the intricacies of the OS.
+        /// </summary>
+        /// <returns>classpath parameter string.</returns>
+        protected abstract string GetDriverClasspath();
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AzureBatchCommandProviderConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AzureBatchCommandProviderConfiguration.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AzureBatchCommandProviderConfiguration.cs
new file mode 100644
index 0000000..f17d312
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AzureBatchCommandProviderConfiguration.cs
@@ -0,0 +1,42 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+// 
+//   http://www.apache.org/licenses/LICENSE-2.0
+// 
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Client.YARN.Parameters;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Util
+{
+    internal sealed class AzureBatchCommandBuilderConfiguration : ConfigurationModuleBuilder
+    {
+        public static readonly OptionalParameter<string> DriverStdoutFilePath = new OptionalParameter<string>();
+        public static readonly OptionalParameter<string> DriverStderrFilePath = new OptionalParameter<string>();
+        public static readonly OptionalParameter<bool> JavaDebugLogging = new OptionalParameter<bool>();
+
+        public static ConfigurationModule ConfigurationModule
+        {
+            get
+            {
+                return new AzureBatchCommandBuilderConfiguration()
+                    .BindNamedParameter(GenericType<DriverStdoutFilePath>.Class, DriverStdoutFilePath)
+                    .BindNamedParameter(GenericType<DriverStderrFilePath>.Class, DriverStderrFilePath)
+                    .BindNamedParameter(GenericType<EnableDebugLogging>.Class, JavaDebugLogging)
+                    .Build();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AzureBatchFileNames.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AzureBatchFileNames.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AzureBatchFileNames.cs
new file mode 100644
index 0000000..fc98c41
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/AzureBatchFileNames.cs
@@ -0,0 +1,55 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Util
+{
+    /// <summary>
+    /// Access to the various places things go according to the REEF Azure Batch runtime.
+    /// </summary>
+    internal sealed class AzureBatchFileNames
+    {
+        private const string StorageJobFolderPath = "apps/reef/jobs/";
+        private const string TaskJarFileName = "local.jar";
+        private readonly REEFFileNames _reefFileNames;
+
+        [Inject]
+        private AzureBatchFileNames(REEFFileNames reefFileNames)
+        {
+            _reefFileNames = reefFileNames;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <param name="jobId">Job Submission Id</param>
+        /// <returns>The relative path to the folder storing the job assets.</returns>
+        public string GetStorageJobFolder(string jobId)
+        {
+            return StorageJobFolderPath + jobId;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The name under which the task jar will be stored.</returns>
+        public static string GetTaskJarFileName()
+        {
+            return TaskJarFileName;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/ICommandBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/ICommandBuilder.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/ICommandBuilder.cs
new file mode 100644
index 0000000..23e83d4
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/ICommandBuilder.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 Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Util
+{
+    /// <summary>
+    /// Build the launch command for Java REEF processes for Azure Batch.
+    /// </summary>
+    [DefaultImplementation(typeof(WindowsCommandBuilder))]
+    internal interface ICommandBuilder
+    {
+        /// <summary>
+        /// Assembles the command to execute the Driver.
+        /// </summary>
+        /// <param name="driverMemory">The memory in megabytes used by driver.</param>
+        /// <returns>The command string.</returns>
+        string BuildDriverCommand(int driverMemory);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs
new file mode 100644
index 0000000..48e62e5
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/JobJarMaker.cs
@@ -0,0 +1,108 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.Avro;
+using Org.Apache.REEF.Client.Avro.AzureBatch;
+using Org.Apache.REEF.Client.AzureBatch.Parameters;
+using Org.Apache.REEF.Client.Common;
+using Org.Apache.REEF.Common.Avro;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+using System;
+using System.IO;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Util
+{
+    internal sealed class JobJarMaker
+    {
+        private readonly IResourceArchiveFileGenerator _resourceArchiveFileGenerator;
+        private readonly DriverFolderPreparationHelper _driverFolderPreparationHelper;
+        private readonly AvroAzureBatchJobSubmissionParameters _avroAzureBatchJobSubmissionParameters;
+        private readonly REEFFileNames _fileNames;
+
+        [Inject]
+        JobJarMaker(
+            IResourceArchiveFileGenerator resourceArchiveFileGenerator,
+            DriverFolderPreparationHelper driverFolderPreparationHelper,
+            REEFFileNames fileNames,
+            [Parameter(typeof(AzureBatchAccountKey))] string azureBatchAccountKey,
+            [Parameter(typeof(AzureBatchAccountName))] string azureBatchAccountName,
+            [Parameter(typeof(AzureBatchAccountUri))] string azureBatchAccountUri,
+            [Parameter(typeof(AzureBatchPoolId))] string azureBatchPoolId,
+            [Parameter(typeof(AzureStorageAccountKey))] string azureStorageAccountKey,
+            [Parameter(typeof(AzureStorageAccountName))] string azureStorageAccountName,
+            [Parameter(typeof(AzureStorageContainerName))] string azureStorageContainerName)
+        {
+            _resourceArchiveFileGenerator = resourceArchiveFileGenerator;
+            _driverFolderPreparationHelper = driverFolderPreparationHelper;
+            _fileNames = fileNames;
+            _avroAzureBatchJobSubmissionParameters = new AvroAzureBatchJobSubmissionParameters
+            {
+                AzureBatchAccountKey = azureBatchAccountKey,
+                AzureBatchAccountName = azureBatchAccountName,
+                AzureBatchAccountUri = azureBatchAccountUri,
+                AzureBatchPoolId = azureBatchPoolId,
+                AzureStorageAccountKey = azureStorageAccountKey,
+                AzureStorageAccountName = azureStorageAccountName,
+                AzureStorageContainerName = azureStorageContainerName,
+                AzureBatchIsWindows = true
+            };
+        }
+
+        /// <summary>
+        /// Creates a JAR file for the job submission.
+        /// </summary>
+        /// <param name="jobRequest">Job request received from the client code.</param>
+        /// <param name="azureBatchjobId">Azure Batch job Id going to be launched.</param>
+        /// <returns>A string path to file.</returns>
+        public string CreateJobSubmissionJAR(JobRequest jobRequest, string azureBatchjobId)
+        {
+            _avroAzureBatchJobSubmissionParameters.sharedJobSubmissionParameters = new AvroJobSubmissionParameters
+            {
+                jobId = jobRequest.JobIdentifier,
+                //// This is dummy in Azure Batch, as it does not use jobSubmissionFolder in Azure Batch.
+                jobSubmissionFolder = Path.PathSeparator.ToString()
+            };
+
+            string localDriverFolderPath = CreateDriverFolder(azureBatchjobId);
+
+            _driverFolderPreparationHelper.PrepareDriverFolderWithGlobalBridgeJar(jobRequest.AppParameters, localDriverFolderPath);
+            SerializeJobFile(localDriverFolderPath, _avroAzureBatchJobSubmissionParameters);
+
+            return _resourceArchiveFileGenerator.CreateArchiveToUpload(localDriverFolderPath);
+        }
+
+        private string CreateDriverFolder(string azureBatchjobId)
+        {
+            return Path.GetFullPath(Path.Combine(Path.GetTempPath(), azureBatchjobId) + Path.DirectorySeparatorChar);
+        }
+
+        private void SerializeJobFile(string localDriverFolderPath, AvroAzureBatchJobSubmissionParameters jobParameters)
+        {
+            var serializedArgs = AvroJsonSerializer<AvroAzureBatchJobSubmissionParameters>.ToBytes(jobParameters);
+
+            var submissionJobArgsFilePath = Path.Combine(localDriverFolderPath,
+                    _fileNames.GetReefFolderName(), _fileNames.GetJobSubmissionParametersFile());
+
+            using (var jobArgsFileStream = new FileStream(submissionJobArgsFilePath, FileMode.CreateNew))
+            {
+                jobArgsFileStream.Write(serializedArgs, 0, serializedArgs.Length);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/LinuxCommandBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/LinuxCommandBuilder.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/LinuxCommandBuilder.cs
new file mode 100644
index 0000000..eed2163
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/LinuxCommandBuilder.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 System;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Util
+{
+    internal sealed class LinuxCommandBuilder : AbstractCommandBuilder
+    {
+        private static readonly string CommandPrefix =
+            "unzip " + AzureBatchFileNames.GetTaskJarFileName() + " -d 'reef/'" + ";";
+        private const string ClassPathSeparator = ":";
+        private const string OsCommandFormat = "/bin/sh c \"{0}\"";
+
+        [Inject]
+        private LinuxCommandBuilder(
+            REEFFileNames fileNames,
+            AzureBatchFileNames azureBatchFileNames) : base(fileNames, azureBatchFileNames,
+                CommandPrefix, OsCommandFormat)
+        {
+        }
+
+        protected override string GetDriverClasspath()
+        {
+            throw new NotImplementedException();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/WindowsCommandBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/WindowsCommandBuilder.cs b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/WindowsCommandBuilder.cs
new file mode 100644
index 0000000..b7d6d2f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/AzureBatch/Util/WindowsCommandBuilder.cs
@@ -0,0 +1,52 @@
+// 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 Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Client.AzureBatch.Util
+{
+    internal sealed class WindowsCommandBuilder : AbstractCommandBuilder
+    {
+        private static readonly string CommandPrefix = @"Add-Type -AssemblyName System.IO.Compression.FileSystem; " +
+          "[System.IO.Compression.ZipFile]::ExtractToDirectory(\\\"$env:AZ_BATCH_TASK_WORKING_DIR\\" +
+              AzureBatchFileNames.GetTaskJarFileName() + "\\\", " +
+              "\\\"$env:AZ_BATCH_TASK_WORKING_DIR\\reef\\\");";
+        private const string ClassPathSeparator = ";";
+        private const string OsCommandFormat = "powershell.exe /c \"{0}\";";
+
+        [Inject]
+        private WindowsCommandBuilder(
+            REEFFileNames fileNames,
+            AzureBatchFileNames azureBatchFileNames) : base(fileNames, azureBatchFileNames,
+                CommandPrefix, OsCommandFormat)
+        {
+        }
+
+        protected override string GetDriverClasspath()
+        {
+            List<string> classpathList = new List<string>()
+            {
+                string.Format("{0}/{1}/*", _fileNames.GetReefFolderName(), _fileNames.GetLocalFolderName()),
+                string.Format("{0}/{1}/*", _fileNames.GetReefFolderName(), _fileNames.GetGlobalFolderName())
+            };
+
+            return string.Format("'{0};'", string.Join(ClassPathSeparator, classpathList));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs b/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
index f51a3c4..90e865d 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
@@ -82,26 +82,32 @@ namespace Org.Apache.REEF.Client.Common
         /// <param name="driverFolderPath"></param>
         internal void PrepareDriverFolder(AppParameters appParameters, string driverFolderPath)
         {
-            Logger.Log(Level.Verbose, "Preparing Driver filesystem layout in {0}", driverFolderPath);
-
-            // Setup the folder structure
-            CreateDefaultFolderStructure(appParameters, driverFolderPath);
-
             // Add the appParameters into that folder structure
             _fileSets.AddJobFiles(appParameters);
 
-            // Add the reef-bridge-client jar to the global files in the manner of JavaClientLauncher.cs.
+            // Add the reef-bridge-client jar to the local files in the manner of JavaClientLauncher.cs.
             _fileSets.AddToLocalFiles(Directory.GetFiles(JarFolder)
                 .Where(file => !string.IsNullOrWhiteSpace(file))
                 .Where(jarFile => Path.GetFileName(jarFile).ToLower().StartsWith(ClientConstants.ClientJarFilePrefix)));
 
-            // Create the driver configuration
-            CreateDriverConfiguration(appParameters, driverFolderPath);
+            InternalPrepareDriverFolder(appParameters, driverFolderPath);
+        }
 
-            // Initiate the final copy
-            _fileSets.CopyToDriverFolder(driverFolderPath);
+        /// <summary>
+        /// Prepares the working directory for a Driver in driverFolderPath.
+        /// </summary>
+        /// <param name="appParameters"></param>
+        /// <param name="driverFolderPath"></param>
+        internal void PrepareDriverFolderWithGlobalBridgeJar(AppParameters appParameters, string driverFolderPath)
+        {
+            // Add the appParameters into that folder structure
+            _fileSets.AddJobFiles(appParameters);
 
-            Logger.Log(Level.Info, "Done preparing Driver filesystem layout in {0}", driverFolderPath);
+            // Add the reef-bridge-client jar to the global files in the manner of JavaClientLauncher.cs.
+            _fileSets.AddToGlobalFiles(Directory.GetFiles(JarFolder)
+                .Where(jarFile => Path.GetFileName(jarFile).ToLower().StartsWith(ClientConstants.ClientJarFilePrefix)));
+
+            InternalPrepareDriverFolder(appParameters, driverFolderPath);
         }
 
         /// <summary>
@@ -144,7 +150,7 @@ namespace Org.Apache.REEF.Client.Common
                     File.WriteAllBytes(fileName, resourceHelper.GetBytes(fileResources.Value));
                 }
             }
-            
+
             // generate .config file for bridge executable
             var config = DefaultDriverConfigurationFileContents;
             if (!string.IsNullOrEmpty(appParameters.DriverConfigurationFileContents))
@@ -165,5 +171,21 @@ namespace Org.Apache.REEF.Client.Common
             Logger.Log(Level.Verbose, "Create EvaluatorConfigFile {0} with config {1}.", evaluatorConfigFilName, evaluatorAppConfigString);
             File.WriteAllText(evaluatorConfigFilName, evaluatorAppConfigString);
         }
+
+        private void InternalPrepareDriverFolder(AppParameters appParameters, string driverFolderPath)
+        {
+            Logger.Log(Level.Info, "Preparing Driver filesystem layout in {0}", driverFolderPath);
+
+            // Setup the folder structure
+            CreateDefaultFolderStructure(appParameters, driverFolderPath);
+
+            // Create the driver configuration
+            CreateDriverConfiguration(appParameters, driverFolderPath);
+
+            // Initiate the final copy
+            _fileSets.CopyToDriverFolder(driverFolderPath);
+
+            Logger.Log(Level.Info, "Done preparing Driver filesystem layout in {0}", driverFolderPath);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.DotNet.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.DotNet.csproj b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.DotNet.csproj
index a7cc2e8..835faf9 100644
--- a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.DotNet.csproj
+++ b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.DotNet.csproj
@@ -34,6 +34,7 @@ under the License.
     </EmbeddedResource>
   </ItemGroup>
   <ItemGroup>
+    <PackageReference Include="Azure.Batch" Version="8.0.0" />
     <PackageReference Include="Microsoft.Avro.Core" Version="$(AvroVersion)" />
     <PackageReference Include="System.Reactive.Interfaces" Version="$(SystemReactiveVersion)" />
     <PackageReference Include="EnterpriseLibrary.TransientFaultHandling.Core" Version="$(TransientFaultHandlingVersion)" />

http://git-wip-us.apache.org/repos/asf/reef/blob/8d0bad24/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 6245647..90c365e 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
@@ -37,11 +37,40 @@ under the License.
     <UseVSHostingProcess>false</UseVSHostingProcess>
   </PropertyGroup>
   <ItemGroup>
+    <Reference Include="Microsoft.Azure.Batch, Version=8.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\Azure.Batch.8.0.1\lib\net452\Microsoft.Azure.Batch.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.Azure.KeyVault.Core, Version=1.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\Microsoft.Azure.KeyVault.Core.1.0.0\lib\net40\Microsoft.Azure.KeyVault.Core.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="Microsoft.CSharp" />
+    <Reference Include="Microsoft.Data.Edm, Version=5.8.1.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\Microsoft.Data.Edm.5.8.2\lib\net40\Microsoft.Data.Edm.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Data.OData, Version=5.8.1.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\Microsoft.Data.OData.5.8.2\lib\net40\Microsoft.Data.OData.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Data.Services.Client, Version=5.8.1.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\Microsoft.Data.Services.Client.5.8.2\lib\net40\Microsoft.Data.Services.Client.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="Microsoft.Practices.TransientFaultHandling.Core, Version=5.1.1209.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
       <HintPath>$(PackagesDir)\TransientFaultHandling.Core.5.1.1209.1\lib\NET4\Microsoft.Practices.TransientFaultHandling.Core.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="Microsoft.Rest.ClientRuntime, Version=2.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\Microsoft.Rest.ClientRuntime.2.3.10\lib\net452\Microsoft.Rest.ClientRuntime.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Rest.ClientRuntime.Azure, Version=3.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\Microsoft.Rest.ClientRuntime.Azure.3.3.10\lib\net452\Microsoft.Rest.ClientRuntime.Azure.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.WindowsAzure.Storage, Version=8.1.3.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\WindowsAzure.Storage.8.1.3\lib\net45\Microsoft.WindowsAzure.Storage.dll</HintPath>
+    </Reference>
     <Reference Include="Newtonsoft.Json">
       <HintPath>$(PackagesDir)\Newtonsoft.Json.$(NewtonsoftJsonVersion)\lib\net45\Newtonsoft.Json.dll</HintPath>
     </Reference>
@@ -50,10 +79,15 @@ under the License.
     </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
+    <Reference Include="System.Data" />
     <Reference Include="System.IO.Compression" />
     <Reference Include="System.IO.Compression.FileSystem" />
     <Reference Include="System.Net.Http" />
     <Reference Include="System.Net.Http.WebRequest" />
+    <Reference Include="System.Spatial, Version=5.8.1.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>..\packages\System.Spatial.5.8.2\lib\net40\System.Spatial.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="System.Xml" />
     <Reference Include="System.Xml.Linq" />
     <Reference Include="System.Runtime.Serialization" />
@@ -83,11 +117,34 @@ under the License.
     <Compile Include="API\TcpPortConfigurationModule.cs" />
     <Compile Include="Avro\AvroAppSubmissionParameters.cs" />
     <Compile Include="Avro\AvroJobSubmissionParameters.cs" />
+    <Compile Include="Avro\AzureBatch\AvroAzureBatchJobSubmissionParameters.cs" />
     <Compile Include="Avro\Local\AvroLocalAppSubmissionParameters.cs" />
     <Compile Include="Avro\Local\AvroLocalJobSubmissionParameters.cs" />
     <Compile Include="Avro\YARN\AvroYarnAppSubmissionParameters.cs" />
     <Compile Include="Avro\YARN\AvroYarnJobSubmissionParameters.cs" />
     <Compile Include="Avro\YARN\AvroYarnClusterJobSubmissionParameters.cs" />
+    <Compile Include="AzureBatch\AzureBatchRuntimeClientConfiguration.cs" />
+    <Compile Include="AzureBatch\Util\AzureBatchCommandProviderConfiguration.cs" />
+    <Compile Include="AzureBatch\AzureBatchDotNetClient.cs" />
+    <Compile Include="AzureBatch\Storage\AzureStorageClient.cs" />
+    <Compile Include="AzureBatch\Service\AzureBatchService.cs" />
+    <Compile Include="AzureBatch\Util\AzureBatchFileNames.cs" />
+    <Compile Include="AzureBatch\Util\ICommandBuilder.cs" />
+    <Compile Include="AzureBatch\Parameters\AzureBatchAccountKey.cs" />
+    <Compile Include="AzureBatch\Parameters\AzureBatchAccountName.cs" />
+    <Compile Include="AzureBatch\Parameters\AzureBatchAccountUri.cs" />
+    <Compile Include="AzureBatch\Parameters\AzureBatchPoolId.cs" />
+    <Compile Include="AzureBatch\Parameters\AzureStorageAccountKey.cs" />
+    <Compile Include="AzureBatch\Parameters\AzureStorageAccountName.cs" />
+    <Compile Include="AzureBatch\Parameters\AzureStorageContainerName.cs" />
+    <Compile Include="AzureBatch\Parameters\DriverStderrFilePath.cs" />
+    <Compile Include="AzureBatch\Parameters\DriverStdoutFilePath.cs" />
+    <Compile Include="AzureBatch\Parameters\EnableDebugLogging.cs" />
+    <Compile Include="AzureBatch\Parameters\IsWindows.cs" />
+    <Compile Include="AzureBatch\Util\AbstractCommandBuilder.cs" />
+    <Compile Include="AzureBatch\Util\JobJarMaker.cs" />
+    <Compile Include="AzureBatch\Util\LinuxCommandBuilder.cs" />
+    <Compile Include="AzureBatch\Util\WindowsCommandBuilder.cs" />
     <Compile Include="Common\DotNetFile.cs" />
     <Compile Include="Common\ClientConstants.cs" />
     <Compile Include="Common\DriverFolderPreparationHelper.cs" />
@@ -260,6 +317,7 @@ under the License.
       <Visible>false</Visible>
     </EmbeddedResource>
   </ItemGroup>
+  <ItemGroup />
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
   <Import Project="$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets" Condition="Exists('$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets')" />
@@ -371,4 +429,4 @@ under the License.
   </Target>
   <Target Name="BeforeBuild" DependsOnTargets="$(BeforeBuildDependsOn);RewriteClientResources">
   </Target>
-</Project>
+</Project>
\ No newline at end of file