You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by mo...@apache.org on 2018/03/28 19:05:21 UTC

[4/4] reef git commit: [REEF-1965] Implement REEF runtime for Azure Batch

[REEF-1965] Implement REEF runtime for Azure Batch

    * Implement REEF runtime for Azure Batch
    * Add HelloReefAzBatch example that allows users to test the runtime
    with their Batch account
    * Ensure all tests in reef-tests package pass when run with Azure
    Batch runtime.
    * Azure Batch runtime security improvements and minor refactor

       1. Driver now uses authentication token to communicate with Azure Batch
       2. Driver now uses a Shared Access Signature to communicate with Azure Storage
       3. org.apache.reef.runtime.azbatch.util package was refactored into sub-packages to better organize a growing number of classes.
       4. Adding "final" keyword for necessary classes
       5. Add missing annotation
    * Fix race condition in AzureBatchTaskStatusAlarmHandler

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

Closes #1432


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

Branch: refs/heads/master
Commit: 561a336f2f0dda8f4a67a96179750a76167b038f
Parents: 7924188
Author: Reed Umbrasas <ry...@microsoft.com>
Authored: Wed Feb 28 17:29:44 2018 -0800
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Wed Mar 28 12:04:55 2018 -0700

----------------------------------------------------------------------
 bin/runazuretests.sh                            |  52 +++
 lang/common/proto/evaluator_shim.proto          |  46 ++
 lang/common/proto/reef_protocol.proto           |   6 +
 lang/java/reef-common/pom.xml                   |   1 +
 .../reef/runtime/common/files/JobJarMaker.java  | 115 +++--
 .../runtime/common/launch/REEFMessageCodec.java |   9 +
 lang/java/reef-examples/pom.xml                 |   5 +
 .../reef/examples/hello/HelloReefAzBatch.java   |  93 ++++
 lang/java/reef-runtime-azbatch/README.md        |  11 +
 lang/java/reef-runtime-azbatch/pom.xml          | 120 +++++
 .../azbatch/AzureBatchClasspathProvider.java    |  62 +++
 .../azbatch/AzureBatchJVMPathProvider.java      |  45 ++
 ...ureBatchDriverConfigurationProviderImpl.java | 100 ++++
 .../client/AzureBatchJobSubmissionHandler.java  | 155 +++++++
 .../client/AzureBatchRuntimeConfiguration.java  | 118 +++++
 .../AzureBatchRuntimeConfigurationCreator.java  |  84 ++++
 .../AzureBatchRuntimeConfigurationProvider.java |  78 ++++
 .../AzureBatchRuntimeConfigurationStatic.java   |  53 +++
 .../runtime/azbatch/client/package-info.java    |  22 +
 .../driver/AzureBatchDriverConfiguration.java   | 129 ++++++
 ...BatchEvaluatorShimConfigurationProvider.java |  54 +++
 .../driver/AzureBatchEvaluatorShimManager.java  | 452 +++++++++++++++++++
 .../driver/AzureBatchResourceLaunchHandler.java |  55 +++
 .../driver/AzureBatchResourceManager.java       | 136 ++++++
 .../AzureBatchResourceManagerStartHandler.java  |  47 ++
 .../AzureBatchResourceManagerStopHandler.java   |  51 +++
 .../AzureBatchResourceReleaseHandler.java       |  51 +++
 .../AzureBatchResourceRequestHandler.java       |  52 +++
 .../AzureBatchTaskStatusAlarmHandler.java       | 152 +++++++
 .../azbatch/driver/REEFEventHandlers.java       |  93 ++++
 .../azbatch/driver/RuntimeIdentifier.java       |  36 ++
 .../runtime/azbatch/driver/package-info.java    |  22 +
 .../azbatch/evaluator/EvaluatorShim.java        | 303 +++++++++++++
 .../evaluator/EvaluatorShimConfiguration.java   |  53 +++
 .../evaluator/EvaluatorShimLauncher.java        | 109 +++++
 .../runtime/azbatch/evaluator/package-info.java |  22 +
 .../reef/runtime/azbatch/package-info.java      |  22 +
 .../parameters/AzureBatchAccountKey.java        |  29 ++
 .../parameters/AzureBatchAccountName.java       |  29 ++
 .../parameters/AzureBatchAccountUri.java        |  29 ++
 .../azbatch/parameters/AzureBatchPoolId.java    |  29 ++
 .../AzureBatchTaskStatusCheckPeriod.java        |  29 ++
 .../parameters/AzureStorageAccountKey.java      |  29 ++
 .../parameters/AzureStorageAccountName.java     |  29 ++
 .../AzureStorageBlobSASTokenValidityHours.java  |  29 ++
 .../parameters/AzureStorageContainerName.java   |  29 ++
 .../azbatch/parameters/ContainerIdentifier.java |  29 ++
 .../parameters/EvaluatorShimConfigFilePath.java |  29 ++
 .../runtime/azbatch/parameters/IsWindows.java   |  29 ++
 .../azbatch/parameters/package-info.java        |  22 +
 .../azbatch/util/AzureBatchFileNames.java       |  96 ++++
 .../azbatch/util/RemoteIdentifierParser.java    |  59 +++
 .../runtime/azbatch/util/TaskStatusMapper.java  |  57 +++
 .../azbatch/util/batch/AzureBatchHelper.java    | 172 +++++++
 .../batch/IAzureBatchCredentialProvider.java    |  33 ++
 .../batch/SharedKeyBatchCredentialProvider.java |  59 +++
 .../batch/TokenBatchCredentialProvider.java     |  69 +++
 .../azbatch/util/batch/package-info.java        |  22 +
 .../util/command/AbstractCommandBuilder.java    | 120 +++++
 .../azbatch/util/command/CommandBuilder.java    |  56 +++
 .../util/command/LinuxCommandBuilder.java       |  68 +++
 .../util/command/WindowsCommandBuilder.java     |  74 +++
 .../azbatch/util/command/package-info.java      |  22 +
 .../reef/runtime/azbatch/util/package-info.java |  22 +
 .../util/storage/AzureStorageClient.java        | 142 ++++++
 .../util/storage/ICloudBlobClientProvider.java  |  51 +++
 ...dAccessSignatureCloudBlobClientProvider.java | 110 +++++
 .../storage/StorageKeyCloudBlobProvider.java    |  92 ++++
 .../azbatch/util/storage/package-info.java      |  22 +
 .../runtime/azbatch/CommandBuilderTests.java    | 118 +++++
 .../reef/runtime/azbatch/package-info.java      |  22 +
 lang/java/reef-tests/pom.xml                    |   5 +
 .../reef/tests/AzureBatchTestEnvironment.java   |  73 +++
 .../reef/tests/TestEnvironmentFactory.java      |   6 +-
 pom.xml                                         |   8 +
 75 files changed, 4932 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/bin/runazuretests.sh
