You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by ju...@apache.org on 2015/03/04 19:45:07 UTC

incubator-reef git commit: [REEF-192] Remove AbstractDriverRuntimeConfiguration from the local runtime

Repository: incubator-reef
Updated Branches:
  refs/heads/master 441a060cd -> 0911c0832


[REEF-192] Remove AbstractDriverRuntimeConfiguration from the local runtime

This removes the use of `AbstractDriverRuntimeConfiguration` in the
local runtime. Its function is taken over by the `ConfigurationModule`
`LocalDriverConfiguration`. Also, this removes the unused parameters for
files and libraries in the local runtime's driver configuration.

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

THis closes #101

Author:    Markus Weimer <we...@apache.org>


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

Branch: refs/heads/master
Commit: 0911c0832f514660082b8bfb0ca6be0a595243bf
Parents: 441a060
Author: Markus Weimer <we...@apache.org>
Authored: Tue Mar 3 18:30:23 2015 -0800
Committer: Julia Wang <jw...@yahoo.com>
Committed: Wed Mar 4 10:39:26 2015 -0800

----------------------------------------------------------------------
 .../client/DriverConfigurationProvider.java     | 74 ++++++++++++++++++++
 .../local/client/LocalJobSubmissionHandler.java | 53 +++++---------
 .../local/driver/LocalDriverConfiguration.java  | 43 +++++-------
 .../driver/LocalDriverRuntimeConfiguration.java | 30 --------
 .../runtime/local/driver/ResourceManager.java   |  5 --
 .../local/driver/parameters/GlobalFiles.java    | 31 --------
 .../driver/parameters/GlobalLibraries.java      | 31 --------
 .../local/driver/parameters/LocalFiles.java     | 31 --------
 .../local/driver/parameters/LocalLibraries.java | 31 --------
 .../local/driver/parameters/package-info.java   | 22 ------
 10 files changed, 108 insertions(+), 243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/DriverConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/DriverConfigurationProvider.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/DriverConfigurationProvider.java
