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

reef git commit: [Trivial] Revise azure batch launcher.

Repository: reef
Updated Branches:
  refs/heads/REEF-335 de1724c81 -> 3b1651917


[Trivial] Revise azure batch launcher.

Pull Request:
   Closes #1458


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

Branch: refs/heads/REEF-335
Commit: 3b1651917df0a8b80b986986a2e350a03da37ae8
Parents: de1724c
Author: Tyson Condie <tc...@apache.org>
Authored: Fri May 11 08:59:25 2018 -0700
Committer: Tyson Condie <tc...@apache.org>
Committed: Tue May 15 08:29:41 2018 -0700

----------------------------------------------------------------------
 lang/common/proto/bridge/ClientProtocol.proto   | 14 ++--
 .../bridge/client/DriverServiceLauncher.java    | 60 ++++++++++-------
 .../IDriverBridgeConfigurationProvider.java     | 32 ---------
 .../IDriverRuntimeConfigurationProvider.java    | 30 ---------
 .../client/IDriverServiceRuntimeLauncher.java   | 30 ---------
 .../client/WindowsRuntimePathProvider.java      | 43 -------------
 .../LocalDriverServiceRuntimeLauncher.java      | 56 ----------------
 .../YarnDriverServiceRuntimeLauncher.java       | 68 --------------------
 .../reef/bridge/client/launch/package-info.java | 22 -------
 .../apache/reef/bridge/client/package-info.java |  2 +-
 ...LocalDriverRuntimeConfigurationProvider.java | 57 ----------------
 .../YarnDriverRuntimeConfigurationProvider.java | 57 ----------------
 .../bridge/client/runtime/package-info.java     | 22 -------
 .../driver/client/grpc/DriverClientService.java |  2 +-
 .../reef/bridge/driver/client/package-info.java |  2 +-
 .../AzureBatchDriverConfigurationProvider.java  | 67 -------------------
 .../launch/azbatch/AzureBatchLauncher.java      | 47 ++++----------
 .../driver/launch/azbatch/package-info.java     |  2 +-
 .../driver/launch/local/package-info.java       |  2 +-
 .../reef/bridge/driver/launch/package-info.java |  2 +-
 .../bridge/driver/launch/yarn/package-info.java |  2 +-
 .../driver/service/grpc/GRPCDriverService.java  | 21 ++++--
 .../reef/bridge/examples/hello/HelloREEF.java   |  1 +
 .../common/launch/JavaLaunchCommandBuilder.java |  2 +-
 24 files changed, 80 insertions(+), 563 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/common/proto/bridge/ClientProtocol.proto