----------------------------------------------------------------------
diff --git a/bin/runazuretests.sh b/bin/runazuretests.sh
new file mode 100644
index 0000000..8553bef
--- /dev/null
+++ b/bin/runazuretests.sh
@@ -0,0 +1,52 @@
+#!/bin/sh
+#
+# 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.
+#
+
+# EXAMPLE USAGE 
+# . ./bin/runazuretests.sh 
+# "./lang/java/reef-examples/target/reef-examples-0.17.0-SNAPSHOT-shaded.jar;
+# ./lang/java/reef-tests/target/reef-tests-0.17.0-SNAPSHOT-test-jar-with-dependencies.jar" 
+# org.apache.reef.tests.examples.TestHelloREEF
+
+
+# RUNTIME
+
+if [ $# -ne 2 ];
+then 
+    echo "Only 2 arguments are accepted - CLASSPATH and TESTCLASS"
+    exit 1;
+fi
+
+[ -z "$VARIABLE" ] && VARIABLE="REEF_TEST_AZBATCH"
+
+if [ $VARIABLE != "true" ]
+then
+    echo "Trying to set REEF_TEST_AZBATCH environment variable."
+    echo "Please run as \". runazuretests.sh\" or set it from your environment."
+    export REEF_TEST_AZBATCH=true
+fi
+
+CLASSPATH=$1
+TESTCLASS=$2
+
+CMD="java -cp ${CLASSPATH} org.junit.runner.JUnitCore ${TESTCLASS}"
+
+echo -e "\n\nRunning Azure Batch Tests...\n\n"
+echo $CMD
+$CMD

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/common/proto/evaluator_shim.proto
----------------------------------------------------------------------
diff --git a/lang/common/proto/evaluator_shim.proto b/lang/common/proto/evaluator_shim.proto
new file mode 100644
index 0000000..3c27b13
--- /dev/null
+++ b/lang/common/proto/evaluator_shim.proto
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+option java_package = "org.apache.reef.proto";
+option java_outer_classname = "EvaluatorShimProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+enum EvaluatorShimStatus {
+    UNKNOWN = 1;
+    ONLINE  = 2;
+}
+
+enum EvaluatorShimCommand {
+    LAUNCH_EVALUATOR = 1;
+    TERMINATE        = 2;
+}
+
+message EvaluatorShimControlProto {
+    required EvaluatorShimCommand command     = 1;
+    optional string evaluatorConfigString     = 2;
+    optional string evaluatorLaunchCommand    = 3;
+    optional string evaluatorFileResourcesUrl = 4;
+}
+
+message EvaluatorShimStatusProto {
+    required EvaluatorShimStatus status = 1;
+    required string remoteIdentifier    = 2;
+    required string containerId         = 3;
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/common/proto/reef_protocol.proto
----------------------------------------------------------------------
diff --git a/lang/common/proto/reef_protocol.proto b/lang/common/proto/reef_protocol.proto
index 86675f9..1fb4489 100644
--- a/lang/common/proto/reef_protocol.proto
+++ b/lang/common/proto/reef_protocol.proto
@@ -23,6 +23,8 @@ import "evaluator_runtime.proto";
 
 import "reef_service_protos.proto";
 
+import "evaluator_shim.proto";
+
 
 option java_package = "org.apache.reef.proto";
 
@@ -43,4 +45,8 @@ message REEFMessage {
     // Messages from evaluator_runtime.proto
     optional EvaluatorControlProto evaluatorControl = 5;
     optional EvaluatorHeartbeatProto evaluatorHeartBeat = 6;
+    optional EvaluatorShimControlProto evaluatorShimCommand = 7;
+
+    // Messages from evaluator_shim.proto
+    optional EvaluatorShimStatusProto evaluatorShimStatus = 8;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-common/pom.xml
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/pom.xml b/lang/java/reef-common/pom.xml
index 94e66c3..db1e87e 100644
--- a/lang/java/reef-common/pom.xml
+++ b/lang/java/reef-common/pom.xml
@@ -74,6 +74,7 @@ under the License.
                                     <arg value="${protoPath}/reef_service_protos.proto"/>
                                     <arg value="${protoPath}/evaluator_runtime.proto"/>
                                     <arg value="${protoPath}/client_runtime.proto"/>
+                                    <arg value="${protoPath}/evaluator_shim.proto"/>
                                     <arg value="${protoPath}/reef_protocol.proto"/>
                                 </exec>
                             </target>

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/JobJarMaker.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/JobJarMaker.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/JobJarMaker.java
index 6b45f0d..9ca7c1f 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/JobJarMaker.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/files/JobJarMaker.java
@@ -18,6 +18,7 @@
  */
 package org.apache.reef.runtime.common.files;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.reef.annotations.audience.ClientSide;
 import org.apache.reef.annotations.audience.Private;
 import org.apache.reef.annotations.audience.RuntimeAuthor;
@@ -32,6 +33,8 @@ import javax.inject.Inject;
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -58,6 +61,15 @@ public final class JobJarMaker {
     this.deleteTempFilesOnExit = deleteTempFilesOnExit;
   }
 
+  /**
+   * Provider builder class for building JAR files.
+   *
+   * @return JarBuilder instance.
+   */
+  public JarBuilder newBuilder() {
+    return new JarBuilder();
+  }
+
   public static void copy(final Iterable<FileResource> files, final File destinationFolder) {
 
     if (!destinationFolder.exists() && !destinationFolder.mkdirs()) {
@@ -96,42 +108,87 @@ public final class JobJarMaker {
       final JobSubmissionEvent jobSubmissionEvent,
       final Configuration driverConfiguration) throws IOException {
 
-    // Copy all files to a local job submission folder
-    final File jobSubmissionFolder = makejobSubmissionFolder();
-    LOG.log(Level.FINE, "Staging submission in {0}", jobSubmissionFolder);
+    return new JarBuilder()
+        .withConfiguration(driverConfiguration)
+        .addGlobalFileSet(jobSubmissionEvent.getGlobalFileSet())
+        .addLocalFileSet(jobSubmissionEvent.getLocalFileSet())
+        .withConfigurationFileName(this.fileNames.getDriverConfigurationName())
+        .build();
+  }
+
+  private File makeJobSubmissionFolder() throws IOException {
+    return Files.createTempDirectory(this.fileNames.getJobFolderPrefix()).toFile();
+  }
 
-    final File localFolder = new File(jobSubmissionFolder, this.fileNames.getLocalFolderName());
-    final File globalFolder = new File(jobSubmissionFolder, this.fileNames.getGlobalFolderName());
+  /**
+   * Builder class for building JAR files.
+   */
+  public final class JarBuilder {
 
-    copy(jobSubmissionEvent.getGlobalFileSet(), globalFolder);
-    copy(jobSubmissionEvent.getLocalFileSet(), localFolder);
+    private final Set<FileResource> localFiles = new HashSet<>();
+    private final Set<FileResource> globalFiles = new HashSet<>();
+    private Configuration configuration = null;
+    private String configurationFilename = null;
 
-    // Store the Driver Configuration in the JAR file.
-    this.configurationSerializer.toFile(
-        driverConfiguration, new File(localFolder, this.fileNames.getDriverConfigurationName()));
+    private JarBuilder() {}
 
-    // Create a JAR File for the submission
-    final File jarFile = File.createTempFile(this.fileNames.getJobFolderPrefix(), this.fileNames.getJarFileSuffix());
+    public JarBuilder addLocalFileSet(final Set<FileResource> localFileResources) {
+      this.localFiles.addAll(localFileResources);
+      return this;
+    }
 
-    LOG.log(Level.FINE, "Creating job submission jar file: {0}", jarFile);
-    new JARFileMaker(jarFile).addChildren(jobSubmissionFolder).close();
+    public JarBuilder addGlobalFileSet(final Set<FileResource> globalFileResources) {
+      this.globalFiles.addAll(globalFileResources);
+      return this;
+    }
 
-    if (this.deleteTempFilesOnExit) {
-      LOG.log(Level.FINE,
-          "Deleting the temporary job folder [{0}] and marking the jar file [{1}] for deletion after the JVM exits.",
-          new Object[]{jobSubmissionFolder.getAbsolutePath(), jarFile.getAbsolutePath()});
-      if (!jobSubmissionFolder.delete()) {
-        LOG.log(Level.WARNING, "Failed to delete [{0}]", jobSubmissionFolder.getAbsolutePath());
-      }
-      jarFile.deleteOnExit();
-    } else {
-      LOG.log(Level.FINE, "Keeping the temporary job folder [{0}] and jar file [{1}] available after job submission.",
-          new Object[]{jobSubmissionFolder.getAbsolutePath(), jarFile.getAbsolutePath()});
+    public JarBuilder withConfiguration(final Configuration withConfiguration) {
+      this.configuration = withConfiguration;
+      return this;
     }
-    return jarFile;
-  }
 
-  private File makejobSubmissionFolder() throws IOException {
-    return Files.createTempDirectory(this.fileNames.getJobFolderPrefix()).toFile();
+    public JarBuilder withConfigurationFileName(final String withConfigurationFilename) {
+      this.configurationFilename = withConfigurationFilename;
+      return this;
+    }
+
+    public File build() throws IOException {
+      // Copy all files to a local job submission folder
+      final File jobSubmissionFolder = makeJobSubmissionFolder();
+      LOG.log(Level.FINE, "Staging submission in {0}", jobSubmissionFolder);
+
+      final File localFolder = new File(jobSubmissionFolder, JobJarMaker.this.fileNames.getLocalFolderName());
+      final File globalFolder = new File(jobSubmissionFolder, JobJarMaker.this.fileNames.getGlobalFolderName());
+
+      JobJarMaker.copy(this.globalFiles, globalFolder);
+      JobJarMaker.copy(this.localFiles, localFolder);
+
+      // Store the Configuration in the JAR file.
+      if (configuration != null && StringUtils.isNotBlank(this.configurationFilename)) {
+        JobJarMaker.this.configurationSerializer
+            .toFile(configuration, new File(localFolder, this.configurationFilename));
+      }
+
+      // Create a JAR File for the submission
+      final File jarFile = File.createTempFile(JobJarMaker.this.fileNames.getJobFolderPrefix(),
+          JobJarMaker.this.fileNames.getJarFileSuffix());
+
+      LOG.log(Level.FINE, "Creating job submission jar file: {0}", jarFile);
+      new JARFileMaker(jarFile).addChildren(jobSubmissionFolder).close();
+
+      if (JobJarMaker.this.deleteTempFilesOnExit) {
+        LOG.log(Level.FINE,
+            "Deleting the temporary job folder [{0}] and marking the jar file [{1}] for deletion after the JVM exits.",
+            new Object[]{jobSubmissionFolder.getAbsolutePath(), jarFile.getAbsolutePath()});
+        if (!jobSubmissionFolder.delete()) {
+          LOG.log(Level.WARNING, "Failed to delete [{0}]", jobSubmissionFolder.getAbsolutePath());
+        }
+        jarFile.deleteOnExit();
+      } else {
+        LOG.log(Level.FINE, "Keeping the temporary job folder [{0}] and jar file [{1}] available after job submission.",
+            new Object[]{jobSubmissionFolder.getAbsolutePath(), jarFile.getAbsolutePath()});
+      }
+      return jarFile;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/REEFMessageCodec.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/REEFMessageCodec.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/REEFMessageCodec.java
index 4c48e0a..ef2235d 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/REEFMessageCodec.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/REEFMessageCodec.java
@@ -27,6 +27,7 @@ import org.apache.reef.annotations.audience.EvaluatorSide;
 import org.apache.reef.annotations.audience.Private;
 import org.apache.reef.proto.ClientRuntimeProtocol;
 import org.apache.reef.proto.EvaluatorRuntimeProtocol;
+import org.apache.reef.proto.EvaluatorShimProtocol;
 import org.apache.reef.proto.REEFProtocol;
 import org.apache.reef.proto.ReefServiceProtos;
 import org.apache.reef.wake.remote.Codec;
@@ -62,6 +63,10 @@ public final class REEFMessageCodec implements Codec<GeneratedMessage> {
         return message.getEvaluatorControl();
       } else if (message.hasEvaluatorHeartBeat()) {
         return message.getEvaluatorHeartBeat();
+      } else if (message.hasEvaluatorShimCommand()) {
+        return message.getEvaluatorShimCommand();
+      } else if (message.hasEvaluatorShimStatus()) {
+        return message.getEvaluatorShimStatus();
       }
       throw new RuntimeException("Unable to decode a message: " + message.toString());
     } catch (final InvalidProtocolBufferException e) {
@@ -83,6 +88,10 @@ public final class REEFMessageCodec implements Codec<GeneratedMessage> {
       message.setEvaluatorControl((EvaluatorRuntimeProtocol.EvaluatorControlProto) msg);
     } else if (msg instanceof EvaluatorRuntimeProtocol.EvaluatorHeartbeatProto) {
       message.setEvaluatorHeartBeat((EvaluatorRuntimeProtocol.EvaluatorHeartbeatProto) msg);
+    } else if (msg instanceof EvaluatorShimProtocol.EvaluatorShimControlProto) {
+      message.setEvaluatorShimCommand((EvaluatorShimProtocol.EvaluatorShimControlProto) msg);
+    } else if (msg instanceof EvaluatorShimProtocol.EvaluatorShimStatusProto) {
+      message.setEvaluatorShimStatus((EvaluatorShimProtocol.EvaluatorShimStatusProto) msg);
     } else {
       throw new RuntimeException("Unable to serialize: " + msg);
     }

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-examples/pom.xml
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/pom.xml b/lang/java/reef-examples/pom.xml
index fafb337..0668452 100644
--- a/lang/java/reef-examples/pom.xml
+++ b/lang/java/reef-examples/pom.xml
@@ -43,6 +43,11 @@ under the License.
         </dependency>
         <dependency>
             <groupId>${project.groupId}</groupId>
+            <artifactId>reef-runtime-azbatch</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
             <artifactId>reef-runtime-local</artifactId>
             <version>${project.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-examples/src/main/java/org/apache/reef/examples/hello/HelloReefAzBatch.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-examples/src/main/java/org/apache/reef/examples/hello/HelloReefAzBatch.java b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/hello/HelloReefAzBatch.java
new file mode 100644
index 0000000..fc6c661
--- /dev/null
+++ b/lang/java/reef-examples/src/main/java/org/apache/reef/examples/hello/HelloReefAzBatch.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.examples.hello;
+
+import org.apache.reef.client.DriverConfiguration;
+import org.apache.reef.client.REEF;
+import org.apache.reef.runtime.azbatch.client.AzureBatchRuntimeConfiguration;
+import org.apache.reef.runtime.azbatch.client.AzureBatchRuntimeConfigurationProvider;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Injector;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.util.EnvironmentUtils;
+
+import java.io.IOException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * A main() for running hello REEF in Azure Batch.
+ */
+public final class HelloReefAzBatch {
+
+  private static final Logger LOG = Logger.getLogger(HelloReefAzBatch.class.getName());
+
+  /**
+   * Builds the runtime configuration for Azure Batch.
+   *
+   * @return the configuration of the runtime.
+   * @throws IOException
+   */
+  private static Configuration getEnvironmentConfiguration() throws IOException {
+    return AzureBatchRuntimeConfiguration.fromEnvironment();
+  }
+
+  /**
+   * Builds and returns driver configuration for HelloREEF driver.
+   *
+   * @return the configuration of the HelloREEF driver.
+   */
+  private static Configuration getDriverConfiguration() {
+    return DriverConfiguration.CONF
+        .set(DriverConfiguration.DRIVER_IDENTIFIER, "HelloREEF")
+        .set(DriverConfiguration.GLOBAL_LIBRARIES, EnvironmentUtils.getClassLocation(HelloDriver.class))
+        .set(DriverConfiguration.ON_DRIVER_STARTED, HelloDriver.StartHandler.class)
+        .set(DriverConfiguration.ON_EVALUATOR_ALLOCATED, HelloDriver.EvaluatorAllocatedHandler.class)
+        .build();
+  }
+
+  /**
+   * Start the Hello REEF job with the Azure Batch runtime.
+   *
+   * @param args command line parameters.
+   * @throws InjectionException configuration error.
+   * @throws IOException
+   */
+  public static void main(final String[] args) throws InjectionException, IOException {
+
+    final Configuration partialConfiguration = getEnvironmentConfiguration();
+    final Injector injector = Tang.Factory.getTang().newInjector(partialConfiguration);
+    final AzureBatchRuntimeConfigurationProvider runtimeConfigurationProvider =
+        injector.getInstance(AzureBatchRuntimeConfigurationProvider.class);
+    final Configuration driverConfiguration = getDriverConfiguration();
+
+    try (final REEF reef = Tang.Factory.getTang().newInjector(
+        runtimeConfigurationProvider.getAzureBatchRuntimeConfiguration()).getInstance(REEF.class)) {
+      reef.submit(driverConfiguration);
+    }
+    LOG.log(Level.INFO, "Job Submitted");
+  }
+
+  /**
+   * Private constructor.
+   */
+  private HelloReefAzBatch() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/README.md
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/README.md b/lang/java/reef-runtime-azbatch/README.md
new file mode 100644
index 0000000..db40852
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/README.md
@@ -0,0 +1,11 @@
+REEF Runtime for Azure Batch
+============================
+
+This folder containes support for running Apache [REEF] applications on Azure Batch. It
+uses the Azure Batch APIs defined [here][azbatchapi] via their [Java
+implementation][azbatchjava].
+
+
+[azbatchapi]: https://docs.microsoft.com/en-us/rest/api/batchservice/
+[azbatchjava]: https://docs.microsoft.com/en-us/java/api/com.microsoft.azure.batch
+[REEF]: https://reef.apache.org
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/pom.xml
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/pom.xml b/lang/java/reef-runtime-azbatch/pom.xml
new file mode 100644
index 0000000..0280e8f
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/pom.xml
@@ -0,0 +1,120 @@
+<?xml version="1.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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.reef</groupId>
+        <artifactId>reef-project</artifactId>
+        <version>0.17.0-SNAPSHOT</version>
+        <relativePath>../../..</relativePath>
+    </parent>
+
+    <properties>
+        <rootPath>${basedir}/../../..</rootPath>
+    </properties>
+
+    <artifactId>reef-runtime-azbatch</artifactId>
+    <name>REEF Runtime for Azure Batch</name>
+    <description>An implementation of REEF to run on Azure Batch.</description>
+
+    <dependencies>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>reef-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.windowsazure.storage</groupId>
+            <artifactId>microsoft-windowsazure-storage-sdk</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- Microsoft Azure Batch APIs -->
+        <dependency>
+            <groupId>com.microsoft.azure</groupId>
+            <artifactId>azure-batch</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+            <version>2.7.2</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.7.2</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+            <version>2.7.2</version>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>20.0</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+            <version>1.10</version>
+        </dependency>
+        <!-- End of Microsoft Azure Batch APIs -->
+    </dependencies>
+
+    <build>
+        <resources>
+            <resource>
+                <targetPath>META-INF/</targetPath>
+                <filtering>false</filtering>
+                <directory>${basedir}/conf</directory>
+                <includes>
+                    <include>*.xml</include>
+                    <include>*.properties</include>
+                </includes>
+                <excludes>
+                </excludes>
+            </resource>
+        </resources>
+        <pluginManagement>
+            <plugins>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-checkstyle-plugin</artifactId>
+                    <configuration>
+                        <configLocation>lang/java/reef-common/src/main/resources/checkstyle-strict.xml</configLocation>
+                    </configuration>
+                </plugin>
+            </plugins>
+        </pluginManagement>
+    </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/AzureBatchClasspathProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/AzureBatchClasspathProvider.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/AzureBatchClasspathProvider.java
new file mode 100644
index 0000000..bf37fe0
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/AzureBatchClasspathProvider.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.runtime.common.files.RuntimeClasspathProvider;
+
+import javax.inject.Inject;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Access to the classpath according to the REEF file system standard.
+ */
+@Private
+public final class AzureBatchClasspathProvider implements RuntimeClasspathProvider {
+
+  private final List<String> classPathPrefix;
+  private final List<String> classPathSuffix;
+
+  @Inject
+  private AzureBatchClasspathProvider() {
+    this.classPathPrefix = new ArrayList<>();
+    this.classPathSuffix = new ArrayList<>();
+  }
+
+  @Override
+  public List<String> getDriverClasspathPrefix() {
+    return this.classPathPrefix;
+  }
+
+  @Override
+  public List<String> getDriverClasspathSuffix() {
+    return this.classPathSuffix;
+  }
+
+  @Override
+  public List<String> getEvaluatorClasspathPrefix() {
+    return this.classPathPrefix;
+  }
+
+  @Override
+  public List<String> getEvaluatorClasspathSuffix() {
+    return this.classPathSuffix;
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/AzureBatchJVMPathProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/AzureBatchJVMPathProvider.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/AzureBatchJVMPathProvider.java
new file mode 100644
index 0000000..f78f62c
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/AzureBatchJVMPathProvider.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.runtime.common.files.RuntimePathProvider;
+
+import javax.inject.Inject;
+
+/**
+ * Supplies the java binary's path for Azure Batch.
+ */
+@Private
+public final class AzureBatchJVMPathProvider implements RuntimePathProvider {
+
+  @Inject
+  public AzureBatchJVMPathProvider() {
+  }
+
+  @Override
+  public String getPath() {
+    return "java";
+  }
+
+  @Override
+  public String toString() {
+    return getPath();
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchDriverConfigurationProviderImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchDriverConfigurationProviderImpl.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchDriverConfigurationProviderImpl.java
new file mode 100644
index 0000000..0887e9d
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchDriverConfigurationProviderImpl.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.runtime.azbatch.driver.AzureBatchDriverConfiguration;
+import org.apache.reef.runtime.azbatch.driver.RuntimeIdentifier;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchAccountName;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchPoolId;
+import org.apache.reef.runtime.azbatch.util.command.CommandBuilder;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchAccountUri;
+import org.apache.reef.runtime.azbatch.parameters.AzureStorageAccountName;
+import org.apache.reef.runtime.azbatch.parameters.AzureStorageContainerName;
+import org.apache.reef.runtime.common.client.DriverConfigurationProvider;
+import org.apache.reef.runtime.common.parameters.JVMHeapSlack;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.Configurations;
+import org.apache.reef.tang.annotations.Parameter;
+
+import javax.inject.Inject;
+import java.net.URI;
+
+/**
+ * Configuration provider for the Azure Batch runtime.
+ */
+@Private
+public final class AzureBatchDriverConfigurationProviderImpl implements DriverConfigurationProvider {
+
+  private final double jvmSlack;
+  private final String azureBatchAccountUri;
+  private final String azureBatchAccountName;
+  private final String azureBatchPoolId;
+  private final String azureStorageAccountName;
+  private final String azureStorageContainerName;
+  private final CommandBuilder commandBuilder;
+
+  @Inject
+  private AzureBatchDriverConfigurationProviderImpl(
+      @Parameter(JVMHeapSlack.class) final double jvmSlack,
+      @Parameter(AzureBatchAccountUri.class) final String azureBatchAccountUri,
+      @Parameter(AzureBatchAccountName.class) final String azureBatchAccountName,
+      @Parameter(AzureBatchPoolId.class) final String azureBatchPoolId,
+      @Parameter(AzureStorageAccountName.class) final String azureStorageAccountName,
+      @Parameter(AzureStorageContainerName.class) final String azureStorageContainerName,
+      final CommandBuilder commandBuilder) {
+    this.jvmSlack = jvmSlack;
+    this.azureBatchAccountUri = azureBatchAccountUri;
+    this.azureBatchAccountName = azureBatchAccountName;
+    this.azureBatchPoolId = azureBatchPoolId;
+    this.azureStorageAccountName = azureStorageAccountName;
+    this.azureStorageContainerName = azureStorageContainerName;
+    this.commandBuilder = commandBuilder;
+  }
+
+  /**
+   * Assembles the Driver configuration.
+   *
+   * @param jobFolder the job folder.
+   * @param clientRemoteId the client remote id.
+   * @param jobId the job id.
+   * @param applicationConfiguration the application configuration.
+   * @return the Driver configuration.
+   */
+  @Override
+  public Configuration getDriverConfiguration(final URI jobFolder,
+                                              final String clientRemoteId,
+                                              final String jobId,
+                                              final Configuration applicationConfiguration) {
+    return Configurations.merge(
+        AzureBatchDriverConfiguration.CONF.getBuilder()
+            .bindImplementation(CommandBuilder.class, this.commandBuilder.getClass()).build()
+            .set(AzureBatchDriverConfiguration.JOB_IDENTIFIER, jobId)
+            .set(AzureBatchDriverConfiguration.CLIENT_REMOTE_IDENTIFIER, clientRemoteId)
+            .set(AzureBatchDriverConfiguration.JVM_HEAP_SLACK, this.jvmSlack)
+            .set(AzureBatchDriverConfiguration.RUNTIME_NAME, RuntimeIdentifier.RUNTIME_NAME)
+            .set(AzureBatchDriverConfiguration.AZURE_BATCH_ACCOUNT_URI, this.azureBatchAccountUri)
+            .set(AzureBatchDriverConfiguration.AZURE_BATCH_ACCOUNT_NAME, this.azureBatchAccountName)
+            .set(AzureBatchDriverConfiguration.AZURE_BATCH_POOL_ID, this.azureBatchPoolId)
+            .set(AzureBatchDriverConfiguration.AZURE_STORAGE_ACCOUNT_NAME, this.azureStorageAccountName)
+            .set(AzureBatchDriverConfiguration.AZURE_STORAGE_CONTAINER_NAME, this.azureStorageContainerName)
+            .build(),
+        applicationConfiguration);
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchJobSubmissionHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchJobSubmissionHandler.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchJobSubmissionHandler.java
new file mode 100644
index 0000000..cbb8a8c
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchJobSubmissionHandler.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.runtime.azbatch.util.AzureBatchFileNames;
+import org.apache.reef.runtime.azbatch.util.batch.AzureBatchHelper;
+import org.apache.reef.runtime.azbatch.util.storage.AzureStorageClient;
+import org.apache.reef.runtime.azbatch.util.command.CommandBuilder;
+import org.apache.reef.runtime.common.client.DriverConfigurationProvider;
+import org.apache.reef.runtime.common.client.api.JobSubmissionEvent;
+import org.apache.reef.runtime.common.client.api.JobSubmissionHandler;
+import org.apache.reef.runtime.common.files.JobJarMaker;
+import org.apache.reef.tang.Configuration;
+
+import javax.inject.Inject;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.*;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * A {@link JobSubmissionHandler} implementation for Azure Batch runtime.
+ */
+@Private
+public final class AzureBatchJobSubmissionHandler implements JobSubmissionHandler {
+
+  private static final Logger LOG = Logger.getLogger(AzureBatchJobSubmissionHandler.class.getName());
+
+  /**
+   * Maximum number of characters allowed in Azure Batch job name. This limit is imposed by Azure Batch.
+   */
+  private static final int MAX_CHARS_JOB_NAME = 64;
+
+  private String applicationId;
+
+  private final AzureStorageClient azureStorageClient;
+  private final DriverConfigurationProvider driverConfigurationProvider;
+  private final JobJarMaker jobJarMaker;
+  private final CommandBuilder launchCommandBuilder;
+  private final AzureBatchFileNames azureBatchFileNames;
+  private final AzureBatchHelper azureBatchHelper;
+
+  @Inject
+  AzureBatchJobSubmissionHandler(
+      final AzureStorageClient azureStorageClient,
+      final DriverConfigurationProvider driverConfigurationProvider,
+      final JobJarMaker jobJarMaker,
+      final CommandBuilder launchCommandBuilder,
+      final AzureBatchFileNames azureBatchFileNames,
+      final AzureBatchHelper azureBatchHelper) {
+    this.azureStorageClient = azureStorageClient;
+    this.driverConfigurationProvider = driverConfigurationProvider;
+    this.jobJarMaker = jobJarMaker;
+    this.launchCommandBuilder = launchCommandBuilder;
+    this.azureBatchHelper = azureBatchHelper;
+    this.azureBatchFileNames = azureBatchFileNames;
+  }
+
+  /**
+   * Returns REEF application id (which corresponds to Azure Batch job id) or null if the application hasn't been
+   * submitted yet.
+   *
+   * @return REEF application id.
+   */
+  @Override
+  public String getApplicationId() {
+    return this.applicationId;
+  }
+
+  /**
+   * Closes the resources.
+   *
+   * @throws Exception
+   */
+  @Override
+  public void close() throws Exception {
+    LOG.log(Level.INFO, "Closing " + AzureBatchJobSubmissionHandler.class.getName());
+  }
+
+  /**
+   * Invoked when JobSubmissionEvent is triggered.
+   *
+   * @param jobSubmissionEvent triggered job submission event.
+   */
+  @Override
+  public void onNext(final JobSubmissionEvent jobSubmissionEvent) {
+    LOG.log(Level.FINEST, "Submitting job: {0}", jobSubmissionEvent);
+
+    try {
+      this.applicationId = createApplicationId(jobSubmissionEvent);
+      final String folderName = this.azureBatchFileNames.getStorageJobFolder(this.applicationId);
+
+      LOG.log(Level.FINE, "Creating a job folder on Azure at: {0}.", folderName);
+      final URI jobFolderURL = this.azureStorageClient.getJobSubmissionFolderUri(folderName);
+
+      LOG.log(Level.FINE, "Getting a shared access signature for {0}.", folderName);
+      final String storageContainerSAS = this.azureStorageClient.createContainerSharedAccessSignature();
+
+      LOG.log(Level.FINE, "Assembling Configuration for the Driver.");
+      final Configuration driverConfiguration = makeDriverConfiguration(jobSubmissionEvent, this.applicationId,
+          jobFolderURL);
+
+      LOG.log(Level.FINE, "Making Job JAR.");
+      final File jobSubmissionJarFile =
+          this.jobJarMaker.createJobSubmissionJAR(jobSubmissionEvent, driverConfiguration);
+
+      LOG.log(Level.FINE, "Uploading Job JAR to Azure.");
+      final URI jobJarSasUri = this.azureStorageClient.uploadFile(folderName, jobSubmissionJarFile);
+
+      LOG.log(Level.FINE, "Assembling application submission.");
+      final String command = this.launchCommandBuilder.buildDriverCommand(jobSubmissionEvent);
+
+      this.azureBatchHelper.submitJob(getApplicationId(), storageContainerSAS, jobJarSasUri, command);
+
+    } catch (final IOException e) {
+      LOG.log(Level.SEVERE, "Error submitting Azure Batch request: {0}", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private Configuration makeDriverConfiguration(
+      final JobSubmissionEvent jobSubmissionEvent,
+      final String appId,
+      final URI jobFolderURL) {
+    return this.driverConfigurationProvider.getDriverConfiguration(
+        jobFolderURL, jobSubmissionEvent.getRemoteId(), appId, jobSubmissionEvent.getConfiguration());
+  }
+
+  private String createApplicationId(final JobSubmissionEvent jobSubmissionEvent) {
+    String uuid = UUID.randomUUID().toString();
+    String jobIdentifier  = jobSubmissionEvent.getIdentifier();
+    String jobNameShort = jobIdentifier.length() + 1 + uuid.length() < MAX_CHARS_JOB_NAME ?
+        jobIdentifier : jobIdentifier.substring(0, MAX_CHARS_JOB_NAME - uuid.length() - 1);
+    return jobNameShort + "-" + uuid;
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfiguration.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfiguration.java
new file mode 100644
index 0000000..541f8c4
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfiguration.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch.client;
+
+import org.apache.reef.annotations.audience.Public;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.formats.AvroConfigurationSerializer;
+
+import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
+import org.apache.reef.tang.formats.RequiredParameter;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Configuration Module for the Azure Batch runtime.
+ */
+@Public
+public final class AzureBatchRuntimeConfiguration extends ConfigurationModuleBuilder {
+
+  /**
+   * The Azure Batch account URI.
+   */
+  public static final RequiredParameter<String> AZURE_BATCH_ACCOUNT_URI = new RequiredParameter<>();
+
+  /**
+   * The Azure Batch account name.
+   */
+  public static final RequiredParameter<String> AZURE_BATCH_ACCOUNT_NAME = new RequiredParameter<>();
+
+  /**
+   * The Azure Batch account key.
+   */
+  public static final RequiredParameter<String> AZURE_BATCH_ACCOUNT_KEY = new RequiredParameter<>();
+
+  /**
+   * The Azure Batch pool ID.
+   */
+  public static final RequiredParameter<String> AZURE_BATCH_POOL_ID = new RequiredParameter<>();
+
+  /**
+   * The environment variable that holds the path to the default configuration file.
+   */
+  public static final String AZBATCH_CONFIGURATION_FILE_ENVIRONMENT_VARIABLE = "REEF_AZBATCH_CONF";
+
+  /**
+   * The Azure Storage account name.
+   */
+  public static final RequiredParameter<String> AZURE_STORAGE_ACCOUNT_NAME = new RequiredParameter<>();
+
+  /**
+   * The Azure Storage account key.
+   */
+  public static final RequiredParameter<String> AZURE_STORAGE_ACCOUNT_KEY = new RequiredParameter<>();
+
+  /**
+   * The Azure Storage container name.
+   */
+  public static final RequiredParameter<String> AZURE_STORAGE_CONTAINER_NAME = new RequiredParameter<>();
+
+  /**
+   * Create a {@link Configuration} object from an Avro configuration file.
+   *
+   * @param file the configuration file.
+   * @return the configuration object.
+   * @throws IOException if the file can't be read
+   */
+  public static Configuration fromTextFile(final File file) throws IOException {
+    return new AvroConfigurationSerializer().fromTextFile(file);
+  }
+
+  /**
+   * Create a {@link Configuration} object from the
+   * {@link AZBATCH_CONFIGURATION_FILE_ENVIRONMENT_VARIABLE} environment variable.
+   *
+   * @return the configuration object.
+   * @throws IOException
+   * @see AZBATCH_CONFIGURATION_FILE_ENVIRONMENT_VARIABLE
+   */
+  public static Configuration fromEnvironment() throws IOException {
+
+    final String configurationPath =
+        System.getenv(AZBATCH_CONFIGURATION_FILE_ENVIRONMENT_VARIABLE);
+
+    if (null == configurationPath) {
+      throw new IOException("Environment Variable " +
+          AZBATCH_CONFIGURATION_FILE_ENVIRONMENT_VARIABLE +
+          " not set.");
+    }
+
+    final File configurationFile = new File(configurationPath);
+    if (!configurationFile.canRead()) {
+      throw new IOException("Environment Variable " +
+          AZBATCH_CONFIGURATION_FILE_ENVIRONMENT_VARIABLE +
+          " points to a file " + configurationFile.getAbsolutePath() +
+          " which can't be read."
+      );
+    }
+
+    return fromTextFile(configurationFile);
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationCreator.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationCreator.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationCreator.java
new file mode 100644
index 0000000..0d8860b
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationCreator.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchAccountKey;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchAccountName;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchAccountUri;
+import org.apache.reef.runtime.azbatch.parameters.AzureBatchPoolId;
+import org.apache.reef.runtime.azbatch.util.command.CommandBuilder;
+import org.apache.reef.runtime.azbatch.util.command.LinuxCommandBuilder;
+import org.apache.reef.runtime.azbatch.util.command.WindowsCommandBuilder;
+import org.apache.reef.runtime.azbatch.parameters.AzureStorageAccountKey;
+import org.apache.reef.runtime.azbatch.parameters.AzureStorageAccountName;
+import org.apache.reef.runtime.azbatch.parameters.AzureStorageContainerName;
+import org.apache.reef.tang.formats.ConfigurationModule;
+import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
+
+/**
+ * Class that builds the ConfigurationModule for Azure Batch runtime.
+ */
+@Private
+public final class AzureBatchRuntimeConfigurationCreator {
+
+  /**
+   * The ConfigurationModule for Azure Batch.
+   */
+  private static ConfigurationModule conf;
+
+  /**
+   * Get or create a {@link ConfigurationModule} for the Azure Batch runtime.
+   *
+   * @param isWindows true if Azure Batch pool nodes run Windows, false otherwise.
+   * @return the configuration module object.
+   */
+  public static ConfigurationModule getOrCreateAzureBatchRuntimeConfiguration(final boolean isWindows) {
+
+    if (AzureBatchRuntimeConfigurationCreator.conf == null) {
+      ConfigurationModuleBuilder builder = AzureBatchRuntimeConfigurationStatic.CONF;
+      ConfigurationModule module;
+      if (isWindows) {
+        module = builder.bindImplementation(CommandBuilder.class, WindowsCommandBuilder.class).build();
+      } else {
+        module = builder.bindImplementation(CommandBuilder.class, LinuxCommandBuilder.class).build();
+      }
+
+      AzureBatchRuntimeConfigurationCreator.conf = new AzureBatchRuntimeConfiguration()
+          .merge(module)
+          .bindNamedParameter(AzureBatchAccountName.class, AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_NAME)
+          .bindNamedParameter(AzureBatchAccountUri.class, AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_URI)
+          .bindNamedParameter(AzureBatchAccountKey.class, AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_KEY)
+          .bindNamedParameter(AzureBatchPoolId.class, AzureBatchRuntimeConfiguration.AZURE_BATCH_POOL_ID)
+          .bindNamedParameter(AzureStorageAccountName.class, AzureBatchRuntimeConfiguration.AZURE_STORAGE_ACCOUNT_NAME)
+          .bindNamedParameter(AzureStorageAccountKey.class, AzureBatchRuntimeConfiguration.AZURE_STORAGE_ACCOUNT_KEY)
+          .bindNamedParameter(
+              AzureStorageContainerName.class, AzureBatchRuntimeConfiguration.AZURE_STORAGE_CONTAINER_NAME)
+          .build();
+    }
+
+    return AzureBatchRuntimeConfigurationCreator.conf;
+  }
+
+  /*
+   * Private constructor since this is a utility class.
+   */
+  private AzureBatchRuntimeConfigurationCreator() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationProvider.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationProvider.java
new file mode 100644
index 0000000..c78e9e3
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationProvider.java
@@ -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.
+ */
+package org.apache.reef.runtime.azbatch.client;
+
+import org.apache.reef.annotations.audience.Public;
+import org.apache.reef.runtime.azbatch.parameters.*;
+import org.apache.reef.tang.Configuration;
+import org.apache.reef.tang.annotations.Parameter;
+
+import javax.inject.Inject;
+
+/**
+ * Class that provides the runtime configuration for Azure Batch.
+ */
+@Public
+public final class AzureBatchRuntimeConfigurationProvider {
+
+  private final String azureBatchAccountName;
+  private final String azureBatchAccountKey;
+  private final String azureBatchAccountUri;
+  private final String azureBatchPoolId;
+  private final String azureStorageAccountName;
+  private final String azureStorageAccountKey;
+  private final String azureStorageContainerName;
+  private final Boolean isWindows;
+
+  /**
+   * Private constructor.
+   */
+  @Inject
+  private AzureBatchRuntimeConfigurationProvider(
+      @Parameter(AzureBatchAccountName.class) final String azureBatchAccountName,
+      @Parameter(AzureBatchAccountKey.class) final String azureBatchAccountKey,
+      @Parameter(AzureBatchAccountUri.class) final String azureBatchAccountUri,
+      @Parameter(AzureBatchPoolId.class) final String azureBatchPoolId,
+      @Parameter(AzureStorageAccountName.class) final String azureStorageAccountName,
+      @Parameter(AzureStorageAccountKey.class) final String azureStorageAccountKey,
+      @Parameter(AzureStorageContainerName.class) final String azureStorageContainerName,
+      @Parameter(IsWindows.class) final Boolean isWindows) {
+    this.azureBatchAccountName = azureBatchAccountName;
+    this.azureBatchAccountKey = azureBatchAccountKey;
+    this.azureBatchAccountUri = azureBatchAccountUri;
+    this.azureBatchPoolId = azureBatchPoolId;
+    this.azureStorageAccountName = azureStorageAccountName;
+    this.azureStorageAccountKey = azureStorageAccountKey;
+    this.azureStorageContainerName = azureStorageContainerName;
+    this.isWindows = isWindows;
+  }
+
+  public Configuration getAzureBatchRuntimeConfiguration() {
+    return AzureBatchRuntimeConfigurationCreator
+        .getOrCreateAzureBatchRuntimeConfiguration(this.isWindows)
+        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_NAME, this.azureBatchAccountName)
+        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_KEY, this.azureBatchAccountKey)
+        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_URI, this.azureBatchAccountUri)
+        .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_POOL_ID, this.azureBatchPoolId)
+        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_ACCOUNT_NAME, this.azureStorageAccountName)
+        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_ACCOUNT_KEY, this.azureStorageAccountKey)
+        .set(AzureBatchRuntimeConfiguration.AZURE_STORAGE_CONTAINER_NAME, this.azureStorageContainerName)
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationStatic.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationStatic.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationStatic.java
new file mode 100644
index 0000000..52d27b8
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/AzureBatchRuntimeConfigurationStatic.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch.client;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.runtime.azbatch.AzureBatchClasspathProvider;
+import org.apache.reef.runtime.azbatch.AzureBatchJVMPathProvider;
+import org.apache.reef.runtime.azbatch.util.batch.IAzureBatchCredentialProvider;
+import org.apache.reef.runtime.azbatch.util.batch.SharedKeyBatchCredentialProvider;
+import org.apache.reef.runtime.azbatch.util.storage.ICloudBlobClientProvider;
+import org.apache.reef.runtime.azbatch.util.storage.StorageKeyCloudBlobProvider;
+import org.apache.reef.runtime.common.client.CommonRuntimeConfiguration;
+import org.apache.reef.runtime.common.client.DriverConfigurationProvider;
+import org.apache.reef.runtime.common.client.api.JobSubmissionHandler;
+import org.apache.reef.runtime.common.files.RuntimeClasspathProvider;
+import org.apache.reef.runtime.common.files.RuntimePathProvider;
+import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
+import org.apache.reef.util.logging.LoggingSetup;
+
+/**
+ * The static part of the AzureBatchRuntimeConfigurationStatic.
+ */
+@Private
+public class AzureBatchRuntimeConfigurationStatic extends ConfigurationModuleBuilder {
+  static {
+    LoggingSetup.setupCommonsLogging();
+  }
+
+  public static final ConfigurationModuleBuilder CONF = new AzureBatchRuntimeConfigurationStatic()
+      .merge(CommonRuntimeConfiguration.CONF)
+      .bindImplementation(JobSubmissionHandler.class, AzureBatchJobSubmissionHandler.class)
+      .bindImplementation(IAzureBatchCredentialProvider.class, SharedKeyBatchCredentialProvider.class)
+      .bindImplementation(ICloudBlobClientProvider.class, StorageKeyCloudBlobProvider.class)
+      .bindImplementation(DriverConfigurationProvider.class, AzureBatchDriverConfigurationProviderImpl.class)
+      .bindImplementation(RuntimeClasspathProvider.class, AzureBatchClasspathProvider.class)
+      .bindImplementation(RuntimePathProvider.class, AzureBatchJVMPathProvider.class);
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/package-info.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/package-info.java
new file mode 100644
index 0000000..b0e6df8
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/client/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Client for the REEF runtime for Azure Batch.
+ */
+package org.apache.reef.runtime.azbatch.client;

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/driver/AzureBatchDriverConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/driver/AzureBatchDriverConfiguration.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/driver/AzureBatchDriverConfiguration.java
new file mode 100644
index 0000000..9a096be
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/driver/AzureBatchDriverConfiguration.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch.driver;
+
+import org.apache.reef.annotations.audience.ClientSide;
+import org.apache.reef.annotations.audience.Public;
+import org.apache.reef.runtime.azbatch.AzureBatchClasspathProvider;
+import org.apache.reef.runtime.azbatch.AzureBatchJVMPathProvider;
+import org.apache.reef.runtime.azbatch.parameters.*;
+import org.apache.reef.runtime.azbatch.util.batch.IAzureBatchCredentialProvider;
+import org.apache.reef.runtime.azbatch.util.batch.TokenBatchCredentialProvider;
+import org.apache.reef.runtime.azbatch.util.storage.ICloudBlobClientProvider;
+import org.apache.reef.runtime.azbatch.util.storage.SharedAccessSignatureCloudBlobClientProvider;
+import org.apache.reef.runtime.common.driver.api.*;
+import org.apache.reef.runtime.common.driver.parameters.ClientRemoteIdentifier;
+import org.apache.reef.runtime.common.driver.parameters.DefinedRuntimes;
+import org.apache.reef.runtime.common.driver.parameters.EvaluatorTimeout;
+import org.apache.reef.runtime.common.driver.parameters.JobIdentifier;
+import org.apache.reef.runtime.common.files.RuntimeClasspathProvider;
+import org.apache.reef.runtime.common.files.RuntimePathProvider;
+import org.apache.reef.runtime.common.launch.parameters.ErrorHandlerRID;
+import org.apache.reef.runtime.common.launch.parameters.LaunchID;
+import org.apache.reef.runtime.common.parameters.JVMHeapSlack;
+import org.apache.reef.tang.formats.ConfigurationModule;
+import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
+import org.apache.reef.tang.formats.OptionalParameter;
+import org.apache.reef.tang.formats.RequiredParameter;
+
+/**
+ * ConfigurationModule to create Azure Batch Driver configurations.
+ */
+@Public
+@ClientSide
+public final class AzureBatchDriverConfiguration extends ConfigurationModuleBuilder {
+
+  /**
+   * @see JobIdentifier
+   */
+  public static final RequiredParameter<String> JOB_IDENTIFIER = new RequiredParameter<>();
+
+  /**
+   * @see DefinedRuntimes
+   */
+  public static final RequiredParameter<String> RUNTIME_NAME = new RequiredParameter<>();
+
+  /**
+   * @see EvaluatorTimeout
+   */
+  public static final OptionalParameter<Long> EVALUATOR_TIMEOUT = new OptionalParameter<>();
+
+  /**
+   * The client remote identifier.
+   */
+  public static final OptionalParameter<String> CLIENT_REMOTE_IDENTIFIER = new OptionalParameter<>();
+
+  /**
+   * The Azure Batch account URI to be used by REEF.
+   */
+  public static final RequiredParameter<String> AZURE_BATCH_ACCOUNT_URI = new RequiredParameter<>();
+
+  /**
+   * The Azure Batch account name to be used by REEF.
+   */
+  public static final RequiredParameter<String> AZURE_BATCH_ACCOUNT_NAME = new RequiredParameter<>();
+
+  /**
+   * The Azure Batch Pool ID.
+   */
+  public static final RequiredParameter<String> AZURE_BATCH_POOL_ID = new RequiredParameter<>();
+
+  /**
+   * The name of the Azure Storage account.
+   */
+  public static final RequiredParameter<String> AZURE_STORAGE_ACCOUNT_NAME = new RequiredParameter<>();
+
+  /**
+   * The name of the Azure Storage account container.
+   */
+  public static final RequiredParameter<String> AZURE_STORAGE_CONTAINER_NAME = new RequiredParameter<>();
+
+  /**
+   * The fraction of the container memory NOT to use for the Java Heap.
+   */
+  public static final OptionalParameter<Double> JVM_HEAP_SLACK = new OptionalParameter<>();
+
+  public static final ConfigurationModule CONF = new AzureBatchDriverConfiguration()
+      .bindImplementation(IAzureBatchCredentialProvider.class, TokenBatchCredentialProvider.class)
+      .bindImplementation(ICloudBlobClientProvider.class, SharedAccessSignatureCloudBlobClientProvider.class)
+      .bindImplementation(ResourceLaunchHandler.class, AzureBatchResourceLaunchHandler.class)
+      .bindImplementation(ResourceReleaseHandler.class, AzureBatchResourceReleaseHandler.class)
+      .bindImplementation(ResourceRequestHandler.class, AzureBatchResourceRequestHandler.class)
+      .bindImplementation(ResourceManagerStartHandler.class, AzureBatchResourceManagerStartHandler.class)
+      .bindImplementation(ResourceManagerStopHandler.class, AzureBatchResourceManagerStopHandler.class)
+
+      // Bind Azure Batch Configuration Parameters
+      .bindNamedParameter(AzureBatchAccountUri.class, AZURE_BATCH_ACCOUNT_URI)
+      .bindNamedParameter(AzureBatchAccountName.class, AZURE_BATCH_ACCOUNT_NAME)
+      .bindNamedParameter(AzureBatchPoolId.class, AZURE_BATCH_POOL_ID)
+      .bindNamedParameter(AzureStorageAccountName.class, AZURE_STORAGE_ACCOUNT_NAME)
+      .bindNamedParameter(AzureStorageContainerName.class, AZURE_STORAGE_CONTAINER_NAME)
+
+      // Bind the fields bound in AbstractDriverRuntimeConfiguration
+      .bindNamedParameter(JobIdentifier.class, JOB_IDENTIFIER)
+      .bindNamedParameter(LaunchID.class, JOB_IDENTIFIER)
+      .bindNamedParameter(EvaluatorTimeout.class, EVALUATOR_TIMEOUT)
+      .bindNamedParameter(ClientRemoteIdentifier.class, CLIENT_REMOTE_IDENTIFIER)
+      .bindNamedParameter(ErrorHandlerRID.class, CLIENT_REMOTE_IDENTIFIER)
+      .bindNamedParameter(JVMHeapSlack.class, JVM_HEAP_SLACK)
+      .bindImplementation(RuntimeClasspathProvider.class, AzureBatchClasspathProvider.class)
+      .bindImplementation(RuntimePathProvider.class, AzureBatchJVMPathProvider.class)
+      .bindSetEntry(DefinedRuntimes.class, RUNTIME_NAME)
+      .build();
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/561a336f/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/driver/AzureBatchEvaluatorShimConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/driver/AzureBatchEvaluatorShimConfigurationProvider.java b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/driver/AzureBatchEvaluatorShimConfigurationProvider.java
new file mode 100644
index 0000000..85b9103
--- /dev/null
+++ b/lang/java/reef-runtime-azbatch/src/main/java/org/apache/reef/runtime/azbatch/driver/AzureBatchEvaluatorShimConfigurationProvider.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.runtime.azbatch.driver;
+
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.runtime.azbatch.evaluator.EvaluatorShimConfiguration;
+import org.apache.reef.runtime.common.utils.RemoteManager;
+import org.apache.reef.tang.Configuration;
+
+import javax.inject.Inject;
+
+/**
+ * Configuration provider for the Azure Batch evaluator shim.
+ */
+@Private
+public class AzureBatchEvaluatorShimConfigurationProvider {
+
+  private final RemoteManager remoteManager;
+
+  @Inject
+  AzureBatchEvaluatorShimConfigurationProvider(final RemoteManager remoteManager) {
+    this.remoteManager = remoteManager;
+  }
+
+  /**
+   * Constructs a {@link Configuration} object which will be serialized and written to shim.config and
+   * used to launch the evaluator shim.
+   *
+   * @param containerId id of the container for which the shim is being launched.
+   * @return A {@link Configuration} object needed to launch the evaluator shim.
+   */
+  public Configuration getConfiguration(final String containerId) {
+    return EvaluatorShimConfiguration.CONF
+        .set(EvaluatorShimConfiguration.DRIVER_REMOTE_IDENTIFIER, this.remoteManager.getMyIdentifier())
+        .set(EvaluatorShimConfiguration.CONTAINER_IDENTIFIER, containerId)
+        .build();
+  }
+}