new file mode 100644
index 0000000..a9ad172
--- /dev/null
+++ b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/DriverConfigurationProvider.java
@@ -0,0 +1,74 @@
+/**
+ * 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.local.client;
+
+import org.apache.reef.runtime.common.parameters.JVMHeapSlack;
+import org.apache.reef.runtime.local.client.parameters.NumberOfProcesses;
+import org.apache.reef.runtime.local.driver.LocalDriverConfiguration;
+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.io.File;
+
+/**
+ * Helper class that assembles the driver configuration when run on the local runtime.
+ */
+public final class DriverConfigurationProvider {
+
+  private final int nThreads;
+  private final double jvmHeapSlack;
+
+  @Inject
+  DriverConfigurationProvider(final @Parameter(NumberOfProcesses.class) int nThreads,
+                              final @Parameter(JVMHeapSlack.class) double jvmHeapSlack) {
+    this.nThreads = nThreads;
+    this.jvmHeapSlack = jvmHeapSlack;
+  }
+
+  private Configuration getDriverConfiguration(final File jobFolder,
+                                               final String clientRemoteId,
+                                               final String jobId) {
+    return LocalDriverConfiguration.CONF
+        .set(LocalDriverConfiguration.NUMBER_OF_PROCESSES, this.nThreads)
+        .set(LocalDriverConfiguration.ROOT_FOLDER, jobFolder.getAbsolutePath())
+        .set(LocalDriverConfiguration.JVM_HEAP_SLACK, this.jvmHeapSlack)
+        .set(LocalDriverConfiguration.CLIENT_REMOTE_IDENTIFIER, clientRemoteId)
+        .set(LocalDriverConfiguration.JOB_IDENTIFIER, jobId)
+        .build();
+  }
+
+  /**
+   * Assembles the driver configuration
+   *
+   * @param jobFolder                The folder in which the local runtime will execute this job.
+   * @param clientRemoteId           the remote identifier of the client. It is used by the Driver to establish a
+   *                                 connection back to the client.
+   * @param jobId                    The identifier of the job.
+   * @param applicationConfiguration The configuration of the application, e.g. a filled out DriverConfiguration
+   * @return The Driver configuration to be used to instantiate the Driver.
+   */
+  public Configuration getDriverConfiguration(final File jobFolder,
+                                              final String clientRemoteId,
+                                              final String jobId,
+                                              final Configuration applicationConfiguration) {
+    return Configurations.merge(getDriverConfiguration(jobFolder, clientRemoteId, jobId), applicationConfiguration);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalJobSubmissionHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalJobSubmissionHandler.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalJobSubmissionHandler.java
index 66b3ce6..8a63af9 100644
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalJobSubmissionHandler.java
+++ b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalJobSubmissionHandler.java
@@ -26,15 +26,11 @@ import org.apache.reef.runtime.common.client.api.JobSubmissionHandler;
 import org.apache.reef.runtime.common.files.ClasspathProvider;
 import org.apache.reef.runtime.common.files.REEFFileNames;
 import org.apache.reef.runtime.common.launch.JavaLaunchCommandBuilder;
-import org.apache.reef.runtime.common.parameters.JVMHeapSlack;
-import org.apache.reef.runtime.local.client.parameters.NumberOfProcesses;
 import org.apache.reef.runtime.local.client.parameters.RootFolder;
 import org.apache.reef.runtime.local.driver.LocalDriverConfiguration;
-import org.apache.reef.runtime.local.driver.LocalDriverRuntimeConfiguration;
 import org.apache.reef.runtime.local.process.LoggingRunnableProcessObserver;
 import org.apache.reef.runtime.local.process.RunnableProcess;
 import org.apache.reef.tang.Configuration;
-import org.apache.reef.tang.Tang;
 import org.apache.reef.tang.annotations.Parameter;
 import org.apache.reef.tang.formats.ConfigurationSerializer;
 import org.apache.reef.util.logging.LoggingScope;
@@ -64,31 +60,28 @@ final class LocalJobSubmissionHandler implements JobSubmissionHandler {
   public static final int DRIVER_MEMORY = 512;
   private static final Logger LOG = Logger.getLogger(LocalJobSubmissionHandler.class.getName());
   private final ExecutorService executor;
-  private final int nThreads;
   private final String rootFolderName;
   private final ConfigurationSerializer configurationSerializer;
-  private final REEFFileNames filenames;
+  private final REEFFileNames fileNames;
   private final ClasspathProvider classpath;
-  private final double jvmHeapSlack;
   private final LoggingScopeFactory loggingScopeFactory;
+  private final DriverConfigurationProvider driverConfigurationProvider;
 
   @Inject
   public LocalJobSubmissionHandler(
       final ExecutorService executor,
       final @Parameter(RootFolder.class) String rootFolderName,
-      final @Parameter(NumberOfProcesses.class) int nThreads,
       final ConfigurationSerializer configurationSerializer,
-      final REEFFileNames filenames,
+      final REEFFileNames fileNames,
       final ClasspathProvider classpath,
-      final @Parameter(JVMHeapSlack.class) double jvmHeapSlack,
-      final LoggingScopeFactory loggingScopeFactory) {
+      final LoggingScopeFactory loggingScopeFactory,
+      final DriverConfigurationProvider driverConfigurationProvider) {
 
     this.executor = executor;
-    this.nThreads = nThreads;
     this.configurationSerializer = configurationSerializer;
-    this.filenames = filenames;
+    this.fileNames = fileNames;
     this.classpath = classpath;
-    this.jvmHeapSlack = jvmHeapSlack;
+    this.driverConfigurationProvider = driverConfigurationProvider;
     this.rootFolderName = new File(rootFolderName).getAbsolutePath();
     this.loggingScopeFactory = loggingScopeFactory;
 
@@ -112,34 +105,20 @@ final class LocalJobSubmissionHandler implements JobSubmissionHandler {
         final File driverFolder = new File(jobFolder, DRIVER_FOLDER_NAME);
         driverFolder.mkdirs();
 
-        final DriverFiles driverFiles = DriverFiles.fromJobSubmission(t, this.filenames);
+        final DriverFiles driverFiles = DriverFiles.fromJobSubmission(t, this.fileNames);
         driverFiles.copyTo(driverFolder);
 
-        final Configuration driverConfigurationPart1 = driverFiles
-            .addNamesTo(LocalDriverConfiguration.CONF,
-                LocalDriverConfiguration.GLOBAL_FILES,
-                LocalDriverConfiguration.GLOBAL_LIBRARIES,
-                LocalDriverConfiguration.LOCAL_FILES,
-                LocalDriverConfiguration.LOCAL_LIBRARIES)
-            .set(LocalDriverConfiguration.NUMBER_OF_PROCESSES, this.nThreads)
-            .set(LocalDriverConfiguration.ROOT_FOLDER, jobFolder.getAbsolutePath())
-            .set(LocalDriverConfiguration.JVM_HEAP_SLACK, this.jvmHeapSlack)
-            .build();
-
-        final Configuration driverConfigurationPart2 = new LocalDriverRuntimeConfiguration()
-            .addClientConfiguration(this.configurationSerializer.fromString(t.getConfiguration()))
-            .setClientRemoteIdentifier(t.getRemoteId())
-            .setJobIdentifier(t.getIdentifier()).build();
+        final Configuration driverConfiguration = this.driverConfigurationProvider
+            .getDriverConfiguration(jobFolder, t.getRemoteId(), t.getIdentifier(),
+                configurationSerializer.fromString(t.getConfiguration()));
 
-        final Configuration driverConfiguration = Tang.Factory.getTang()
-            .newConfigurationBuilder(driverConfigurationPart1, driverConfigurationPart2).build();
-        final File runtimeConfigurationFile = new File(driverFolder, this.filenames.getDriverConfigurationPath());
-        this.configurationSerializer.toFile(driverConfiguration, runtimeConfigurationFile);
+        this.configurationSerializer.toFile(driverConfiguration,
+            new File(driverFolder, this.fileNames.getDriverConfigurationPath()));
 
         final List<String> command = new JavaLaunchCommandBuilder()
             .setErrorHandlerRID(t.getRemoteId())
             .setLaunchID(t.getIdentifier())
-            .setConfigurationFileName(this.filenames.getDriverConfigurationPath())
+            .setConfigurationFileName(this.fileNames.getDriverConfigurationPath())
             .setClassPath(this.classpath.getDriverClasspath())
             .setMemory(DRIVER_MEMORY)
             .build();
@@ -152,8 +131,8 @@ final class LocalJobSubmissionHandler implements JobSubmissionHandler {
             "driver",
             driverFolder,
             new LoggingRunnableProcessObserver(),
-            this.filenames.getDriverStdoutFileName(),
-            this.filenames.getDriverStderrFileName());
+            this.fileNames.getDriverStdoutFileName(),
+            this.fileNames.getDriverStderrFileName());
         this.executor.submit(process);
         this.executor.shutdown();
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverConfiguration.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverConfiguration.java
index c6a6206..8258567 100644
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverConfiguration.java
+++ b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverConfiguration.java
@@ -18,15 +18,15 @@
  */
 package org.apache.reef.runtime.local.driver;
 
+import org.apache.reef.runtime.common.driver.api.AbstractDriverRuntimeConfiguration;
+import org.apache.reef.runtime.common.driver.api.ResourceLaunchHandler;
+import org.apache.reef.runtime.common.driver.api.ResourceReleaseHandler;
+import org.apache.reef.runtime.common.driver.api.ResourceRequestHandler;
 import org.apache.reef.runtime.common.files.RuntimeClasspathProvider;
 import org.apache.reef.runtime.common.parameters.JVMHeapSlack;
 import org.apache.reef.runtime.local.LocalClasspathProvider;
 import org.apache.reef.runtime.local.client.parameters.NumberOfProcesses;
 import org.apache.reef.runtime.local.client.parameters.RootFolder;
-import org.apache.reef.runtime.local.driver.parameters.GlobalFiles;
-import org.apache.reef.runtime.local.driver.parameters.GlobalLibraries;
-import org.apache.reef.runtime.local.driver.parameters.LocalFiles;
-import org.apache.reef.runtime.local.driver.parameters.LocalLibraries;
 import org.apache.reef.tang.formats.ConfigurationModule;
 import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
 import org.apache.reef.tang.formats.OptionalParameter;
@@ -37,23 +37,6 @@ import org.apache.reef.tang.formats.RequiredParameter;
  * LocalDriverRuntimeConfiguration.
  */
 public class LocalDriverConfiguration extends ConfigurationModuleBuilder {
-
-  /**
-   * Files for the driver only.
-   */
-  public static final OptionalParameter<String> LOCAL_FILES = new OptionalParameter<>();
-  /**
-   * Libraries for the driver only.
-   */
-  public static final OptionalParameter<String> LOCAL_LIBRARIES = new OptionalParameter<>();
-  /**
-   * Files for the driver and all evaluators.
-   */
-  public static final OptionalParameter<String> GLOBAL_FILES = new OptionalParameter<>();
-  /**
-   * Libraries for the driver and all evaluators.
-   */
-  public static final OptionalParameter<String> GLOBAL_LIBRARIES = new OptionalParameter<>();
   /**
    * The maximum number or processes to spawn.
    */
@@ -67,12 +50,22 @@ public class LocalDriverConfiguration extends ConfigurationModuleBuilder {
    */
   public static final OptionalParameter<Double> JVM_HEAP_SLACK = new OptionalParameter<>();
 
+  /**
+   * The remote identifier to use for communications back to the client
+   */
+  public static final OptionalParameter<String> CLIENT_REMOTE_IDENTIFIER = new OptionalParameter<>();
+
+  /**
+   * The identifier of the Job submitted.
+   */
+  public static final RequiredParameter<String> JOB_IDENTIFIER = new RequiredParameter<>();
 
   public static final ConfigurationModule CONF = new LocalDriverConfiguration()
-      .bindSetEntry(LocalFiles.class, LOCAL_FILES)
-      .bindSetEntry(LocalLibraries.class, LOCAL_LIBRARIES)
-      .bindSetEntry(GlobalFiles.class, GLOBAL_FILES)
-      .bindSetEntry(GlobalLibraries.class, GLOBAL_LIBRARIES)
+      .bindImplementation(ResourceLaunchHandler.class, LocalResourceLaunchHandler.class)
+      .bindImplementation(ResourceRequestHandler.class, LocalResourceRequestHandler.class)
+      .bindImplementation(ResourceReleaseHandler.class, LocalResourceReleaseHandler.class)
+      .bindNamedParameter(AbstractDriverRuntimeConfiguration.ClientRemoteIdentifier.class, CLIENT_REMOTE_IDENTIFIER)
+      .bindNamedParameter(AbstractDriverRuntimeConfiguration.JobIdentifier.class, JOB_IDENTIFIER)
       .bindNamedParameter(NumberOfProcesses.class, NUMBER_OF_PROCESSES)
       .bindNamedParameter(RootFolder.class, ROOT_FOLDER)
       .bindNamedParameter(JVMHeapSlack.class, JVM_HEAP_SLACK)

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverRuntimeConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverRuntimeConfiguration.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverRuntimeConfiguration.java
deleted file mode 100644
index 52bd622..0000000
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/LocalDriverRuntimeConfiguration.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.runtime.local.driver;
-
-import org.apache.reef.runtime.common.driver.api.AbstractDriverRuntimeConfiguration;
-
-import javax.inject.Inject;
-
-public class LocalDriverRuntimeConfiguration extends AbstractDriverRuntimeConfiguration {
-  @Inject
-  public LocalDriverRuntimeConfiguration() {
-    super(LocalResourceLaunchHandler.class, LocalResourceReleaseHandler.class, LocalResourceRequestHandler.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ResourceManager.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ResourceManager.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ResourceManager.java
index 9708e77..e30eba3 100644
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ResourceManager.java
+++ b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/ResourceManager.java
@@ -32,8 +32,6 @@ import org.apache.reef.runtime.common.parameters.JVMHeapSlack;
 import org.apache.reef.runtime.common.utils.RemoteManager;
 import org.apache.reef.runtime.local.client.parameters.DefaultMemorySize;
 import org.apache.reef.runtime.local.client.parameters.DefaultNumberOfCores;
-import org.apache.reef.runtime.local.driver.parameters.GlobalFiles;
-import org.apache.reef.runtime.local.driver.parameters.GlobalLibraries;
 import org.apache.reef.tang.annotations.Parameter;
 import org.apache.reef.tang.exceptions.BindException;
 import org.apache.reef.tang.formats.ConfigurationSerializer;
@@ -46,7 +44,6 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -78,8 +75,6 @@ public final class ResourceManager {
       final ContainerManager containerManager,
       final @Parameter(RuntimeParameters.ResourceAllocationHandler.class) EventHandler<DriverRuntimeProtocol.ResourceAllocationProto> allocationHandler,
       final @Parameter(RuntimeParameters.RuntimeStatusHandler.class) EventHandler<DriverRuntimeProtocol.RuntimeStatusProto> runtimeStatusHandlerEventHandler,
-      final @Parameter(GlobalLibraries.class) Set<String> globalLibraries,
-      final @Parameter(GlobalFiles.class) Set<String> globalFiles,
       final @Parameter(DefaultMemorySize.class) int defaultMemorySize,
       final @Parameter(DefaultNumberOfCores.class) int defaultNumberOfCores,
       final @Parameter(JVMHeapSlack.class) double jvmHeapSlack,

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/GlobalFiles.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/GlobalFiles.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/GlobalFiles.java
deleted file mode 100644
index 489054f..0000000
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/GlobalFiles.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.reef.runtime.local.driver.parameters;
-
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-
-import java.util.Set;
-
-/**
- * The names of files that are to be copied to all evaluators.
- */
-@NamedParameter(doc = "The names of files that are to be copied to all evaluators.")
-public final class GlobalFiles implements Name<Set<String>> {
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/GlobalLibraries.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/GlobalLibraries.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/GlobalLibraries.java
deleted file mode 100644
index 1912829..0000000
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/GlobalLibraries.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.reef.runtime.local.driver.parameters;
-
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-
-import java.util.Set;
-
-/**
- * The names of files that are to be copied to all evaluators.
- */
-@NamedParameter(doc = "The names of files that are to be copied to all evaluators.")
-public final class GlobalLibraries implements Name<Set<String>> {
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/LocalFiles.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/LocalFiles.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/LocalFiles.java
deleted file mode 100644
index 87226cf..0000000
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/LocalFiles.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.reef.runtime.local.driver.parameters;
-
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-
-import java.util.Set;
-
-/**
- * Created by marku_000 on 2014-07-07.
- */
-@NamedParameter(doc = "The names of files that are to be kept on the driver only.")
-public final class LocalFiles implements Name<Set<String>> {
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/LocalLibraries.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/LocalLibraries.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/LocalLibraries.java
deleted file mode 100644
index 0aefb2b..0000000
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/LocalLibraries.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.reef.runtime.local.driver.parameters;
-
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-
-import java.util.Set;
-
-/**
- * The names of files that are to be kept on the driver only.
- */
-@NamedParameter(doc = "The names of files that are to be kept on the driver only.")
-public final class LocalLibraries implements Name<Set<String>> {
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/0911c083/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/package-info.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/package-info.java
deleted file mode 100644
index fccf5c7..0000000
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/driver/parameters/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.
- */
-/**
- * The named parameters for the Driver executed on the local runtime
- */
-package org.apache.reef.runtime.local.driver.parameters;
\ No newline at end of file