----------------------------------------------------------------------
diff --git a/lang/common/proto/bridge/ClientProtocol.proto b/lang/common/proto/bridge/ClientProtocol.proto
index 317f9d7..68c48d7 100644
--- a/lang/common/proto/bridge/ClientProtocol.proto
+++ b/lang/common/proto/bridge/ClientProtocol.proto
@@ -58,13 +58,6 @@ message AzureBatchRuntimeParameters {
   string azure_storage_account_key = 11;
 
   string azure_storage_container_name = 12;
-
-  // General information
-  enum OS {
-    WINDOWS = 0;
-    LINUX = 1;
-  }
-  OS operating_system = 20;
 }
 
 message MesosRuntimeParameters {
@@ -110,4 +103,11 @@ message DriverClientConfiguration {
 
   // enable http driver
   bool enable_http_driver = 25;
+
+  // General information
+  enum OS {
+    WINDOWS = 0;
+    LINUX = 1;
+  }
+  OS operating_system = 30;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/DriverServiceLauncher.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/DriverServiceLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/DriverServiceLauncher.java
index f2dfddd..5a7342c 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/DriverServiceLauncher.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/DriverServiceLauncher.java
@@ -28,6 +28,7 @@ import org.apache.reef.bridge.driver.service.IDriverServiceConfigurationProvider
 import org.apache.reef.bridge.driver.service.grpc.GRPCDriverServiceConfigurationProvider;
 import org.apache.reef.bridge.driver.client.JavaDriverClientLauncher;
 import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.runtime.azbatch.AzureBatchClasspathProvider;
 import org.apache.reef.runtime.common.files.*;
 import org.apache.reef.runtime.common.launch.JavaLaunchCommandBuilder;
 import org.apache.reef.runtime.local.LocalClasspathProvider;
@@ -39,7 +40,6 @@ import org.apache.reef.tang.Tang;
 import org.apache.reef.tang.exceptions.BindException;
 import org.apache.reef.tang.exceptions.InjectionException;
 import org.apache.reef.tang.formats.ConfigurationSerializer;
-import org.apache.reef.util.OSUtils;
 
 import java.io.File;
 import java.io.IOException;
@@ -61,6 +61,8 @@ public final class DriverServiceLauncher {
    */
   private static final Logger LOG = Logger.getLogger(DriverServiceLauncher.class.getName());
 
+  private static final Tang TANG = Tang.Factory.getTang();
+
   /**
    * This class should not be instantiated.
    */
@@ -74,7 +76,7 @@ public final class DriverServiceLauncher {
       throws InjectionException, IOException {
     ClientProtocol.DriverClientConfiguration.Builder builder =
         ClientProtocol.DriverClientConfiguration.newBuilder(driverClientConfigurationProto);
-    final File driverClientConfigurationFile = new File("driverclient.conf");
+    final File driverClientConfigurationFile = File.createTempFile("driverclient", ".conf");
     try {
       // Write driver client configuration to a file
       final Injector driverClientInjector = Tang.Factory.getTang().newInjector(driverClientConfiguration);
@@ -82,48 +84,58 @@ public final class DriverServiceLauncher {
           driverClientInjector.getInstance(ConfigurationSerializer.class);
       configurationSerializer.toFile(driverClientConfiguration, driverClientConfigurationFile);
 
-      // Resolve OS Runtime Path Provider.
-      final Configuration runtimeOSConfiguration =
-          driverClientConfigurationProto.getRuntimeCase() ==
-              ClientProtocol.DriverClientConfiguration.RuntimeCase.YARN_RUNTIME ?
-              Tang.Factory.getTang().newConfigurationBuilder()
-                  .bind(RuntimePathProvider.class, WindowsRuntimePathProvider.class)
-                  .bind(RuntimeClasspathProvider.class, YarnClasspathProvider.class)
-                  .bindConstructor(org.apache.hadoop.yarn.conf.YarnConfiguration.class,
-                      YarnConfigurationConstructor.class)
-                  .build() :
-              Tang.Factory.getTang().newConfigurationBuilder()
-                  .bind(RuntimePathProvider.class, OSUtils.isWindows() ?
-                      WindowsRuntimePathProvider.class : UnixJVMPathProvider.class)
-                  .bind(RuntimeClasspathProvider.class, LocalClasspathProvider.class)
-                  .build();
-      final Injector runtimeInjector = Tang.Factory.getTang().newInjector(runtimeOSConfiguration);
+      // Resolve Runtime ClassPath Provider.
+      final Configuration runtimeClassPathProvider;
+      switch (driverClientConfigurationProto.getRuntimeCase()) {
+      case YARN_RUNTIME:
+        runtimeClassPathProvider = TANG.newConfigurationBuilder()
+            .bind(RuntimeClasspathProvider.class, YarnClasspathProvider.class)
+            .bindConstructor(org.apache.hadoop.yarn.conf.YarnConfiguration.class,
+                YarnConfigurationConstructor.class)
+            .build();
+        break;
+      case LOCAL_RUNTIME:
+        runtimeClassPathProvider = TANG.newConfigurationBuilder()
+            .bind(RuntimeClasspathProvider.class, LocalClasspathProvider.class)
+            .build();
+        break;
+      case AZBATCH_RUNTIME:
+        runtimeClassPathProvider = TANG.newConfigurationBuilder()
+            .bind(RuntimeClasspathProvider.class, AzureBatchClasspathProvider.class)
+            .build();
+        break;
+      default:
+        throw new RuntimeException("unknown runtime " + driverClientConfigurationProto.getRuntimeCase());
+      }
+      final Injector runtimeInjector = TANG.newInjector(runtimeClassPathProvider);
       final REEFFileNames fileNames = runtimeInjector.getInstance(REEFFileNames.class);
       final ClasspathProvider classpathProvider = runtimeInjector.getInstance(ClasspathProvider.class);
-      final RuntimePathProvider runtimePathProvider = runtimeInjector.getInstance(RuntimePathProvider.class);
       final List<String> launchCommand = new JavaLaunchCommandBuilder(JavaDriverClientLauncher.class, null)
           .setConfigurationFilePaths(
               Collections.singletonList("./" + fileNames.getLocalFolderPath() + "/" +
                   driverClientConfigurationFile.getName()))
-          .setJavaPath(runtimePathProvider.getPath())
-          .setClassPath(classpathProvider.getEvaluatorClasspath())
+          .setJavaPath("java")
+          .setClassPath(driverClientConfigurationProto.getOperatingSystem() ==
+              ClientProtocol.DriverClientConfiguration.OS.WINDOWS ?
+              StringUtils.join(classpathProvider.getDriverClasspath(), ";") :
+                  StringUtils.join(classpathProvider.getDriverClasspath(), ":"))
           .build();
       final String cmd = StringUtils.join(launchCommand, ' ');
       builder.setDriverClientLaunchCommand(cmd);
       builder.addLocalFiles(driverClientConfigurationFile.getAbsolutePath());
 
       // call main()
-      final File driverClientConfFile = new File("driverclient.json");
+      final File driverClientConfFile = File.createTempFile("driverclient", ".json");
       try {
         try (PrintWriter out = new PrintWriter(driverClientConfFile)) {
           out.println(JsonFormat.printer().print(builder.build()));
         }
         main(new String[]{driverClientConfFile.getAbsolutePath()});
       } finally {
-        driverClientConfFile.delete();
+        driverClientConfFile.deleteOnExit();
       }
     } finally {
-      driverClientConfigurationFile.delete();
+      driverClientConfigurationFile.deleteOnExit();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverBridgeConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverBridgeConfigurationProvider.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverBridgeConfigurationProvider.java
deleted file mode 100644
index c9fefb0..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverBridgeConfigurationProvider.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.bridge.client;
-
-import org.apache.reef.bridge.proto.ClientProtocol;
-import org.apache.reef.tang.Configuration;
-
-/**
- * Driver bridge configuration provider.
- */
-public interface IDriverBridgeConfigurationProvider {
-
-  Configuration getConfiguration(final ClientProtocol.DriverClientConfiguration driverConfiguration);
-
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverRuntimeConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverRuntimeConfigurationProvider.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverRuntimeConfigurationProvider.java
deleted file mode 100644
index 4e32464..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverRuntimeConfigurationProvider.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.bridge.client;
-
-import org.apache.reef.bridge.client.runtime.LocalDriverRuntimeConfigurationProvider;
-import org.apache.reef.tang.annotations.DefaultImplementation;
-
-/**
- * Configuration provider for the runtime.
- */
-@DefaultImplementation(LocalDriverRuntimeConfigurationProvider.class)
-public interface IDriverRuntimeConfigurationProvider extends IDriverBridgeConfigurationProvider {
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverServiceRuntimeLauncher.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverServiceRuntimeLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverServiceRuntimeLauncher.java
deleted file mode 100644
index 789fa03..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/IDriverServiceRuntimeLauncher.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.bridge.client;
-
-import org.apache.reef.bridge.proto.ClientProtocol;
-
-/**
- * Driver service launcher.
- */
-public interface IDriverServiceRuntimeLauncher {
-
-  void launch(final ClientProtocol.DriverClientConfiguration driverClientConfiguration);
-}

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

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/LocalDriverServiceRuntimeLauncher.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/LocalDriverServiceRuntimeLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/LocalDriverServiceRuntimeLauncher.java
deleted file mode 100644
index 44ac5ea..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/LocalDriverServiceRuntimeLauncher.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.bridge.client.launch;
-
-import org.apache.reef.bridge.client.IDriverRuntimeConfigurationProvider;
-import org.apache.reef.bridge.driver.service.IDriverServiceConfigurationProvider;
-import org.apache.reef.bridge.client.IDriverServiceRuntimeLauncher;
-import org.apache.reef.bridge.proto.ClientProtocol;
-import org.apache.reef.client.DriverLauncher;
-import org.apache.reef.tang.exceptions.InjectionException;
-
-import javax.inject.Inject;
-
-/**
- * Local driver service launcher.
- */
-public final class LocalDriverServiceRuntimeLauncher implements IDriverServiceRuntimeLauncher {
-
-  private final IDriverRuntimeConfigurationProvider driverRuntimeConfigurationProvider;
-
-  private final IDriverServiceConfigurationProvider driverServiceConfigurationProvider;
-  @Inject
-  private LocalDriverServiceRuntimeLauncher(
-      final IDriverRuntimeConfigurationProvider driverRuntimeConfigurationProvider,
-      final IDriverServiceConfigurationProvider driverServiceConfigurationProvider) {
-    this.driverRuntimeConfigurationProvider = driverRuntimeConfigurationProvider;
-    this.driverServiceConfigurationProvider = driverServiceConfigurationProvider;
-  }
-
-  @Override
-  public void launch(final ClientProtocol.DriverClientConfiguration driverClientConfiguration) {
-    try {
-      DriverLauncher.getLauncher(driverRuntimeConfigurationProvider.getConfiguration(driverClientConfiguration))
-          .run(driverServiceConfigurationProvider.getDriverServiceConfiguration(driverClientConfiguration));
-    } catch (InjectionException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/YarnDriverServiceRuntimeLauncher.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/YarnDriverServiceRuntimeLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/YarnDriverServiceRuntimeLauncher.java
deleted file mode 100644
index 9ada0d1..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/YarnDriverServiceRuntimeLauncher.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.bridge.client.launch;
-
-import org.apache.reef.bridge.client.IDriverRuntimeConfigurationProvider;
-import org.apache.reef.bridge.driver.service.IDriverServiceConfigurationProvider;
-import org.apache.reef.bridge.client.IDriverServiceRuntimeLauncher;
-import org.apache.reef.bridge.proto.ClientProtocol;
-import org.apache.reef.client.DriverLauncher;
-import org.apache.reef.client.LauncherStatus;
-import org.apache.reef.tang.exceptions.InjectionException;
-
-import javax.inject.Inject;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * Yarn driver service launcher.
- */
-public final class YarnDriverServiceRuntimeLauncher implements IDriverServiceRuntimeLauncher {
-
-  private static final Logger LOG = Logger.getLogger(YarnDriverServiceRuntimeLauncher.class.getName());
-
-  private final IDriverRuntimeConfigurationProvider driverRuntimeConfigurationProvider;
-
-  private final IDriverServiceConfigurationProvider driverServiceConfigurationProvider;
-
-  @Inject
-  private YarnDriverServiceRuntimeLauncher(
-      final IDriverRuntimeConfigurationProvider driverRuntimeConfigurationProvider,
-      final IDriverServiceConfigurationProvider driverServiceConfigurationProvider) {
-    this.driverRuntimeConfigurationProvider = driverRuntimeConfigurationProvider;
-    this.driverServiceConfigurationProvider = driverServiceConfigurationProvider;
-  }
-
-  @Override
-  public void launch(final ClientProtocol.DriverClientConfiguration driverClientConfiguration) {
-    try {
-      final LauncherStatus status = DriverLauncher.getLauncher(
-          driverRuntimeConfigurationProvider.getConfiguration(driverClientConfiguration))
-          .run(driverServiceConfigurationProvider.getDriverServiceConfiguration(driverClientConfiguration));
-      LOG.log(Level.INFO, "Job complete status: " + status.toString());
-      if (status.getError().isPresent()) {
-        LOG.log(Level.SEVERE, status.getError().get().getMessage());
-        status.getError().get().printStackTrace();
-      }
-    } catch (InjectionException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/package-info.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/package-info.java
deleted file mode 100644
index bc89ef7..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/launch/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Client bridge runtime specific launchers.
- */
-package org.apache.reef.bridge.client.launch;

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/package-info.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/package-info.java
index b0793e3..9b6021d 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/package-info.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/package-info.java
@@ -17,6 +17,6 @@
  * under the License.
  */
 /**
- * Client bridge.
+ * Java client launcher.
  */
 package org.apache.reef.bridge.client;

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/LocalDriverRuntimeConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/LocalDriverRuntimeConfigurationProvider.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/LocalDriverRuntimeConfigurationProvider.java
deleted file mode 100644
index 0f8e3a8..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/LocalDriverRuntimeConfigurationProvider.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.bridge.client.runtime;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.reef.bridge.client.IDriverRuntimeConfigurationProvider;
-import org.apache.reef.bridge.proto.ClientProtocol;
-import org.apache.reef.runtime.local.client.LocalRuntimeConfiguration;
-import org.apache.reef.tang.Configuration;
-import org.apache.reef.tang.formats.ConfigurationModule;
-
-import javax.inject.Inject;
-
-/**
- * Local driver runtime configuration provider for the bridge.
- */
-public final class LocalDriverRuntimeConfigurationProvider implements IDriverRuntimeConfigurationProvider {
-
-  @Inject
-  LocalDriverRuntimeConfigurationProvider() {
-  }
-
-  @Override
-  public Configuration getConfiguration(final ClientProtocol.DriverClientConfiguration driverConfiguration) {
-    ConfigurationModule localRuntimeCM = LocalRuntimeConfiguration.CONF;
-    if (driverConfiguration.getLocalRuntime().getMaxNumberOfEvaluators() > 0) {
-      localRuntimeCM = localRuntimeCM.set(LocalRuntimeConfiguration.MAX_NUMBER_OF_EVALUATORS,
-          driverConfiguration.getLocalRuntime().getMaxNumberOfEvaluators());
-    }
-    if (StringUtils.isNotEmpty(driverConfiguration.getLocalRuntime().getRuntimeRootFolder())) {
-      localRuntimeCM = localRuntimeCM.set(LocalRuntimeConfiguration.RUNTIME_ROOT_FOLDER,
-          driverConfiguration.getLocalRuntime().getRuntimeRootFolder());
-    }
-    if (driverConfiguration.getLocalRuntime().getJvmHeapSlack() > 0.0) {
-      localRuntimeCM = localRuntimeCM.set(LocalRuntimeConfiguration.JVM_HEAP_SLACK,
-          driverConfiguration.getLocalRuntime().getJvmHeapSlack());
-    }
-    return localRuntimeCM.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/YarnDriverRuntimeConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/YarnDriverRuntimeConfigurationProvider.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/YarnDriverRuntimeConfigurationProvider.java
deleted file mode 100644
index 016f885..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/YarnDriverRuntimeConfigurationProvider.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.reef.bridge.client.runtime;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.reef.bridge.client.IDriverRuntimeConfigurationProvider;
-import org.apache.reef.bridge.proto.ClientProtocol;
-import org.apache.reef.runtime.yarn.client.YarnClientConfiguration;
-import org.apache.reef.runtime.yarn.driver.parameters.FileSystemUrl;
-import org.apache.reef.tang.Configuration;
-import org.apache.reef.tang.Configurations;
-import org.apache.reef.tang.JavaConfigurationBuilder;
-import org.apache.reef.tang.Tang;
-
-import javax.inject.Inject;
-
-/**
- * Yarn driver runtime configuration provider for the bridge.
- */
-public final class YarnDriverRuntimeConfigurationProvider implements IDriverRuntimeConfigurationProvider {
-
-  @Inject
-  YarnDriverRuntimeConfigurationProvider() {
-  }
-
-  @Override
-  public Configuration getConfiguration(final ClientProtocol.DriverClientConfiguration driverConfiguration) {
-    Configuration yarnConfiguration = YarnClientConfiguration.CONF
-        .set(YarnClientConfiguration.UNMANAGED_DRIVER, driverConfiguration.getYarnRuntime().getUnmangedDriver())
-        .set(YarnClientConfiguration.YARN_PRIORITY, driverConfiguration.getYarnRuntime().getPriority())
-        .set(YarnClientConfiguration.JVM_HEAP_SLACK, 0.0)
-        .build();
-    if (StringUtils.isNotEmpty(driverConfiguration.getYarnRuntime().getFilesystemUrl())) {
-      final JavaConfigurationBuilder providerConfig = Tang.Factory.getTang().newConfigurationBuilder()
-          .bindNamedParameter(FileSystemUrl.class, driverConfiguration.getYarnRuntime().getFilesystemUrl());
-      yarnConfiguration = Configurations.merge(yarnConfiguration, providerConfig.build());
-    }
-    return yarnConfiguration;
-  }
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/package-info.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/package-info.java
deleted file mode 100644
index 24dc42c..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/client/runtime/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * Client bridge.
- */
-package org.apache.reef.bridge.client.runtime;

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/DriverClientService.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/DriverClientService.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/DriverClientService.java
index 3ddeafb..b646d6e 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/DriverClientService.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/grpc/DriverClientService.java
@@ -127,7 +127,7 @@ public final class DriverClientService extends DriverClientGrpc.DriverClientImpl
       LOG.log(Level.INFO, "possibly idle. waiting for some action.");
       this.isIdle = true;
       try {
-        Thread.sleep(5000);
+        Thread.sleep(120000); // a couple of minutes
       } catch (InterruptedException e) {
         LOG.log(Level.WARNING, e.getMessage());
       }

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/package-info.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/package-info.java
index bba19b2..b94b8e5 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/package-info.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/client/package-info.java
@@ -17,6 +17,6 @@
  * under the License.
  */
 /**
- * Java bridge client driver.
+ * Example (Java) driver client.
  */
 package org.apache.reef.bridge.driver.client;

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchDriverConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchDriverConfigurationProvider.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchDriverConfigurationProvider.java
deleted file mode 100644
index c4dd7f2..0000000
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchDriverConfigurationProvider.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.reef.bridge.driver.launch.azbatch;
-
-import org.apache.reef.bridge.driver.service.IDriverServiceConfigurationProvider;
-import org.apache.reef.bridge.proto.ClientProtocol;
-import org.apache.reef.runtime.common.client.DriverConfigurationProvider;
-import org.apache.reef.runtime.common.driver.parameters.ClientRemoteIdentifier;
-import org.apache.reef.tang.Configuration;
-
-import javax.inject.Inject;
-import java.io.File;
-import java.io.IOException;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * This is the Java Driver configuration generator for .NET Drivers that generates
- * the Driver configuration at runtime. Called by {@link AzureBatchLauncher}.
- */
-final class AzureBatchDriverConfigurationProvider {
-
-  private static final Logger LOG = Logger.getLogger(AzureBatchDriverConfigurationProvider.class.getName());
-
-  // The driver service configuration provider
-  private final IDriverServiceConfigurationProvider driverServiceConfigurationProvider;
-
-  // The driver runtime configuration provider
-  private final DriverConfigurationProvider driverConfigurationProvider;
-
-  @Inject
-  private AzureBatchDriverConfigurationProvider(
-      final DriverConfigurationProvider driverConfigurationProvider,
-      final IDriverServiceConfigurationProvider driverServiceConfigurationProvider) {
-    this.driverConfigurationProvider = driverConfigurationProvider;
-    this.driverServiceConfigurationProvider = driverServiceConfigurationProvider;
-  }
-
-  Configuration getDriverConfigurationFromParams(
-      final ClientProtocol.DriverClientConfiguration driverClientConfiguration) throws IOException {
-
-    final String jobId = driverClientConfiguration.getJobid();
-    final File jobFolder = new File(driverClientConfiguration.getDriverJobSubmissionDirectory());
-
-    LOG.log(Level.INFO, "jobFolder {0} jobId {1}.", new Object[]{jobFolder.toURI(), jobId});
-
-    return this.driverConfigurationProvider.getDriverConfiguration(
-        jobFolder.toURI(), ClientRemoteIdentifier.NONE, jobId,
-        driverServiceConfigurationProvider.getDriverServiceConfiguration(driverClientConfiguration));
-  }
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchLauncher.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchLauncher.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchLauncher.java
index 6390035..d1b4557 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchLauncher.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/AzureBatchLauncher.java
@@ -19,21 +19,16 @@
 package org.apache.reef.bridge.driver.launch.azbatch;
 
 import org.apache.reef.bridge.driver.launch.IDriverLauncher;
+import org.apache.reef.bridge.driver.service.IDriverServiceConfigurationProvider;
 import org.apache.reef.bridge.proto.ClientProtocol;
+import org.apache.reef.client.REEF;
 import org.apache.reef.runtime.azbatch.client.AzureBatchRuntimeConfiguration;
 import org.apache.reef.runtime.azbatch.client.AzureBatchRuntimeConfigurationCreator;
-import org.apache.reef.runtime.common.REEFEnvironment;
-import org.apache.reef.runtime.common.evaluator.PIDStoreStartHandler;
-import org.apache.reef.runtime.common.launch.REEFErrorHandler;
-import org.apache.reef.runtime.common.launch.REEFMessageCodec;
 import org.apache.reef.tang.Configuration;
 import org.apache.reef.tang.Tang;
 import org.apache.reef.tang.exceptions.InjectionException;
-import org.apache.reef.wake.remote.RemoteConfiguration;
-import org.apache.reef.wake.time.Clock;
 
 import javax.inject.Inject;
-import java.io.IOException;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -47,35 +42,21 @@ public final class AzureBatchLauncher implements IDriverLauncher {
   private static final Logger LOG = Logger.getLogger(AzureBatchLauncher.class.getName());
   private static final Tang TANG = Tang.Factory.getTang();
 
+  private final IDriverServiceConfigurationProvider driverServiceConfigurationProvider;
+
   @Inject
-  private AzureBatchLauncher() {
+  private AzureBatchLauncher(final IDriverServiceConfigurationProvider driverServiceConfigurationProvider) {
+    this.driverServiceConfigurationProvider = driverServiceConfigurationProvider;
   }
 
   public void launch(final ClientProtocol.DriverClientConfiguration driverClientConfiguration) {
-    try {
-      final AzureBatchDriverConfigurationProvider azureBatchDriverConfigurationProvider =
-          TANG.newInjector(generateConfigurationFromJobSubmissionParameters(driverClientConfiguration))
-              .getInstance(AzureBatchDriverConfigurationProvider.class);
-
-      final Configuration launcherConfig =
-          TANG.newConfigurationBuilder()
-              .bindNamedParameter(RemoteConfiguration.ManagerName.class, "AzureBatchLauncher")
-              .bindNamedParameter(RemoteConfiguration.ErrorHandler.class, REEFErrorHandler.class)
-              .bindNamedParameter(RemoteConfiguration.MessageCodec.class, REEFMessageCodec.class)
-              .bindSetEntry(Clock.RuntimeStartHandler.class, PIDStoreStartHandler.class)
-              .build();
-
-      try (final REEFEnvironment reef = REEFEnvironment.fromConfiguration(
-          azureBatchDriverConfigurationProvider.getDriverConfigurationFromParams(driverClientConfiguration),
-          launcherConfig)) {
-        reef.run();
-      } catch (final InjectionException ex) {
-        throw fatal("Unable to configure and start REEFEnvironment.", ex);
-      }
-    } catch (InjectionException | IOException e) {
-      throw fatal("Unable to configure and start REEFEnvironment.", e);
+    try (final REEF reef = TANG.newInjector(
+        generateConfigurationFromJobSubmissionParameters(driverClientConfiguration)).getInstance(REEF.class)) {
+      LOG.log(Level.INFO, "Submitting job");
+      reef.submit(driverServiceConfigurationProvider.getDriverServiceConfiguration(driverClientConfiguration));
+    } catch (InjectionException e) {
+      fatal("unable to launch", e);
     }
-
     LOG.log(Level.INFO, "Exiting BootstrapLauncher.main()");
 
     System.exit(0); // TODO[REEF-1715]: Should be able to exit cleanly at the end of main()
@@ -84,8 +65,8 @@ public final class AzureBatchLauncher implements IDriverLauncher {
   private static Configuration generateConfigurationFromJobSubmissionParameters(
       final ClientProtocol.DriverClientConfiguration driverClientConfiguration) {
     return AzureBatchRuntimeConfigurationCreator.getOrCreateAzureBatchRuntimeConfiguration(
-        driverClientConfiguration.getAzbatchRuntime().getOperatingSystem() ==
-                ClientProtocol.AzureBatchRuntimeParameters.OS.WINDOWS)
+        driverClientConfiguration.getOperatingSystem() ==
+            ClientProtocol.DriverClientConfiguration.OS.WINDOWS)
         .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_NAME,
             driverClientConfiguration.getAzbatchRuntime().getAzureBatchAccountName())
         .set(AzureBatchRuntimeConfiguration.AZURE_BATCH_ACCOUNT_KEY,

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/package-info.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/package-info.java
index 8d1d237..e39f432 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/package-info.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/azbatch/package-info.java
@@ -17,6 +17,6 @@
  * under the License.
  */
 /**
- * GRPC implementation for driver bridge service.
+ * Azure Batch driver bridge service launcher.
  */
 package org.apache.reef.bridge.driver.launch.azbatch;

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/package-info.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/package-info.java
index 2ff6dcc..8359938 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/package-info.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/local/package-info.java
@@ -17,6 +17,6 @@
  * under the License.
  */
 /**
- * GRPC implementation for driver bridge service.
+ * Local driver bridge service launcher.
  */
 package org.apache.reef.bridge.driver.launch.local;

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/package-info.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/package-info.java
index 85e9825..2eb3422 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/package-info.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/package-info.java
@@ -17,6 +17,6 @@
  * under the License.
  */
 /**
- * GRPC implementation for driver bridge service.
+ * Driver launchers for the bridge service on various runtimes.
  */
 package org.apache.reef.bridge.driver.launch;

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/package-info.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/package-info.java
index fcbbdb7..40a715f 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/package-info.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/launch/yarn/package-info.java
@@ -17,6 +17,6 @@
  * under the License.
  */
 /**
- * GRPC implementation for driver bridge service.
+ * YARN driver bridge service launcher.
  */
 package org.apache.reef.bridge.driver.launch.yarn;

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverService.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverService.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverService.java
index 86fbc1f..7666695 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverService.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/driver/service/grpc/GRPCDriverService.java
@@ -132,7 +132,7 @@ public final class GRPCDriverService implements IDriverService {
       final String cmd = this.driverClientCommand + " " + this.server.getPort();
       final String cmdOs = OSUtils.isWindows() ? "cmd.exe /c \"" + cmd + "\"" : cmd;
       LOG.log(Level.INFO, "CMD: " + cmdOs);
-      this.driverProcess = Runtime.getRuntime().exec(cmdOs);
+      this.driverProcess = Runtime.getRuntime().exec(cmdOs, null, new File(System.getProperty("user.dir")));
       synchronized (this) {
         // wait for driver client process to register
         while (this.clientStub == null && driverProcessIsAlive()) {
@@ -192,19 +192,25 @@ public final class GRPCDriverService implements IDriverService {
       LOG.log(Level.INFO, "Exit code: " + this.driverProcess.exitValue());
     }
     LOG.log(Level.INFO, "capturing driver process stderr");
-    StringBuffer errBuffer = new StringBuffer();
-    InputStream errStream = this.driverProcess.getErrorStream();
+    StringBuffer outputBuffer = new StringBuffer();
     try {
       int nextChar;
-      errBuffer.append("\n==============================================\n");
+      final InputStream errStream = this.driverProcess.getErrorStream();
+      outputBuffer.append("\nSTDERR =======================================\n");
       while ((nextChar = errStream.read()) != -1) {
-        errBuffer.append((char) nextChar);
+        outputBuffer.append((char) nextChar);
       }
-      errBuffer.append("\n==============================================\n");
+      outputBuffer.append("\n==============================================\n");
+      final InputStream outStream = this.driverProcess.getInputStream();
+      outputBuffer.append("\nSTDOUT =======================================\n");
+      while ((nextChar = outStream.read()) != -1) {
+        outputBuffer.append((char) nextChar);
+      }
+      outputBuffer.append("\n==============================================\n");
     } catch (IOException e) {
       LOG.log(Level.WARNING, "Error while capturing output stream: " + e.getMessage());
     }
-    LOG.log(Level.INFO, errBuffer.toString());
+    LOG.log(Level.INFO, outputBuffer.toString());
   }
 
   /**
@@ -616,6 +622,7 @@ public final class GRPCDriverService implements IDriverService {
           GRPCDriverService.this.clientStub = DriverClientGrpc.newFutureStub(channel);
           GRPCDriverService.this.notifyAll();
         }
+        LOG.log(Level.INFO, "Driver has registered on port " + request.getPort());
       } finally {
         responseObserver.onNext(null);
         responseObserver.onCompleted();

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/HelloREEF.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/HelloREEF.java b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/HelloREEF.java
index 9b48823..3526b81 100644
--- a/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/HelloREEF.java
+++ b/lang/java/reef-bridge-proto-java/src/main/java/org/apache/reef/bridge/examples/hello/HelloREEF.java
@@ -57,6 +57,7 @@ public final class HelloREEF {
         ClientProtocol.DriverClientConfiguration.newBuilder();
     builder.setJobid("HelloREEF");
     builder.setEnableHttpDriver(false);
+    builder.setOperatingSystem(ClientProtocol.DriverClientConfiguration.OS.LINUX);
     builder.setAzbatchRuntime(ClientProtocol.AzureBatchRuntimeParameters.newBuilder()
         .build());
     builder.addGlobalLibraries(EnvironmentUtils.getClassLocation(HelloDriver.class));

http://git-wip-us.apache.org/repos/asf/reef/blob/3b165191/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/JavaLaunchCommandBuilder.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/JavaLaunchCommandBuilder.java b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/JavaLaunchCommandBuilder.java
index bd7784b..5f81a5d 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/JavaLaunchCommandBuilder.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/runtime/common/launch/JavaLaunchCommandBuilder.java
@@ -108,7 +108,7 @@ public final class JavaLaunchCommandBuilder implements LaunchCommandBuilder {
 
         if (classPath != null && !classPath.isEmpty()) {
           add("-classpath");
-          add("\"" + classPath + "\"");
+          add(classPath);
         }
 
         propagateProperties(this, true, "proc_reef");