You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by ch...@apache.org on 2014/12/18 08:22:29 UTC

[1/4] incubator-reef git commit: [REEF-77] Remove REEF Bridge

Repository: incubator-reef
Updated Branches:
  refs/heads/branch-0.10.0-incubating 19019dc1a -> dee0a76ff


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobClient.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobClient.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobClient.java
deleted file mode 100644
index 62bfac1..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobClient.java
+++ /dev/null
@@ -1,322 +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.javabridge.generic;
-
-import org.apache.reef.client.*;
-import org.apache.reef.io.network.naming.NameServerConfiguration;
-import org.apache.reef.javabridge.NativeInterop;
-import org.apache.reef.tang.Configuration;
-import org.apache.reef.tang.Configurations;
-import org.apache.reef.tang.annotations.Unit;
-import org.apache.reef.tang.exceptions.BindException;
-import org.apache.reef.tang.formats.AvroConfigurationSerializer;
-import org.apache.reef.tang.formats.ConfigurationModule;
-import org.apache.reef.util.EnvironmentUtils;
-import org.apache.reef.util.logging.LoggingScope;
-import org.apache.reef.util.logging.LoggingScopeFactory;
-import org.apache.reef.wake.EventHandler;
-import org.apache.reef.webserver.HttpHandlerConfiguration;
-import org.apache.reef.webserver.HttpServerReefEventHandler;
-import org.apache.reef.webserver.ReefEventStateManager;
-
-import javax.inject.Inject;
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * Clr Bridge Client.
- */
-@Unit
-public class JobClient {
-
-  /**
-   * Standard java logger.
-   */
-  private static final Logger LOG = Logger.getLogger(JobClient.class.getName());
-
-  /**
-   * Reference to the REEF framework.
-   * This variable is injected automatically in the constructor.
-   */
-  private final REEF reef;
-
-  /**
-   * Job Driver configuration.
-   */
-  private Configuration driverConfiguration;
-  private ConfigurationModule driverConfigModule;
-
-  /**
-   * A reference to the running job that allows client to send messages back to the job driver
-   */
-  private RunningJob runningJob;
-
-  /**
-   * Set to false when job driver is done.
-   */
-  private boolean isBusy = true;
-
-  private int driverMemory;
-
-  private String driverId;
-
-  private String jobSubmissionDirectory = "reefTmp/job_" + System.currentTimeMillis();
-
-  /**
-   * A factory that provides LoggingScope
-   */
-  private final LoggingScopeFactory loggingScopeFactory;
-  /**
-   * Clr Bridge client.
-   * Parameters are injected automatically by TANG.
-   *
-   * @param reef Reference to the REEF framework.
-   */
-  @Inject
-  JobClient(final REEF reef, final LoggingScopeFactory loggingScopeFactory) throws BindException {
-    this.loggingScopeFactory = loggingScopeFactory;
-    this.reef = reef;
-    this.driverConfigModule = getDriverConfiguration();
-  }
-
-  public static ConfigurationModule getDriverConfiguration() {
-    return EnvironmentUtils.addClasspath(DriverConfiguration.CONF, DriverConfiguration.GLOBAL_LIBRARIES)
-        .set(DriverConfiguration.ON_EVALUATOR_ALLOCATED, JobDriver.AllocatedEvaluatorHandler.class)
-        .set(DriverConfiguration.ON_EVALUATOR_FAILED, JobDriver.FailedEvaluatorHandler.class)
-        .set(DriverConfiguration.ON_CONTEXT_ACTIVE, JobDriver.ActiveContextHandler.class)
-        .set(DriverConfiguration.ON_DRIVER_RESTART_CONTEXT_ACTIVE, JobDriver.DriverRestartActiveContextHandler.class)
-        .set(DriverConfiguration.ON_CONTEXT_CLOSED, JobDriver.ClosedContextHandler.class)
-        .set(DriverConfiguration.ON_CONTEXT_FAILED, JobDriver.FailedContextHandler.class)
-        .set(DriverConfiguration.ON_CONTEXT_MESSAGE, JobDriver.ContextMessageHandler.class)
-        .set(DriverConfiguration.ON_TASK_MESSAGE, JobDriver.TaskMessageHandler.class)
-        .set(DriverConfiguration.ON_TASK_FAILED, JobDriver.FailedTaskHandler.class)
-        .set(DriverConfiguration.ON_TASK_RUNNING, JobDriver.RunningTaskHandler.class)
-        .set(DriverConfiguration.ON_DRIVER_RESTART_TASK_RUNNING, JobDriver.DriverRestartRunningTaskHandler.class)
-        .set(DriverConfiguration.ON_DRIVER_RESTART_COMPLETED, JobDriver.DriverRestartCompletedHandler.class)
-        .set(DriverConfiguration.ON_TASK_COMPLETED, JobDriver.CompletedTaskHandler.class)
-        .set(DriverConfiguration.ON_DRIVER_STARTED, JobDriver.StartHandler.class)
-        .set(DriverConfiguration.ON_DRIVER_RESTARTED, JobDriver.RestartHandler.class)
-        .set(DriverConfiguration.ON_TASK_SUSPENDED, JobDriver.SuspendedTaskHandler.class)
-        .set(DriverConfiguration.ON_EVALUATOR_COMPLETED, JobDriver.CompletedEvaluatorHandler.class);
-  }
-
-  private static Configuration getNameServerConfiguration() {
-    return NameServerConfiguration.CONF
-        .set(NameServerConfiguration.NAME_SERVICE_PORT, 0)
-        .build();
-  }
-
-  /**
-   * @return the driver-side configuration to be merged into the DriverConfiguration to enable the HTTP server.
-   */
-  public static Configuration getHTTPConfiguration() {
-    Configuration httpHandlerConfiguration = HttpHandlerConfiguration.CONF
-        .set(HttpHandlerConfiguration.HTTP_HANDLERS, HttpServerReefEventHandler.class)
-        .build();
-
-    Configuration driverConfigurationForHttpServer = DriverServiceConfiguration.CONF
-        .set(DriverServiceConfiguration.ON_EVALUATOR_ALLOCATED, ReefEventStateManager.AllocatedEvaluatorStateHandler.class)
-        .set(DriverServiceConfiguration.ON_CONTEXT_ACTIVE, ReefEventStateManager.ActiveContextStateHandler.class)
-        .set(DriverServiceConfiguration.ON_DRIVER_RESTART_CONTEXT_ACTIVE, ReefEventStateManager.DrivrRestartActiveContextStateHandler.class)
-        .set(DriverServiceConfiguration.ON_TASK_RUNNING, ReefEventStateManager.TaskRunningStateHandler.class)
-        .set(DriverServiceConfiguration.ON_DRIVER_RESTART_TASK_RUNNING, ReefEventStateManager.DriverRestartTaskRunningStateHandler.class)
-        .set(DriverServiceConfiguration.ON_DRIVER_STARTED, ReefEventStateManager.StartStateHandler.class)
-        .set(DriverServiceConfiguration.ON_DRIVER_STOP, ReefEventStateManager.StopStateHandler.class)
-        .build();
-    return Configurations.merge(httpHandlerConfiguration, driverConfigurationForHttpServer);
-  }
-
-  public void addCLRFiles(final File folder) throws BindException {
-    try (final LoggingScope ls = this.loggingScopeFactory.getNewLoggingScope("JobClient::addCLRFiles")) {
-      ConfigurationModule result = this.driverConfigModule;
-      for (final File f : folder.listFiles()) {
-        if (f.canRead() && f.exists() && f.isFile()) {
-          result = result.set(DriverConfiguration.GLOBAL_FILES, f.getAbsolutePath());
-        }
-      }
-
-      // set the driver memory, id and job submission directory
-      this.driverConfigModule = result
-          .set(DriverConfiguration.DRIVER_MEMORY, this.driverMemory)
-          .set(DriverConfiguration.DRIVER_IDENTIFIER, this.driverId)
-          .set(DriverConfiguration.DRIVER_JOB_SUBMISSION_DIRECTORY, this.jobSubmissionDirectory);
-
-
-      Path globalLibFile = Paths.get(NativeInterop.GLOBAL_LIBRARIES_FILENAME);
-      if (!Files.exists(globalLibFile)) {
-        LOG.log(Level.FINE, "Cannot find global classpath file at: {0}, assume there is none.", globalLibFile.toAbsolutePath());
-      } else {
-        String globalLibString = "";
-        try {
-          globalLibString = new String(Files.readAllBytes(globalLibFile));
-        } catch (final Exception e) {
-          LOG.log(Level.WARNING, "Cannot read from {0}, global libraries not added  " + globalLibFile.toAbsolutePath());
-        }
-
-        for (final String s : globalLibString.split(",")) {
-          File f = new File(s);
-          this.driverConfigModule = this.driverConfigModule.set(DriverConfiguration.GLOBAL_LIBRARIES, f.getPath());
-        }
-      }
-
-      this.driverConfiguration = Configurations.merge(this.driverConfigModule.build(), getHTTPConfiguration(), getNameServerConfiguration());
-    }
-  }
-
-  /**
-   * Launch the job driver.
-   *
-   * @throws org.apache.reef.tang.exceptions.BindException configuration error.
-   */
-  public void submit(final File clrFolder, final boolean submitDriver, final Configuration clientConfig) {
-    try (final LoggingScope ls = this.loggingScopeFactory.driverSubmit(submitDriver)) {
-      try {
-        addCLRFiles(clrFolder);
-      } catch (final BindException e) {
-        LOG.log(Level.FINE, "Failed to bind", e);
-      }
-      if (submitDriver) {
-        this.reef.submit(this.driverConfiguration);
-      } else {
-        File driverConfig = new File(System.getProperty("user.dir") + "/driver.config");
-        try {
-          new AvroConfigurationSerializer().toFile(Configurations.merge(this.driverConfiguration, clientConfig), driverConfig);
-          LOG.log(Level.INFO, "Driver configuration file created at " + driverConfig.getAbsolutePath());
-        } catch (final IOException e) {
-          throw new RuntimeException("Cannot create driver configuration file at " + driverConfig.getAbsolutePath());
-        }
-      }
-    }
-  }
-
-  /**
-   * Set the driver memory
-   */
-  public void setDriverInfo(final String identifier, final int memory, final String jobSubmissionDirectory) {
-    if (identifier == null || identifier.isEmpty()) {
-      throw new RuntimeException("driver id cannot be null or empty");
-    }
-    if (memory <= 0) {
-      throw new RuntimeException("driver memory cannot be negative number: " + memory);
-    }
-    this.driverMemory = memory;
-    this.driverId = identifier;
-    if (jobSubmissionDirectory != null && !jobSubmissionDirectory.equals("empty")) {
-      this.jobSubmissionDirectory = jobSubmissionDirectory;
-    } else {
-      LOG.log(Level.FINE, "No job submission directory provided by CLR user, will use " + this.jobSubmissionDirectory);
-    }
-  }
-
-  /**
-   * Notify the process in waitForCompletion() method that the main process has finished.
-   */
-  private synchronized void stopAndNotify() {
-    this.runningJob = null;
-    this.isBusy = false;
-    this.notify();
-  }
-
-  /**
-   * Wait for the job driver to complete. This method is called from Launcher.main()
-   */
-  public void waitForCompletion(final int waitTime) {
-    LOG.info("Waiting for the Job Driver to complete: " + waitTime);
-    if (waitTime == 0) {
-      close(0);
-      return;
-    } else if (waitTime < 0) {
-      waitTillDone();
-    }
-    long endTime = System.currentTimeMillis() + waitTime * 1000;
-    close(endTime);
-  }
-
-  public void close(final long endTime) {
-    while (endTime > System.currentTimeMillis()) {
-      try {
-        Thread.sleep(1000);
-      } catch (final InterruptedException e) {
-        LOG.log(Level.SEVERE, "Thread sleep failed");
-      }
-    }
-    LOG.log(Level.INFO, "Done waiting.");
-    this.stopAndNotify();
-    reef.close();
-  }
-
-  private void waitTillDone() {
-    while (this.isBusy) {
-      try {
-        synchronized (this) {
-          this.wait();
-        }
-      } catch (final InterruptedException ex) {
-        LOG.log(Level.WARNING, "Waiting for result interrupted.", ex);
-      }
-    }
-    this.reef.close();
-  }
-
-  /**
-   * Receive notification from the job driver that the job had failed.
-   */
-  final class FailedJobHandler implements EventHandler<FailedJob> {
-    @Override
-    public void onNext(final FailedJob job) {
-      LOG.log(Level.SEVERE, "Failed job: " + job.getId(), job.getMessage());
-      stopAndNotify();
-    }
-  }
-
-  /**
-   * Receive notification from the job driver that the job had completed successfully.
-   */
-  final class CompletedJobHandler implements EventHandler<CompletedJob> {
-    @Override
-    public void onNext(final CompletedJob job) {
-      LOG.log(Level.INFO, "Completed job: {0}", job.getId());
-      stopAndNotify();
-    }
-  }
-
-  /**
-   * Receive notification that there was an exception thrown from the job driver.
-   */
-  final class RuntimeErrorHandler implements EventHandler<FailedRuntime> {
-    @Override
-    public void onNext(final FailedRuntime error) {
-      LOG.log(Level.SEVERE, "Error in job driver: " + error, error.getMessage());
-      stopAndNotify();
-    }
-  }
-
-  final class WakeErrorHandler implements EventHandler<Throwable> {
-    @Override
-    public void onNext(Throwable error) {
-      LOG.log(Level.SEVERE, "Error communicating with job driver, exiting... ", error);
-      stopAndNotify();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java
deleted file mode 100644
index adea40f..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/JobDriver.java
+++ /dev/null
@@ -1,705 +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.javabridge.generic;
-
-import org.apache.reef.driver.client.JobMessageObserver;
-import org.apache.reef.driver.context.ActiveContext;
-import org.apache.reef.driver.context.ClosedContext;
-import org.apache.reef.driver.context.ContextMessage;
-import org.apache.reef.driver.context.FailedContext;
-import org.apache.reef.driver.evaluator.*;
-import org.apache.reef.driver.task.*;
-import org.apache.reef.io.network.naming.NameServer;
-import org.apache.reef.javabridge.*;
-import org.apache.reef.runtime.common.DriverRestartCompleted;
-import org.apache.reef.runtime.common.driver.DriverStatusManager;
-import org.apache.reef.tang.annotations.Unit;
-import org.apache.reef.util.Optional;
-import org.apache.reef.util.logging.CLRBufferedLogHandler;
-import org.apache.reef.util.logging.LoggingScope;
-import org.apache.reef.util.logging.LoggingScopeFactory;
-import org.apache.reef.wake.EventHandler;
-import org.apache.reef.wake.remote.NetUtils;
-import org.apache.reef.wake.remote.impl.ObjectSerializableCodec;
-import org.apache.reef.wake.time.Clock;
-import org.apache.reef.wake.time.event.Alarm;
-import org.apache.reef.wake.time.event.StartTime;
-import org.apache.reef.wake.time.event.StopTime;
-import org.apache.reef.webserver.*;
-
-import javax.inject.Inject;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletResponse;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.logging.Handler;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * Generic job driver for CLRBridge.
- */
-@Unit
-public final class JobDriver {
-
-  private static final Logger LOG = Logger.getLogger(JobDriver.class.getName());
-  /**
-   * String codec is used to encode the results
-   * before passing them back to the client.
-   */
-  private static final ObjectSerializableCodec<String> JVM_CODEC = new ObjectSerializableCodec<>();
-  private final InteropLogger interopLogger = new InteropLogger();
-  private final NameServer nameServer;
-  private final String nameServerInfo;
-  private final HttpServer httpServer;
-  /**
-   * Wake clock is used to schedule periodical job check-ups.
-   */
-  private final Clock clock;
-  /**
-   * Job observer on the client.
-   * We use it to send results from the driver back to the client.
-   */
-  private final JobMessageObserver jobMessageObserver;
-  /**
-   * Job driver uses EvaluatorRequestor
-   * to request Evaluators that will run the Tasks.
-   */
-  private final EvaluatorRequestor evaluatorRequestor;
-  private final DriverStatusManager driverStatusManager;
-  /**
-   * Shell execution results from each Evaluator.
-   */
-  private final List<String> results = new ArrayList<>();
-  /**
-   * Map from context ID to running evaluator context.
-   */
-  private final Map<String, ActiveContext> contexts = new HashMap<>();
-
-  /**
-   * Logging scope factory that provides LoggingScope
-   */
-  private final LoggingScopeFactory loggingScopeFactory;
-
-  private long evaluatorRequestorHandler = 0;
-  private long allocatedEvaluatorHandler = 0;
-  private long activeContextHandler = 0;
-  private long taskMessageHandler = 0;
-  private long failedTaskHandler = 0;
-  private long failedEvaluatorHandler = 0;
-  private long httpServerEventHandler = 0;
-  private long completedTaskHandler = 0;
-  private long runningTaskHandler = 0;
-  private long suspendedTaskHandler = 0;
-  private long completedEvaluatorHandler = 0;
-  private long closedContextHandler = 0;
-  private long failedContextHandler = 0;
-  private long contextMessageHandler = 0;
-  private long driverRestartHandler = 0;
-  private long driverRestartActiveContextHandler = 0;
-  private long driverRestartRunningTaskHandler = 0;
-  private boolean clrBridgeSetup = false;
-  private boolean isRestarted = false;
-
-  /**
-   * Job driver constructor.
-   * All parameters are injected from TANG automatically.
-   *
-   * @param clock              Wake clock to schedule and check up running jobs.
-   * @param jobMessageObserver is used to send messages back to the client.
-   * @param evaluatorRequestor is used to request Evaluators.
-   */
-  @Inject
-  JobDriver(final Clock clock,
-            final HttpServer httpServer,
-            final NameServer nameServer,
-            final JobMessageObserver jobMessageObserver,
-            final EvaluatorRequestor evaluatorRequestor,
-            final DriverStatusManager driverStatusManager,
-            final LoggingScopeFactory loggingScopeFactory) {
-    this.clock = clock;
-    this.httpServer = httpServer;
-    this.jobMessageObserver = jobMessageObserver;
-    this.evaluatorRequestor = evaluatorRequestor;
-    this.nameServer = nameServer;
-    this.driverStatusManager = driverStatusManager;
-    this.nameServerInfo = NetUtils.getLocalAddress() + ":" + this.nameServer.getPort();
-    this.loggingScopeFactory = loggingScopeFactory;
-  }
-
-  private void setupBridge(final StartTime startTime) {
-    // Signal to the clr buffered log handler that the driver has started and that
-    // we can begin logging
-    LOG.log(Level.INFO, "Initializing CLRBufferedLogHandler...");
-    try (final LoggingScope lb = this.loggingScopeFactory.setupBridge()) {
-      final CLRBufferedLogHandler handler = getCLRBufferedLogHandler();
-      if (handler == null) {
-        LOG.log(Level.WARNING, "CLRBufferedLogHandler could not be initialized");
-      } else {
-        handler.setDriverInitialized();
-        LOG.log(Level.INFO, "CLRBufferedLogHandler init complete.");
-      }
-
-      LOG.log(Level.INFO, "StartTime: {0}", new Object[]{startTime});
-      String portNumber = httpServer == null ? null : Integer.toString((httpServer.getPort()));
-      long[] handlers = NativeInterop.CallClrSystemOnStartHandler(startTime.toString(), portNumber);
-      if (handlers != null) {
-        if (handlers.length != NativeInterop.nHandlers) {
-          throw new RuntimeException(
-              String.format("%s handlers initialized in CLR while native bridge is expecting %s handlers",
-                  String.valueOf(handlers.length),
-                  String.valueOf(NativeInterop.nHandlers)));
-        }
-        this.evaluatorRequestorHandler = handlers[NativeInterop.Handlers.get(NativeInterop.EvaluatorRequestorKey)];
-        this.allocatedEvaluatorHandler = handlers[NativeInterop.Handlers.get(NativeInterop.AllocatedEvaluatorKey)];
-        this.activeContextHandler = handlers[NativeInterop.Handlers.get(NativeInterop.ActiveContextKey)];
-        this.taskMessageHandler = handlers[NativeInterop.Handlers.get(NativeInterop.TaskMessageKey)];
-        this.failedTaskHandler = handlers[NativeInterop.Handlers.get(NativeInterop.FailedTaskKey)];
-        this.failedEvaluatorHandler = handlers[NativeInterop.Handlers.get(NativeInterop.FailedEvaluatorKey)];
-        this.httpServerEventHandler = handlers[NativeInterop.Handlers.get(NativeInterop.HttpServerKey)];
-        this.completedTaskHandler = handlers[NativeInterop.Handlers.get(NativeInterop.CompletedTaskKey)];
-        this.runningTaskHandler = handlers[NativeInterop.Handlers.get(NativeInterop.RunningTaskKey)];
-        this.suspendedTaskHandler = handlers[NativeInterop.Handlers.get(NativeInterop.SuspendedTaskKey)];
-        this.completedEvaluatorHandler = handlers[NativeInterop.Handlers.get(NativeInterop.CompletedEvaluatorKey)];
-        this.closedContextHandler = handlers[NativeInterop.Handlers.get(NativeInterop.ClosedContextKey)];
-        this.failedContextHandler = handlers[NativeInterop.Handlers.get(NativeInterop.FailedContextKey)];
-        this.contextMessageHandler = handlers[NativeInterop.Handlers.get(NativeInterop.ContextMessageKey)];
-        this.driverRestartHandler = handlers[NativeInterop.Handlers.get(NativeInterop.DriverRestartKey)];
-        this.driverRestartActiveContextHandler = handlers[NativeInterop.Handlers.get(NativeInterop.DriverRestartActiveContextKey)];
-        this.driverRestartRunningTaskHandler = handlers[NativeInterop.Handlers.get(NativeInterop.DriverRestartRunningTaskKey)];
-      }
-
-      try (final LoggingScope lp = this.loggingScopeFactory.getNewLoggingScope("setupBridge::ClrSystemHttpServerHandlerOnNext")) {
-        final HttpServerEventBridge httpServerEventBridge = new HttpServerEventBridge("SPEC");
-        NativeInterop.ClrSystemHttpServerHandlerOnNext(this.httpServerEventHandler, httpServerEventBridge, this.interopLogger);
-        final String specList = httpServerEventBridge.getUriSpecification();
-        LOG.log(Level.INFO, "Starting http server, getUriSpecification: {0}", specList);
-        if (specList != null) {
-          final String[] specs = specList.split(":");
-          for (final String s : specs) {
-            final HttpHandler h = new HttpServerBridgeEventHandler();
-            h.setUriSpecification(s);
-            this.httpServer.addHttpHandler(h);
-          }
-        }
-      }
-      this.clrBridgeSetup = true;
-    }
-    LOG.log(Level.INFO, "CLR Bridge setup.");
-  }
-
-  private CLRBufferedLogHandler getCLRBufferedLogHandler() {
-    for (Handler handler : Logger.getLogger("").getHandlers()) {
-      if (handler instanceof CLRBufferedLogHandler)
-        return (CLRBufferedLogHandler) handler;
-    }
-    return null;
-  }
-
-  private void submitEvaluator(final AllocatedEvaluator eval, EvaluatorType type) {
-    synchronized (JobDriver.this) {
-      eval.setType(type);
-      LOG.log(Level.INFO, "Allocated Evaluator: {0}, total running running {1}",
-          new Object[]{eval.getId(), JobDriver.this.contexts.size()});
-      if (JobDriver.this.allocatedEvaluatorHandler == 0) {
-        throw new RuntimeException("Allocated Evaluator Handler not initialized by CLR.");
-      }
-      AllocatedEvaluatorBridge allocatedEvaluatorBridge = new AllocatedEvaluatorBridge(eval, JobDriver.this.nameServerInfo);
-      NativeInterop.ClrSystemAllocatedEvaluatorHandlerOnNext(JobDriver.this.allocatedEvaluatorHandler, allocatedEvaluatorBridge, this.interopLogger);
-    }
-  }
-
-  /**
-   * Submit a Task to a single Evaluator.
-   */
-  private void submit(final ActiveContext context) {
-    try {
-      LOG.log(Level.INFO, "Send task to context: {0}", new Object[]{context});
-      if (JobDriver.this.activeContextHandler == 0) {
-        throw new RuntimeException("Active Context Handler not initialized by CLR.");
-      }
-      ActiveContextBridge activeContextBridge = new ActiveContextBridge(context);
-      NativeInterop.ClrSystemActiveContextHandlerOnNext(JobDriver.this.activeContextHandler, activeContextBridge, JobDriver.this.interopLogger);
-    } catch (final Exception ex) {
-      LOG.log(Level.SEVERE, "Fail to submit task to active context");
-      context.close();
-      throw new RuntimeException(ex);
-    }
-  }
-
-  /**
-   * Handles AllocatedEvaluator: Submit an empty context
-   */
-  final class AllocatedEvaluatorHandler implements EventHandler<AllocatedEvaluator> {
-    @Override
-    public void onNext(final AllocatedEvaluator allocatedEvaluator) {
-      try (final LoggingScope ls = loggingScopeFactory.evaluatorAllocated(allocatedEvaluator.getId())) {
-        synchronized (JobDriver.this) {
-          LOG.log(Level.INFO, "AllocatedEvaluatorHandler.OnNext");
-            JobDriver.this.submitEvaluator(allocatedEvaluator, EvaluatorType.CLR);
-        }
-      }
-    }
-  }
-
-  /**
-   * Receive notification that a new Context is available.
-   */
-  final class ActiveContextHandler implements EventHandler<ActiveContext> {
-    @Override
-    public void onNext(final ActiveContext context) {
-      try (final LoggingScope ls = loggingScopeFactory.activeContextReceived(context.getId())) {
-        synchronized (JobDriver.this) {
-          LOG.log(Level.INFO, "ActiveContextHandler: Context available: {0}",
-              new Object[]{context.getId()});
-          JobDriver.this.contexts.put(context.getId(), context);
-          JobDriver.this.submit(context);
-        }
-      }
-    }
-  }
-
-  /**
-   * Receive notification that the Task has completed successfully.
-   */
-  final class CompletedTaskHandler implements EventHandler<CompletedTask> {
-    @Override
-    public void onNext(final CompletedTask task) {
-      LOG.log(Level.INFO, "Completed task: {0}", task.getId());
-      try (final LoggingScope ls = loggingScopeFactory.taskCompleted(task.getId())) {
-        // Take the message returned by the task and add it to the running result.
-        String result = "default result";
-        try {
-          result = new String(task.get());
-        } catch (final Exception e) {
-          LOG.log(Level.WARNING, "failed to decode task outcome");
-        }
-        LOG.log(Level.INFO, "Return results to the client:\n{0}", result);
-        JobDriver.this.jobMessageObserver.sendMessageToClient(JVM_CODEC.encode(result));
-        if (JobDriver.this.completedTaskHandler == 0) {
-          LOG.log(Level.INFO, "No CLR handler bound to handle completed task.");
-        } else {
-          LOG.log(Level.INFO, "CLR CompletedTaskHandler handler set, handling things with CLR handler.");
-          CompletedTaskBridge completedTaskBridge = new CompletedTaskBridge(task);
-          NativeInterop.ClrSystemCompletedTaskHandlerOnNext(JobDriver.this.completedTaskHandler, completedTaskBridge, JobDriver.this.interopLogger);
-        }
-      }
-    }
-  }
-
-  /**
-   * Receive notification that the entire Evaluator had failed.
-   */
-  final class FailedEvaluatorHandler implements EventHandler<FailedEvaluator> {
-    @Override
-    public void onNext(final FailedEvaluator eval) {
-      try (final LoggingScope ls = loggingScopeFactory.evaluatorFailed(eval.getId())) {
-        synchronized (JobDriver.this) {
-          LOG.log(Level.SEVERE, "FailedEvaluator", eval);
-          for (final FailedContext failedContext : eval.getFailedContextList()) {
-            String failedContextId = failedContext.getId();
-            LOG.log(Level.INFO, "removing context " + failedContextId + " from job driver contexts.");
-            JobDriver.this.contexts.remove(failedContextId);
-          }
-          String message = "Evaluator " + eval.getId() + " failed with message: "
-              + eval.getEvaluatorException().getMessage();
-          JobDriver.this.jobMessageObserver.sendMessageToClient(message.getBytes());
-
-          if (failedEvaluatorHandler == 0) {
-            if (JobDriver.this.clrBridgeSetup) {
-              message = "No CLR FailedEvaluator handler was set, exiting now";
-              LOG.log(Level.WARNING, message);
-              JobDriver.this.jobMessageObserver.sendMessageToClient(message.getBytes());
-              return;
-            } else {
-              clock.scheduleAlarm(0, new EventHandler<Alarm>() {
-                @Override
-                public void onNext(final Alarm time) {
-                  if (JobDriver.this.clrBridgeSetup) {
-                    handleFailedEvaluatorInCLR(eval);
-                  } else {
-                    LOG.log(Level.INFO, "Waiting for CLR bridge to be set up");
-                    clock.scheduleAlarm(5000, this);
-                  }
-                }
-              });
-            }
-          } else {
-            handleFailedEvaluatorInCLR(eval);
-          }
-        }
-      }
-    }
-
-    private void handleFailedEvaluatorInCLR(final FailedEvaluator eval) {
-      final String message = "CLR FailedEvaluator handler set, handling things with CLR handler.";
-      LOG.log(Level.INFO, message);
-      FailedEvaluatorBridge failedEvaluatorBridge = new FailedEvaluatorBridge(eval, JobDriver.this.evaluatorRequestor, JobDriver.this.isRestarted, loggingScopeFactory);
-      NativeInterop.ClrSystemFailedEvaluatorHandlerOnNext(JobDriver.this.failedEvaluatorHandler, failedEvaluatorBridge, JobDriver.this.interopLogger);
-      int additionalRequestedEvaluatorNumber = failedEvaluatorBridge.getNewlyRequestedEvaluatorNumber();
-      if (additionalRequestedEvaluatorNumber > 0) {
-        LOG.log(Level.INFO, "number of additional evaluators requested after evaluator failure: " + additionalRequestedEvaluatorNumber);
-      }
-      JobDriver.this.jobMessageObserver.sendMessageToClient(message.getBytes());
-    }
-  }
-
-  final class HttpServerBridgeEventHandler implements HttpHandler {
-    private String uriSpecification;
-
-    /**
-     * returns URI specification for the handler
-     */
-    @Override
-    public String getUriSpecification() {
-      return uriSpecification;
-    }
-
-    public void setUriSpecification(String s) {
-      uriSpecification = s;
-    }
-
-    /**
-     * process http request
-     */
-    @Override
-    public void onHttpRequest(final ParsedHttpRequest parsedHttpRequest, final HttpServletResponse response) throws IOException, ServletException {
-      LOG.log(Level.INFO, "HttpServerBridgeEventHandler onHttpRequest: {0}", parsedHttpRequest.getRequestUri());
-      try (final LoggingScope ls = loggingScopeFactory.httpRequest(parsedHttpRequest.getRequestUri())) {
-        final AvroHttpSerializer httpSerializer = new AvroHttpSerializer();
-        final AvroHttpRequest avroHttpRequest = httpSerializer.toAvro(parsedHttpRequest);
-        final byte[] requestBytes = httpSerializer.toBytes(avroHttpRequest);
-
-        try {
-          final HttpServerEventBridge httpServerEventBridge = new HttpServerEventBridge(requestBytes);
-          NativeInterop.ClrSystemHttpServerHandlerOnNext(JobDriver.this.httpServerEventHandler, httpServerEventBridge, JobDriver.this.interopLogger);
-          final String responseBody = new String(httpServerEventBridge.getQueryResponseData(), "UTF-8");
-          response.getWriter().println(responseBody);
-          LOG.log(Level.INFO, "HttpServerBridgeEventHandler onHttpRequest received response: {0}", responseBody);
-        } catch (final Exception ex) {
-          LOG.log(Level.SEVERE, "Fail to invoke CLR Http Server handler", ex);
-          throw new RuntimeException(ex);
-        }
-      }
-    }
-  }
-
-  /**
-   * Handle failed task.
-   */
-  final class FailedTaskHandler implements EventHandler<FailedTask> {
-    @Override
-    public void onNext(final FailedTask task) throws RuntimeException {
-      LOG.log(Level.SEVERE, "FailedTask received, will be handle in CLR handler, if set.");
-      if (JobDriver.this.failedTaskHandler == 0) {
-        LOG.log(Level.SEVERE, "Failed Task Handler not initialized by CLR, fail for real.");
-        throw new RuntimeException("Failed Task Handler not initialized by CLR.");
-      }
-      try {
-        FailedTaskBridge failedTaskBridge = new FailedTaskBridge(task);
-        NativeInterop.ClrSystemFailedTaskHandlerOnNext(JobDriver.this.failedTaskHandler, failedTaskBridge, JobDriver.this.interopLogger);
-      } catch (final Exception ex) {
-        LOG.log(Level.SEVERE, "Fail to invoke CLR failed task handler");
-        throw new RuntimeException(ex);
-      }
-    }
-  }
-
-  /**
-   * Receive notification that the Task is running.
-   */
-  final class RunningTaskHandler implements EventHandler<RunningTask> {
-    @Override
-    public void onNext(final RunningTask task) {
-      try (final LoggingScope ls = loggingScopeFactory.taskRunning(task.getId())) {
-        if (JobDriver.this.runningTaskHandler == 0) {
-          LOG.log(Level.INFO, "RunningTask event received but no CLR handler was bound. Exiting handler.");
-        } else {
-          LOG.log(Level.INFO, "RunningTask will be handled by CLR handler. Task Id: {0}", task.getId());
-          try {
-            final RunningTaskBridge runningTaskBridge = new RunningTaskBridge(task);
-            NativeInterop.ClrSystemRunningTaskHandlerOnNext(JobDriver.this.runningTaskHandler, runningTaskBridge, JobDriver.this.interopLogger);
-          } catch (final Exception ex) {
-            LOG.log(Level.WARNING, "Fail to invoke CLR running task handler");
-            throw new RuntimeException(ex);
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Receive notification that the Task is running when driver restarted.
-   */
-  final class DriverRestartRunningTaskHandler implements EventHandler<RunningTask> {
-    @Override
-    public void onNext(final RunningTask task) {
-      try (final LoggingScope ls = loggingScopeFactory.driverRestartRunningTask(task.getId())) {
-        clock.scheduleAlarm(0, new EventHandler<Alarm>() {
-          @Override
-          public void onNext(final Alarm time) {
-            if (JobDriver.this.clrBridgeSetup) {
-              if (JobDriver.this.driverRestartRunningTaskHandler != 0) {
-                LOG.log(Level.INFO, "CLR driver restart RunningTask handler implemented, now handle it in CLR.");
-                NativeInterop.ClrSystemDriverRestartRunningTaskHandlerOnNext(JobDriver.this.driverRestartRunningTaskHandler, new RunningTaskBridge(task));
-              } else {
-                LOG.log(Level.WARNING, "No CLR driver restart RunningTask handler implemented, done with DriverRestartRunningTaskHandler.");
-              }
-            } else {
-              LOG.log(Level.INFO, "Waiting for driver to complete restart process before checking out CLR driver restart RunningTaskHandler...");
-              clock.scheduleAlarm(2000, this);
-            }
-          }
-        });
-      }
-    }
-  }
-
-  /**
-   * Receive notification that an context is active on Evaluator when the driver restarted
-   */
-  final class DriverRestartActiveContextHandler implements EventHandler<ActiveContext> {
-    @Override
-    public void onNext(final ActiveContext context) {
-      try (final LoggingScope ls = loggingScopeFactory.driverRestartActiveContextReceived(context.getId())) {
-        JobDriver.this.contexts.put(context.getId(), context);
-      LOG.log(Level.INFO, "DriverRestartActiveContextHandler event received: " + context.getId());
-        clock.scheduleAlarm(0, new EventHandler<Alarm>() {
-          @Override
-          public void onNext(final Alarm time) {
-            if (JobDriver.this.clrBridgeSetup) {
-              if (JobDriver.this.driverRestartActiveContextHandler != 0) {
-                LOG.log(Level.INFO, "CLR driver restart ActiveContext handler implemented, now handle it in CLR.");
-                NativeInterop.ClrSystemDriverRestartActiveContextHandlerOnNext(JobDriver.this.driverRestartActiveContextHandler, new ActiveContextBridge(context));
-              } else {
-                LOG.log(Level.WARNING, "No CLR driver restart ActiveContext handler implemented, done with DriverRestartActiveContextHandler.");
-              }
-            } else {
-              LOG.log(Level.INFO, "Waiting for driver to complete restart process before checking out CLR driver restart DriverRestartActiveContextHandler...");
-              clock.scheduleAlarm(2000, this);
-            }
-          }
-        });
-      }
-    }
-  }
-
-  /**
-   * Job Driver is ready and the clock is set up: request the evaluators.
-   */
-  final class StartHandler implements EventHandler<StartTime> {
-    @Override
-    public void onNext(final StartTime startTime) {
-      try (final LoggingScope ls = loggingScopeFactory.driverStart(startTime)) {
-        synchronized (JobDriver.this) {
-
-          setupBridge(startTime);
-
-          LOG.log(Level.INFO, "Driver Started");
-
-          if (JobDriver.this.evaluatorRequestorHandler == 0) {
-            throw new RuntimeException("Evaluator Requestor Handler not initialized by CLR.");
-          }
-          EvaluatorRequestorBridge evaluatorRequestorBridge = new EvaluatorRequestorBridge(JobDriver.this.evaluatorRequestor, false, loggingScopeFactory);
-          NativeInterop.ClrSystemEvaluatorRequstorHandlerOnNext(JobDriver.this.evaluatorRequestorHandler, evaluatorRequestorBridge, JobDriver.this.interopLogger);
-          // get the evaluator numbers set by CLR handler
-          LOG.log(Level.INFO, "evaluator requested at start up: " + evaluatorRequestorBridge.getEvaluatorNumber());
-        }
-      }
-    }
-  }
-
-
-  /**
-   * Job driver is restarted after previous crash
-   */
-  final class RestartHandler implements EventHandler<StartTime> {
-    @Override
-    public void onNext(final StartTime startTime) {
-      try (final LoggingScope ls = loggingScopeFactory.driverRestart(startTime)) {
-        synchronized (JobDriver.this) {
-
-          setupBridge(startTime);
-
-          JobDriver.this.isRestarted = true;
-
-          LOG.log(Level.INFO, "Driver Restarted and CLR bridge set up.");
-        }
-      }
-    }
-  }
-
-  /**
-   * Receive notification that driver restart has completed.
-   */
-  final class DriverRestartCompletedHandler implements EventHandler<DriverRestartCompleted> {
-    @Override
-    public void onNext(final DriverRestartCompleted driverRestartCompleted) {
-      LOG.log(Level.INFO, "Java DriverRestartCompleted event received at time [{0}]. ", driverRestartCompleted.getTimeStamp());
-      try (final LoggingScope ls = loggingScopeFactory.driverRestartCompleted(driverRestartCompleted.getTimeStamp())) {
-        if (JobDriver.this.driverRestartHandler != 0) {
-          LOG.log(Level.INFO, "CLR driver restart handler implemented, now handle it in CLR.");
-          NativeInterop.ClrSystemDriverRestartHandlerOnNext(JobDriver.this.driverRestartHandler);
-        } else {
-          LOG.log(Level.WARNING, "No CLR driver restart handler implemented, done with DriverRestartCompletedHandler.");
-
-        }
-      }
-    }
-  }
-
-  /**
-   * Shutting down the job driver: close the evaluators.
-   */
-  final class StopHandler implements EventHandler<StopTime> {
-    @Override
-    public void onNext(final StopTime time) {
-      LOG.log(Level.INFO, " StopTime: {0}", new Object[]{time});
-      try (final LoggingScope ls = loggingScopeFactory.driverStop(time.getTimeStamp())) {
-        for (final ActiveContext context : contexts.values()) {
-          context.close();
-        }
-      }
-    }
-  }
-
-  final class TaskMessageHandler implements EventHandler<TaskMessage> {
-    @Override
-    public void onNext(final TaskMessage taskMessage) {
-      String msg = new String(taskMessage.get());
-      LOG.log(Level.INFO, "Received TaskMessage: {0} from CLR", msg);
-      //try (LoggingScope ls = loggingScopeFactory.taskMessageReceived(new String(msg))) {
-      if (JobDriver.this.taskMessageHandler != 0) {
-        TaskMessageBridge taskMessageBridge = new TaskMessageBridge(taskMessage);
-        // if CLR implements the task message handler, handle the bytes in CLR handler
-        NativeInterop.ClrSystemTaskMessageHandlerOnNext(JobDriver.this.taskMessageHandler, taskMessage.get(), taskMessageBridge, JobDriver.this.interopLogger);
-      }
-      //}
-    }
-  }
-
-  /**
-   * Receive notification that the Task has been suspended.
-   */
-  final class SuspendedTaskHandler implements EventHandler<SuspendedTask> {
-    @Override
-    public final void onNext(final SuspendedTask task) {
-      final String message = "Received notification that task [" + task.getId() + "] has been suspended.";
-      LOG.log(Level.INFO, message);
-      try (final LoggingScope ls = loggingScopeFactory.taskSuspended(task.getId())) {
-        if (JobDriver.this.suspendedTaskHandler != 0) {
-          SuspendedTaskBridge suspendedTaskBridge = new SuspendedTaskBridge(task);
-          // if CLR implements the suspended task handler, handle it in CLR
-          LOG.log(Level.INFO, "Handling the event of suspended task in CLR bridge.");
-          NativeInterop.ClrSystemSupendedTaskHandlerOnNext(JobDriver.this.suspendedTaskHandler, suspendedTaskBridge);
-        }
-        JobDriver.this.jobMessageObserver.sendMessageToClient(JVM_CODEC.encode(message));
-      }
-    }
-  }
-
-  /**
-   * Receive notification that the Evaluator has been shut down.
-   */
-  final class CompletedEvaluatorHandler implements EventHandler<CompletedEvaluator> {
-    @Override
-    public void onNext(final CompletedEvaluator evaluator) {
-      LOG.log(Level.INFO, " Completed Evaluator {0}", evaluator.getId());
-      try (final LoggingScope ls = loggingScopeFactory.evaluatorCompleted(evaluator.getId())) {
-        if (JobDriver.this.completedEvaluatorHandler != 0) {
-          CompletedEvaluatorBridge completedEvaluatorBridge = new CompletedEvaluatorBridge(evaluator);
-          // if CLR implements the completed evaluator handler, handle it in CLR
-          LOG.log(Level.INFO, "Handling the event of completed evaluator in CLR bridge.");
-          NativeInterop.ClrSystemCompletdEvaluatorHandlerOnNext(completedEvaluatorHandler, completedEvaluatorBridge);
-        }
-      }
-    }
-  }
-
-
-  /**
-   * Receive notification that the Context had completed.
-   * Remove context from the list of active context.
-   */
-  final class ClosedContextHandler implements EventHandler<ClosedContext> {
-    @Override
-    public void onNext(final ClosedContext context) {
-      LOG.log(Level.INFO, "Completed Context: {0}", context.getId());
-      try (final LoggingScope ls = loggingScopeFactory.closedContext(context.getId())) {
-        if (JobDriver.this.closedContextHandler != 0) {
-          ClosedContextBridge closedContextBridge = new ClosedContextBridge(context);
-          // if CLR implements the closed context handler, handle it in CLR
-          LOG.log(Level.INFO, "Handling the event of closed context in CLR bridge.");
-          NativeInterop.ClrSystemClosedContextHandlerOnNext(JobDriver.this.closedContextHandler, closedContextBridge);
-        }
-        synchronized (JobDriver.this) {
-          JobDriver.this.contexts.remove(context.getId());
-        }
-      }
-    }
-  }
-
-
-  /**
-   * Receive notification that the Context had failed.
-   * Remove context from the list of active context and notify the client.
-   */
-  final class FailedContextHandler implements EventHandler<FailedContext> {
-    @Override
-    public void onNext(final FailedContext context) {
-      LOG.log(Level.SEVERE, "FailedContext", context);
-      try (final LoggingScope ls = loggingScopeFactory.evaluatorFailed(context.getId())) {
-        if (JobDriver.this.failedContextHandler != 0) {
-          FailedContextBridge failedContextBridge = new FailedContextBridge(context);
-          // if CLR implements the failed context handler, handle it in CLR
-          LOG.log(Level.INFO, "Handling the event of failed context in CLR bridge.");
-          NativeInterop.ClrSystemFailedContextHandlerOnNext(JobDriver.this.failedContextHandler, failedContextBridge);
-        }
-        synchronized (JobDriver.this) {
-          JobDriver.this.contexts.remove(context.getId());
-        }
-        Optional<byte[]> err = context.getData();
-        if (err.isPresent()) {
-          JobDriver.this.jobMessageObserver.sendMessageToClient(err.get());
-        }
-      }
-    }
-  }
-
-  /**
-   * Receive notification that a ContextMessage has been received
-   */
-  final class ContextMessageHandler implements EventHandler<ContextMessage> {
-    @Override
-    public void onNext(final ContextMessage message) {
-      LOG.log(Level.SEVERE, "Received ContextMessage:", message.get());
-      try (final LoggingScope ls = loggingScopeFactory.contextMessageReceived(message.get().toString())) {
-        if (JobDriver.this.contextMessageHandler != 0) {
-          ContextMessageBridge contextMessageBridge = new ContextMessageBridge(message);
-          // if CLR implements the context message handler, handle it in CLR
-          LOG.log(Level.INFO, "Handling the event of context message in CLR bridge.");
-          NativeInterop.ClrSystemContextMessageHandlerOnNext(JobDriver.this.contextMessageHandler, contextMessageBridge);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/Launch.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/Launch.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/Launch.java
deleted file mode 100644
index b1473ee..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/Launch.java
+++ /dev/null
@@ -1,236 +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.javabridge.generic;
-
-import org.apache.reef.client.ClientConfiguration;
-import org.apache.reef.runtime.local.client.LocalRuntimeConfiguration;
-import org.apache.reef.runtime.yarn.client.YarnClientConfiguration;
-import org.apache.reef.tang.Configuration;
-import org.apache.reef.tang.Injector;
-import org.apache.reef.tang.JavaConfigurationBuilder;
-import org.apache.reef.tang.Tang;
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-import org.apache.reef.tang.exceptions.BindException;
-import org.apache.reef.tang.exceptions.InjectionException;
-import org.apache.reef.tang.formats.CommandLine;
-import org.apache.reef.util.logging.LoggingScope;
-import org.apache.reef.util.logging.LoggingScopeFactory;
-import org.apache.reef.util.logging.LoggingScopeImpl;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * Clr Bridge example - main class.
- */
-public final class Launch {
-
-  /**
-   * Number of REEF worker threads in local mode. We assume maximum 10 evaluators can be requested on local runtime
-   */
-  private static final int NUM_LOCAL_THREADS = 10;
-  /**
-   * Standard Java logger
-   */
-  private static final Logger LOG = Logger.getLogger(Launch.class.getName());
-
-  /**
-   * This class should not be instantiated.
-   */
-  private Launch() {
-    throw new RuntimeException("Do not instantiate this class!");
-  }
-
-  /**
-   * Parse the command line arguments.
-   *
-   * @param args command line arguments, as passed to main()
-   * @return Configuration object.
-   * @throws org.apache.reef.tang.exceptions.BindException configuration error.
-   * @throws java.io.IOException                           error reading the configuration.
-   */
-  private static Configuration parseCommandLine(final String[] args)
-      throws BindException, IOException {
-    final JavaConfigurationBuilder confBuilder = Tang.Factory.getTang().newConfigurationBuilder();
-    final CommandLine cl = new CommandLine(confBuilder);
-    cl.registerShortNameOfClass(Local.class);
-    cl.registerShortNameOfClass(NumRuns.class);
-    cl.registerShortNameOfClass(WaitTimeForDriver.class);
-    cl.registerShortNameOfClass(DriverMemoryInMb.class);
-    cl.registerShortNameOfClass(DriverIdentifier.class);
-    cl.registerShortNameOfClass(DriverJobSubmissionDirectory.class);
-    cl.registerShortNameOfClass(Submit.class);
-    cl.processCommandLine(args);
-    return confBuilder.build();
-  }
-
-  private static Configuration cloneCommandLineConfiguration(final Configuration commandLineConf)
-      throws InjectionException, BindException {
-    final Injector injector = Tang.Factory.getTang().newInjector(commandLineConf);
-    final JavaConfigurationBuilder cb = Tang.Factory.getTang().newConfigurationBuilder();
-    cb.bindNamedParameter(NumRuns.class, String.valueOf(injector.getNamedInstance(NumRuns.class)));
-    return cb.build();
-  }
-
-  /**
-   * Parse command line arguments and create TANG configuration ready to be submitted to REEF.
-   *
-   * @param args Command line arguments, as passed into main().
-   * @return (immutable) TANG Configuration object.
-   * @throws org.apache.reef.tang.exceptions.BindException      if configuration commandLineInjector fails.
-   * @throws org.apache.reef.tang.exceptions.InjectionException if configuration commandLineInjector fails.
-   * @throws java.io.IOException                                error reading the configuration.
-   */
-  private static Configuration getClientConfiguration(final String[] args)
-      throws BindException, InjectionException, IOException {
-
-    try (final LoggingScope ls = LoggingScopeFactory.getNewLoggingScope(Level.INFO, "Launch::getClientConfiguration")) {
-      final Configuration commandLineConf = parseCommandLine(args);
-
-      final Configuration clientConfiguration = ClientConfiguration.CONF
-          .set(ClientConfiguration.ON_JOB_COMPLETED, JobClient.CompletedJobHandler.class)
-          .set(ClientConfiguration.ON_JOB_FAILED, JobClient.FailedJobHandler.class)
-          .set(ClientConfiguration.ON_RUNTIME_ERROR, JobClient.RuntimeErrorHandler.class)
-          //.set(ClientConfiguration.ON_WAKE_ERROR, JobClient.WakeErrorHandler.class )
-          .build();
-
-      // TODO: Remove the injector, have stuff injected.
-      final Injector commandLineInjector = Tang.Factory.getTang().newInjector(commandLineConf);
-      final boolean isLocal = commandLineInjector.getNamedInstance(Local.class);
-      final Configuration runtimeConfiguration;
-      if (isLocal) {
-        LOG.log(Level.INFO, "Running on the local runtime");
-        runtimeConfiguration = LocalRuntimeConfiguration.CONF
-            .set(LocalRuntimeConfiguration.NUMBER_OF_THREADS, NUM_LOCAL_THREADS)
-            .build();
-      } else {
-        LOG.log(Level.INFO, "Running on YARN");
-        runtimeConfiguration = YarnClientConfiguration.CONF.build();
-      }
-
-      return Tang.Factory.getTang()
-          .newConfigurationBuilder(runtimeConfiguration, clientConfiguration,
-              cloneCommandLineConfiguration(commandLineConf))
-          .build();
-    }
-  }
-
-  /**
-   * Main method that starts the CLR Bridge from Java
-   *
-   * @param args command line parameters.
-   */
-  public static void main(final String[] args) {
-    LOG.log(Level.INFO, "Entering Launch at :::" + new Date());
-    try {
-      if (args == null || args.length == 0) {
-        throw new IllegalArgumentException("No arguments provided, at least a clrFolder should be supplied.");
-      }
-      final File dotNetFolder = new File(args[0]).getAbsoluteFile();
-      String[] removedArgs = Arrays.copyOfRange(args, 1, args.length);
-
-      final Configuration config = getClientConfiguration(removedArgs);
-      final Injector commandLineInjector = Tang.Factory.getTang().newInjector(parseCommandLine(removedArgs));
-      final int waitTime = commandLineInjector.getNamedInstance(WaitTimeForDriver.class);
-      final int driverMemory = commandLineInjector.getNamedInstance(DriverMemoryInMb.class);
-      final String driverIdentifier = commandLineInjector.getNamedInstance(DriverIdentifier.class);
-      final String jobSubmissionDirectory = commandLineInjector.getNamedInstance(DriverJobSubmissionDirectory.class);
-      final boolean submit = commandLineInjector.getNamedInstance(Submit.class);
-      final Injector injector = Tang.Factory.getTang().newInjector(config);
-      final JobClient client = injector.getInstance(JobClient.class);
-      client.setDriverInfo(driverIdentifier, driverMemory, jobSubmissionDirectory);
-
-      if (submit) {
-        client.submit(dotNetFolder, true, null);
-        client.waitForCompletion(waitTime);
-      } else {
-        client.submit(dotNetFolder, false, config);
-        client.waitForCompletion(0);
-      }
-
-
-      LOG.info("Done!");
-    } catch (final BindException | InjectionException | IOException ex) {
-      LOG.log(Level.SEVERE, "Job configuration error", ex);
-    }
-  }
-
-  /**
-   * Command line parameter: number of experiments to run.
-   */
-  @NamedParameter(doc = "Number of times to run the command",
-      short_name = "num_runs", default_value = "1")
-  public static final class NumRuns implements Name<Integer> {
-  }
-
-  /**
-   * Command line parameter = true to run locally, or false to run on YARN.
-   */
-  @NamedParameter(doc = "Whether or not to run on the local runtime",
-      short_name = "local", default_value = "true")
-  public static final class Local implements Name<Boolean> {
-  }
-
-  /**
-   * Command line parameter, number of seconds  to wait till driver finishes ,
-   * = -1 : waits forever
-   * = 0: exit immediately without wait for driver.
-   */
-  @NamedParameter(doc = "Whether or not to wait for driver to finish",
-      short_name = "wait_time", default_value = "-1")
-  public static final class WaitTimeForDriver implements Name<Integer> {
-  }
-
-  /**
-   * Command line parameter, driver memory, in MB
-   */
-  @NamedParameter(doc = "memory allocated to driver JVM",
-      short_name = "driver_memory", default_value = "512")
-  public static final class DriverMemoryInMb implements Name<Integer> {
-  }
-
-  /**
-   * Command line parameter, driver identifier
-   */
-  @NamedParameter(doc = "driver identifier for clr bridge",
-      short_name = "driver_id", default_value = "ReefClrBridge")
-  public static final class DriverIdentifier implements Name<String> {
-  }
-
-  /**
-   * Command line parameter = true to submit the job with driver config, or false to write config to current directory
-   */
-  @NamedParameter(doc = "Whether or not to submit the reef job after driver config is constructed",
-      short_name = "submit", default_value = "true")
-  public static final class Submit implements Name<Boolean> {
-  }
-
-  /**
-   * Command line parameter, job submission directory, if set, user should guarantee its uniqueness
-   */
-  @NamedParameter(doc = "driver job submission directory",
-      short_name = "submission_directory", default_value = "empty")
-  public static final class DriverJobSubmissionDirectory implements Name<String> {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/LaunchHeadless.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/LaunchHeadless.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/LaunchHeadless.java
deleted file mode 100644
index ba2a5cb..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/LaunchHeadless.java
+++ /dev/null
@@ -1,100 +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.javabridge.generic;
-
-import org.apache.reef.client.DriverConfiguration;
-import org.apache.reef.client.REEF;
-import org.apache.reef.runtime.common.client.REEFImplementation;
-import org.apache.reef.runtime.yarn.client.YarnClientConfiguration;
-import org.apache.reef.tang.Configuration;
-import org.apache.reef.tang.Configurations;
-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.ConfigurationModule;
-
-import java.io.File;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-/**
- * Clr Bridge example - main class.
- */
-public final class LaunchHeadless {
-
-  /**
-   * Standard Java logger
-   */
-  private static final Logger LOG = Logger.getLogger(LaunchHeadless.class.getName());
-
-  /**
-   * This class should not be instantiated.
-   */
-  private LaunchHeadless() {
-    throw new RuntimeException("Do not instantiate this class!");
-  }
-
-
-  /**
-   * Parse command line arguments and create TANG configuration ready to be submitted to REEF.
-   *
-   * @param args Command line arguments, as passed into main().
-   * @return (immutable) TANG Configuration object.
-   * @throws org.apache.reef.tang.exceptions.BindException      if configuration commandLineInjector fails.
-   * @throws org.apache.reef.tang.exceptions.InjectionException if configuration commandLineInjector fails.
-   * @throws java.io.IOException        error reading the configuration.
-   */
-
-  /**
-   * Main method that starts the CLR Bridge from Java
-   *
-   * @param args command line parameters.
-   */
-  public static void main(final String[] args) {
-    try {
-      if (args == null || args.length == 0) {
-        throw new IllegalArgumentException("No arguments provided, at least a clrFolder should be supplied.");
-      }
-      final File dotNetFolder = new File(args[0]).getAbsoluteFile();
-
-      ConfigurationModule driverConfigModule = JobClient.getDriverConfiguration();
-
-      ConfigurationModule result = driverConfigModule;
-      for (final File f : dotNetFolder.listFiles()) {
-        if (f.canRead() && f.exists() && f.isFile()) {
-          result = result.set(DriverConfiguration.GLOBAL_FILES, f.getAbsolutePath());
-        }
-      }
-
-      driverConfigModule = result;
-      Configuration driverConfiguration = Configurations.merge(driverConfigModule.build(), JobClient.getHTTPConfiguration());
-
-      LOG.log(Level.INFO, "Running on YARN");
-
-      final Configuration runtimeConfiguration = YarnClientConfiguration.CONF.build();
-
-      final REEF reef = Tang.Factory.getTang().newInjector(runtimeConfiguration).getInstance(REEFImplementation.class);
-      reef.submit(driverConfiguration);
-
-      LOG.info("Done!");
-    } catch (final BindException | InjectionException ex) {
-      LOG.log(Level.SEVERE, "Job configuration error", ex);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/package-info.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/package-info.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/package-info.java
deleted file mode 100644
index d93f6f4..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/generic/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.
- */
-/**
- * Generic java bridge driver/client
- */
-package org.apache.reef.javabridge.generic;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/CLRBufferedLogHandler.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/CLRBufferedLogHandler.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/CLRBufferedLogHandler.java
deleted file mode 100644
index 46629c9..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/CLRBufferedLogHandler.java
+++ /dev/null
@@ -1,167 +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.util.logging;
-
-import org.apache.reef.javabridge.NativeInterop;
-
-import javax.inject.Inject;
-import java.util.ArrayList;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.logging.Handler;
-import java.util.logging.Level;
-import java.util.logging.LogRecord;
-import java.util.logging.SimpleFormatter;
-
-/**
- * Logging Handler to intercept java logs and transfer them
- * to the CLR side via the reef-bridge.
- * <p/>
- * Logs are buffered to avoid the cost of reef-bridge function calls.
- * A thread is also scheduled to flush the log buffer at a certain interval,
- * in case the log buffer remains unfilled for an extended period of time.
- */
-public class CLRBufferedLogHandler extends Handler {
-  private static final int BUFFER_LEN = 10;
-  private static final int NUM_THREADS = 1;
-  private static final long LOG_SCHEDULE_PERIOD = 15;  // seconds
-  private SimpleFormatter formatter;
-  private ArrayList<LogRecord> logs;
-  private boolean driverInitialized;
-  private ScheduledThreadPoolExecutor logScheduler;
-
-  @Inject
-  public CLRBufferedLogHandler() {
-    super();
-    this.formatter = new SimpleFormatter();
-    this.logs = new ArrayList<LogRecord>();
-    this.driverInitialized = false;
-    this.logScheduler = new ScheduledThreadPoolExecutor(NUM_THREADS);
-  }
-
-  /**
-   * Signals the java-bridge has been initialized and that we can begin logging.
-   * Usually called from the StartHandler after the driver is up.
-   */
-  public void setDriverInitialized() {
-    synchronized (this) {
-      this.driverInitialized = true;
-    }
-    startLogScheduler();
-  }
-
-  /**
-   * Called whenever a log message is received on the java side.
-   * <p/>
-   * Adds the log record to the log buffer. If the log buffer is full and
-   * the driver has already been initialized, flush the buffer of the logs.
-   */
-  @Override
-  public void publish(LogRecord record) {
-    if (record == null)
-      return;
-
-    if (!isLoggable(record))
-      return;
-
-    synchronized (this) {
-      this.logs.add(record);
-      if (!this.driverInitialized || this.logs.size() < BUFFER_LEN)
-        return;
-    }
-
-    logAll();
-  }
-
-  @Override
-  public void flush() {
-    logAll();
-  }
-
-  /**
-   * Flushes the remaining buffered logs and shuts down the log scheduler thread.
-   */
-  @Override
-  public synchronized void close() throws SecurityException {
-    if (driverInitialized) {
-      this.logAll();
-    }
-    this.logScheduler.shutdown();
-  }
-
-  /**
-   * Starts a thread to flush the log buffer on an interval.
-   * <p/>
-   * This will ensure that logs get flushed periodically, even
-   * if the log buffer is not full.
-   */
-  private void startLogScheduler() {
-    this.logScheduler.scheduleAtFixedRate(
-        new Runnable() {
-          @Override
-          public void run() {
-            CLRBufferedLogHandler.this.logAll();
-          }
-        }, 0, LOG_SCHEDULE_PERIOD, TimeUnit.SECONDS);
-  }
-
-  /**
-   * Flushes the log buffer, logging each buffered log message using
-   * the reef-bridge log function.
-   */
-  private void logAll() {
-    synchronized (this) {
-      final StringBuilder sb = new StringBuilder();
-      Level highestLevel = Level.FINEST;
-      for (final LogRecord record : this.logs) {
-        sb.append(formatter.format(record));
-        sb.append("\n");
-        if (record.getLevel().intValue() > highestLevel.intValue()) {
-          highestLevel = record.getLevel();
-        }
-      }
-      try {
-        final int level = getLevel(highestLevel);
-        NativeInterop.ClrBufferedLog(level, sb.toString());
-      } catch (Exception e) {
-        System.err.println("Failed to perform CLRBufferedLogHandler");
-      }
-
-      this.logs.clear();
-    }
-  }
-
-  /**
-   * Returns the integer value of the log record's level to be used
-   * by the CLR Bridge log function.
-   */
-  private int getLevel(Level recordLevel) {
-    if (recordLevel.equals(Level.OFF)) {
-      return 0;
-    } else if (recordLevel.equals(Level.SEVERE)) {
-      return 1;
-    } else if (recordLevel.equals(Level.WARNING)) {
-      return 2;
-    } else if (recordLevel.equals(Level.ALL)) {
-      return 4;
-    } else {
-      return 3;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/CLRLoggingConfig.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/CLRLoggingConfig.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/CLRLoggingConfig.java
deleted file mode 100644
index 7d82937..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/CLRLoggingConfig.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.util.logging;
-
-import java.io.IOException;
-import java.util.logging.LogManager;
-
-public final class CLRLoggingConfig {
-
-  public CLRLoggingConfig() throws IOException {
-    LogManager.getLogManager().readConfiguration(
-        Thread.currentThread().getContextClassLoader()
-            .getResourceAsStream("com/microsoft/reef/clr.logging.properties"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/package-info.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/package-info.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/package-info.java
deleted file mode 100644
index e0e79ce..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/util/logging/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.
- */
-/**
- * Logging handler for clr bridge
- */
-package org.apache.reef.util.logging;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/resources/org/apache/reef/clr.logging.properties
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/resources/org/apache/reef/clr.logging.properties b/reef-bridge-project/reef-bridge-java/src/main/resources/org/apache/reef/clr.logging.properties
deleted file mode 100644
index 41c4024..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/resources/org/apache/reef/clr.logging.properties
+++ /dev/null
@@ -1,82 +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.
-#
-
-# Properties file which configures the operation of the JDK
-# logging facility.
-
-# The system will look for this config file, first using
-# a System property specified at startup:
-#
-# >java -Djava.utils.logging.config.file=myLoggingConfigFilePath
-#
-# If this property is not specified, then the config file is
-# retrieved from its default location at:
-#
-# JDK_HOME/jre/lib/logging.properties
-
-# Global logging properties.
-# ------------------------------------------
-# The set of handlers to be loaded upon startup.
-# Comma-separated list of class names.
-# (? LogManager docs say no comma here, but JDK example has comma.)
-# handlers=java.utils.logging.FileHandler, java.utils.logging.ConsoleHandler
-handlers=java.util.logging.ConsoleHandler,org.apache.reef.util.logging.CLRBufferedLogHandler
-
-java.util.logging.SimpleFormatter.format=%1$tF %1$tT,%1$tL %4$s %2$s - %5$s%6$s%n
-
-# Default global logging level.
-# Loggers and Handlers may override this level
-.level=ALL
-
-# Loggers
-# ------------------------------------------
-# Loggers are usually attached to packages.
-# Here, the level for each package is specified.
-# The global level is used by default, so levels
-# specified here simply act as an override.
-
-# org.apache.reef.examples.level=FINEST
-# org.apache.reef.tang.level=INFO
-
-# Handlers
-# -----------------------------------------
-
-# --- ConsoleHandler ---
-# Override of global logging level
-java.util.logging.ConsoleHandler.level=FINEST
-java.util.logging.ConsoleHandler.formatter=java.util.logging.SimpleFormatter
-
-# --- FileHandler ---
-# Override of global logging level
-java.util.logging.FileHandler.level=FINEST
-
-# Naming style for the output file:
-# (The output file is placed in the directory
-# defined by the "user.home" System property.)
-java.util.logging.FileHandler.pattern=%h/reef.%u.log
-
-# Limiting size of output file in bytes:
-java.util.logging.FileHandler.limit=512000
-
-# Number of output files to cycle through, by appending an
-# integer to the base file name:
-java.util.logging.FileHandler.count=100
-
-# Style of output (Simple or XML):
-java.util.logging.FileHandler.formatter=java.util.logging.SimpleFormatter

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge/pom.xml
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge/pom.xml b/reef-bridge-project/reef-bridge/pom.xml
deleted file mode 100644
index fab4e2f..0000000
--- a/reef-bridge-project/reef-bridge/pom.xml
+++ /dev/null
@@ -1,111 +0,0 @@
-<?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:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
-         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>
-    <artifactId>reef-bridge</artifactId>
-    <name>REEF Bridge</name>
-    <description>Bridge between JVM and CLR.</description>
-
-
-    <parent>
-        <groupId>org.apache.reef</groupId>
-        <artifactId>reef-bridge-project</artifactId>
-        <version>0.10-incubating-SNAPSHOT</version>
-    </parent>
-
-
-    <dependencies>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-common</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-runtime-local</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-runtime-yarn</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-io</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-checkpoint</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-bridge-java</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-bridge-clr</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-
-    </dependencies>
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-dependency-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>unpack-dependencies</id>
-                        <phase>process-resources</phase>
-                        <goals>
-                            <goal>unpack-dependencies</goal>
-                        </goals>
-
-                        <configuration>
-                            <includeArtifactIds>reef-bridge-java,reef-bridge-clr</includeArtifactIds>
-                            <outputDirectory>
-                                ${project.build.directory}/classes/ReefDriverAppDlls
-                            </outputDirectory>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-
-            <plugin>
-                <artifactId>maven-jar-plugin</artifactId>
-                <configuration>
-                    <archive>
-                        <manifest>
-                            <addClasspath>false</addClasspath>
-                            <classpathPrefix>lib/</classpathPrefix>
-                            <mainClass>org.apache.reef.javabridge.JavaBridge</mainClass>
-                        </manifest>
-                    </archive>
-                </configuration>
-            </plugin>
-
-        </plugins>
-    </build>
-</project>


[4/4] incubator-reef git commit: [REEF-77] Remove REEF Bridge

Posted by ch...@apache.org.
[REEF-77] Remove REEF Bridge

Remove REEF Bridge from the release branch as it depends on binary files.
Remove the project directory and references to it in the root pom.


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

Branch: refs/heads/branch-0.10.0-incubating
Commit: dee0a76ff0fd21525d220d16d71170dbf5424d56
Parents: 19019dc
Author: chobrian <ch...@apache.org>
Authored: Thu Dec 18 16:00:33 2014 +0900
Committer: chobrian <ch...@apache.org>
Committed: Thu Dec 18 16:00:33 2014 +0900

----------------------------------------------------------------------
 pom.xml                                         |  14 -
 reef-bridge-project/.gitignore                  |  34 -
 reef-bridge-project/pom.xml                     | 101 ---
 reef-bridge-project/reef-bridge-clr/pom.xml     | 162 -----
 .../CSharp/CSharp/ClrHandler/ClrHandler.csproj  |  66 --
 .../ClrHandler/Properties/AssemblyInfo.cs       |  65 --
 .../externals/Microsoft.Reef.Driver.dll         | Bin 123392 -> 0 bytes
 .../CSharp/ClrHandler/externals/msvcr110.dll    | Bin 849360 -> 0 bytes
 .../ClrHandler/interface/IInteropReturnInfo.cs  |  46 --
 .../CSharp/ClrHandler/interface/ILogger.cs      |  53 --
 .../JavaClrBridge/ActiveContextClr2Java.cpp     | 106 ---
 .../AllocatedEvaluatorClr2Java.cpp              | 164 -----
 .../CppBridge/JavaClrBridge/AssemblyInfo.cpp    |  50 --
 .../CppBridge/JavaClrBridge/AssemblyUtil.cpp    |  53 --
 .../Cpp/CppBridge/JavaClrBridge/BinaryUtil.cpp  | 102 ---
 .../Cpp/CppBridge/JavaClrBridge/BinaryUtil.h    |  26 -
 .../JavaClrBridge/ClosedContextClr2Java.cpp     |  86 ---
 .../CppBridge/JavaClrBridge/Clr2JavaImpl.cpp    |  44 --
 .../Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.h  | 220 ------
 .../CppBridge/JavaClrBridge/CommonUtilities.cpp |  51 --
 .../CompletedEvaluatorClr2Java.cpp              |  58 --
 .../JavaClrBridge/CompletedTaskClr2Java.cpp     |  69 --
 .../JavaClrBridge/ContextMessageClr2Java.cpp    |  76 --
 .../EvaluatorRequestorClr2Java.cpp              |  69 --
 .../JavaClrBridge/FailedContextClr2Java.cpp     |  94 ---
 .../JavaClrBridge/FailedEvaluatorClr2Java.cpp   |  72 --
 .../JavaClrBridge/FailedTaskClr2Java.cpp        |  79 ---
 .../JavaClrBridge/HttpServerClr2Java.cpp        | 135 ----
 .../CppBridge/JavaClrBridge/InteropAssemblies.h |  40 --
 .../CppBridge/JavaClrBridge/InteropLogger.cpp   |  50 --
 .../Cpp/CppBridge/JavaClrBridge/InteropLogger.h |  47 --
 .../JavaClrBridge/InteropReturnInfo.cpp         |  91 ---
 .../CppBridge/JavaClrBridge/InteropReturnInfo.h |  57 --
 .../Cpp/CppBridge/JavaClrBridge/InteropUtil.cpp | 129 ----
 .../Cpp/CppBridge/JavaClrBridge/InteropUtil.h   |  65 --
 .../CppBridge/JavaClrBridge/JavaClrBridge.cpp   | 492 -------------
 .../Cpp/CppBridge/JavaClrBridge/JavaClrBridge.h |  33 -
 .../CppBridge/JavaClrBridge/JavaClrBridge.sln   |  56 --
 .../JavaClrBridge/JavaClrBridge.vcxproj         | 173 -----
 .../JavaClrBridge/JavaClrBridge.vcxproj.filters | 104 ---
 .../CppBridge/JavaClrBridge/ManagedLogger.cpp   |  47 --
 .../main/Cpp/CppBridge/JavaClrBridge/ReadMe.txt |  57 --
 .../JavaClrBridge/RunningTaskClr2Java.cpp       |  90 ---
 .../JavaClrBridge/SuspendedTaskClr2Java.cpp     |  83 ---
 .../JavaClrBridge/TaskMessageClr2Java.cpp       |  58 --
 reef-bridge-project/reef-bridge-java/pom.xml    | 116 ---
 .../reef/javabridge/ActiveContextBridge.java    |  80 ---
 .../javabridge/AllocatedEvaluatorBridge.java    | 141 ----
 .../reef/javabridge/ClosedContextBridge.java    |  81 ---
 .../javabridge/CompletedEvaluatorBridge.java    |  43 --
 .../reef/javabridge/CompletedTaskBridge.java    |  40 --
 .../reef/javabridge/ContextMessageBridge.java   |  56 --
 .../javabridge/EvaluatorRequestorBridge.java    |  76 --
 .../reef/javabridge/FailedContextBridge.java    |  83 ---
 .../reef/javabridge/FailedEvaluatorBridge.java  |  47 --
 .../reef/javabridge/FailedTaskBridge.java       |  60 --
 .../reef/javabridge/HttpServerEventBridge.java  |  79 ---
 .../apache/reef/javabridge/InteropLogger.java   |  55 --
 .../reef/javabridge/InteropReturnInfo.java      |  52 --
 .../org/apache/reef/javabridge/JavaBridge.java  |  31 -
 .../apache/reef/javabridge/NativeBridge.java    |  32 -
 .../apache/reef/javabridge/NativeInterop.java   | 262 -------
 .../reef/javabridge/RunningTaskBridge.java      |  49 --
 .../reef/javabridge/SuspendedTaskBridge.java    |  54 --
 .../reef/javabridge/TaskMessageBridge.java      |  36 -
 .../org/apache/reef/javabridge/Utilities.java   |  57 --
 .../reef/javabridge/generic/JobClient.java      | 322 ---------
 .../reef/javabridge/generic/JobDriver.java      | 705 -------------------
 .../apache/reef/javabridge/generic/Launch.java  | 236 -------
 .../reef/javabridge/generic/LaunchHeadless.java | 100 ---
 .../reef/javabridge/generic/package-info.java   |  22 -
 .../util/logging/CLRBufferedLogHandler.java     | 167 -----
 .../reef/util/logging/CLRLoggingConfig.java     |  31 -
 .../apache/reef/util/logging/package-info.java  |  22 -
 .../org/apache/reef/clr.logging.properties      |  82 ---
 reef-bridge-project/reef-bridge/pom.xml         | 111 ---
 76 files changed, 7095 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 457e7de..233a76a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -552,7 +552,6 @@ under the License.
 
     <modules>
         <module>reef-annotations</module>
-        <module>reef-bridge-project</module>
         <module>reef-checkpoint</module>
         <module>reef-common</module>
         <module>reef-examples</module>
@@ -592,18 +591,5 @@ under the License.
                 </plugins>
             </build>
         </profile>
-        <profile>
-            <id>reef-bridge</id>
-            <activation>
-                <os>
-                    <name>Windows</name>
-                    <family>Windows</family>
-                </os>
-            </activation>
-            <modules>
-                <module>reef-bridge-project</module>
-            </modules>
-
-        </profile>
     </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/.gitignore
----------------------------------------------------------------------
diff --git a/reef-bridge-project/.gitignore b/reef-bridge-project/.gitignore
deleted file mode 100644
index dd32f71..0000000
--- a/reef-bridge-project/.gitignore
+++ /dev/null
@@ -1,34 +0,0 @@
-ml-data
-tmp
-tang.conf
-.DS_Store
-target
-generated
-build
-.settings
-.classpath
-.project
-*.sw[op]
-.externalToolBuilders
-nbactions*.xml
-nb-configuration.xml
-*~
-\#*
-*.iml
-.idea
-atlassian-ide-plugin.xml
-REEF_LOCAL_RUNTIME
-profile-*.json
-.obj
-.dll
-.class
-.tlog
-dotnetHello
-lib
-x64
-*.sdf
-*.suo
-*.opensdf
-obj
-*.cache
-*.log

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/pom.xml
----------------------------------------------------------------------
diff --git a/reef-bridge-project/pom.xml b/reef-bridge-project/pom.xml
deleted file mode 100644
index c6b1eef..0000000
--- a/reef-bridge-project/pom.xml
+++ /dev/null
@@ -1,101 +0,0 @@
-<?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>
-    <artifactId>reef-bridge-project</artifactId>
-    <name>REEF Bridge Project</name>
-    <description>Bridge between JVM and CLR.</description>
-    <packaging>pom</packaging>
-
-
-    <parent>
-        <groupId>org.apache.reef</groupId>
-        <artifactId>reef-project</artifactId>
-        <version>0.10-incubating-SNAPSHOT</version>
-    </parent>
-
-
-    <dependencies>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-common</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-runtime-local</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-runtime-yarn</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-io</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-checkpoint</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-    </dependencies>
-    <modules>
-        <module>reef-bridge-java</module>
-        <module>reef-bridge-clr</module>
-        <module>reef-bridge</module>
-    </modules>
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-shade-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>shade</goal>
-                        </goals>
-                    </execution>
-                </executions>
-                <configuration>
-                    <outputFile>
-                        ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar
-                    </outputFile>
-                    <filters>
-                        <filter>
-                            <artifact>*:*</artifact>
-                            <excludes>
-                                <exclude>yarn-default.xml</exclude>
-                                <exclude>yarn-version-info.properties</exclude>
-                                <exclude>core-default.xml</exclude>
-                                <exclude>LICENSE</exclude>
-                                <exclude>META-INF/*</exclude>
-                            </excludes>
-                        </filter>
-                    </filters>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/pom.xml
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/pom.xml b/reef-bridge-project/reef-bridge-clr/pom.xml
deleted file mode 100644
index 14d7745..0000000
--- a/reef-bridge-project/reef-bridge-clr/pom.xml
+++ /dev/null
@@ -1,162 +0,0 @@
-<?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>
-    <artifactId>reef-bridge-clr</artifactId>
-    <name>REEF Bridge CLR</name>
-    <description>Bridge between JVM and CLR.</description>
-
-
-    <parent>
-        <groupId>org.apache.reef</groupId>
-        <artifactId>reef-bridge-project</artifactId>
-        <version>0.10-incubating-SNAPSHOT</version>
-    </parent>
-
-
-    <dependencies>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-common</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-runtime-local</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-runtime-yarn</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-io</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-checkpoint</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-bridge-java</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro</artifactId>
-        </dependency>
-    </dependencies>
-
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.rat</groupId>
-                <artifactId>apache-rat-plugin</artifactId>
-                <configuration>
-                    <excludes>
-                        <!-- Build files are frequently overwritten by Visual Studio -->
-                        <exclude>src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.sln</exclude>
-                        <exclude>src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj</exclude>
-                        <exclude>src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj.filters</exclude>
-                        <exclude>src/main/CSharp/CSharp/ClrHandler/ClrHandler.csproj</exclude>
-                        <!--End of Visual Studio build files-->
-                    </excludes>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-    <profiles>
-        <profile>
-            <id>Bridge</id>
-            <build>
-                <plugins>
-                    <plugin>
-                        <groupId>org.codehaus.mojo</groupId>
-                        <artifactId>exec-maven-plugin</artifactId>
-                        <configuration>
-                            <executable>msbuild.exe</executable>
-                        </configuration>
-                        <executions>
-                            <execution>
-                                <id>clean</id>
-                                <phase>clean</phase>
-                                <configuration>
-                                    <arguments>
-                                        <argument>
-                                            ${project.basedir}/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.sln
-                                        </argument>
-                                        <argument>/p:Configuration="Release"</argument>
-                                        <argument>/p:Platform="x64"</argument>
-                                        <argument>/t:Clean</argument>
-                                    </arguments>
-                                </configuration>
-                                <goals>
-                                    <goal>exec</goal>
-                                </goals>
-                            </execution>
-                            <execution>
-                                <id>build</id>
-                                <phase>compile</phase>
-                                <configuration>
-                                    <arguments>
-                                        <argument>
-                                            ${project.basedir}/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.sln
-                                        </argument>
-                                        <argument>/p:Configuration="Release"</argument>
-                                        <argument>/p:Platform="x64"</argument>
-                                    </arguments>
-                                </configuration>
-                                <goals>
-                                    <goal>exec</goal>
-                                </goals>
-                            </execution>
-                        </executions>
-                    </plugin>
-                    <plugin>
-                        <artifactId>maven-resources-plugin</artifactId>
-                        <executions>
-                            <execution>
-                                <id>copy-external-dlls</id>
-                                <phase>process-resources</phase>
-                                <goals>
-                                    <goal>copy-resources</goal>
-                                </goals>
-                                <configuration>
-                                    <overwrite>true</overwrite>
-                                    <outputDirectory>${basedir}/target/classes</outputDirectory>
-                                    <resources>
-                                        <resource>
-                                            <directory>src/main/CSharp/CSharp/ClrHandler/externals</directory>
-                                        </resource>
-                                    </resources>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>
-                </plugins>
-            </build>
-        </profile>
-    </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/ClrHandler.csproj
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/ClrHandler.csproj b/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/ClrHandler.csproj
deleted file mode 100644
index 952dd0d..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/ClrHandler.csproj
+++ /dev/null
@@ -1,66 +0,0 @@
-<?xml version="1.0" encoding="utf-8"?>
-<Project ToolsVersion="4.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
-  <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
-  <PropertyGroup>
-    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
-    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
-    <ProjectGuid>{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}</ProjectGuid>
-    <OutputType>Library</OutputType>
-    <AppDesignerFolder>Properties</AppDesignerFolder>
-    <RootNamespace>ClrHandler</RootNamespace>
-    <AssemblyName>ClrHandler</AssemblyName>
-    <TargetFrameworkVersion>v4.5</TargetFrameworkVersion>
-    <FileAlignment>512</FileAlignment>
-  </PropertyGroup>
-  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug|AnyCPU' ">
-    <DebugSymbols>true</DebugSymbols>
-    <DebugType>full</DebugType>
-    <Optimize>false</Optimize>
-    <OutputPath>..\..\..\..\..\target\classes\</OutputPath>
-    <DefineConstants>DEBUG;TRACE</DefineConstants>
-    <ErrorReport>prompt</ErrorReport>
-    <WarningLevel>4</WarningLevel>
-  </PropertyGroup>
-  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
-    <DebugType>pdbonly</DebugType>
-    <Optimize>true</Optimize>
-    <OutputPath>..\..\..\..\..\target\classes\</OutputPath>
-    <DefineConstants>TRACE</DefineConstants>
-    <ErrorReport>prompt</ErrorReport>
-    <WarningLevel>4</WarningLevel>
-  </PropertyGroup>
-  <PropertyGroup>
-    <SignAssembly>false</SignAssembly>
-  </PropertyGroup>
-  <PropertyGroup>
-    <AssemblyOriginatorKeyFile>
-    </AssemblyOriginatorKeyFile>
-  </PropertyGroup>
-  <ItemGroup>
-    <Reference Include="Microsoft.Reef.Driver">
-      <HintPath>externals\Microsoft.Reef.Driver.dll</HintPath>
-      <Private>true</Private>
-    </Reference>
-    <Reference Include="System" />
-    <Reference Include="System.Core" />
-    <Reference Include="System.Runtime.Serialization" />
-    <Reference Include="System.Xml.Linq" />
-    <Reference Include="System.Data.DataSetExtensions" />
-    <Reference Include="Microsoft.CSharp" />
-    <Reference Include="System.Data" />
-    <Reference Include="System.Xml" />
-  </ItemGroup>
-  <ItemGroup>
-    <Compile Include="interface\ILogger.cs" />
-    <Compile Include="interface\IInteropReturnInfo.cs" />
-    <Compile Include="Properties\AssemblyInfo.cs" />
-  </ItemGroup>
-  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
-  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
-       Other similar extension points exist, see Microsoft.Common.targets.
-  <Target Name="BeforeBuild">
-  </Target>
-  <Target Name="AfterBuild">
-  </Target>
-  -->
-</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/Properties/AssemblyInfo.cs b/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/Properties/AssemblyInfo.cs
deleted file mode 100644
index 819b995..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/Properties/AssemblyInfo.cs
+++ /dev/null
@@ -1,65 +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.
- */
-/**
- * Copyright (C) 2014 Microsoft Corporation
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *         http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-using System.Reflection;
-using System.Runtime.InteropServices;
-
-// General Information about an assembly is controlled through the following 
-// set of attributes. Change these attribute values to modify the information
-// associated with an assembly.
-[assembly: AssemblyTitle("ClrHandler")]
-[assembly: AssemblyDescription("The interface dll between CPP and CLR code")]
-[assembly: AssemblyProduct("ClrHandler")]
-[assembly: AssemblyCopyright("Copyright ©  2014")]
-
-// Setting ComVisible to false makes the types in this assembly not visible 
-// to COM components.  If you need to access a type in this assembly from 
-// COM, set the ComVisible attribute to true on that type.
-[assembly: ComVisible(false)]
-
-// The following GUID is for the ID of the typelib if this project is exposed to COM
-[assembly: Guid("3efe4d3c-087b-4076-b331-8f3e36c10016")]
-
-// Version information for an assembly consists of the following four values:
-//
-//      Major Version
-//      Minor Version 
-//      Build Number
-//      Revision
-//
-// You can specify all the values or you can default the Build and Revision Numbers 
-// by using the '*' as shown below:
-// [assembly: AssemblyVersion("1.0.*")]
-[assembly: AssemblyVersion("1.0.0.0")]
-[assembly: AssemblyFileVersion("1.0.0.0")]

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/externals/Microsoft.Reef.Driver.dll
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/externals/Microsoft.Reef.Driver.dll b/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/externals/Microsoft.Reef.Driver.dll
deleted file mode 100644
index d95a8c7..0000000
Binary files a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/externals/Microsoft.Reef.Driver.dll and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/externals/msvcr110.dll
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/externals/msvcr110.dll b/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/externals/msvcr110.dll
deleted file mode 100644
index dd484a5..0000000
Binary files a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/externals/msvcr110.dll and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/interface/IInteropReturnInfo.cs
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/interface/IInteropReturnInfo.cs b/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/interface/IInteropReturnInfo.cs
deleted file mode 100644
index 7620092..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/interface/IInteropReturnInfo.cs
+++ /dev/null
@@ -1,46 +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.
- */
-/**
- * Copyright (C) 2014 Microsoft Corporation
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *         http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-using System;
-
-namespace Microsoft.Reef.Interop
-{
-    public interface IInteropReturnInfo
-    {
-        void AddExceptionString(String exceptionString);       
-        Boolean HasExceptions();
-        void SetReturnCode(int rc);
-        int GetReturnCode();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/interface/ILogger.cs
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/interface/ILogger.cs b/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/interface/ILogger.cs
deleted file mode 100644
index 7f200e7..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/CSharp/CSharp/ClrHandler/interface/ILogger.cs
+++ /dev/null
@@ -1,53 +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.
- */
-/**
- * Copyright (C) 2014 Microsoft Corporation
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *         http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-using System;
-
-namespace Microsoft.Reef.Interop
-{
-    public enum TraceLevel : int
-    {
-        NoTrace = Int32.MaxValue,
-
-        Error = 1000,
-        Warning = 900,
-        Info = 800,
-        Verbose = 300, 
-    }
-
-    public interface ILogger
-    {
-        void Log(TraceLevel traceLevel, String message);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ActiveContextClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ActiveContextClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ActiveContextClr2Java.cpp
deleted file mode 100644
index fc04b06..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ActiveContextClr2Java.cpp
+++ /dev/null
@@ -1,106 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        private ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        ActiveContextClr2Java::ActiveContextClr2Java(JNIEnv *env, jobject jobjectActiveContext) {
-          ManagedLog::LOGGER->LogStart("ActiveContextClr2Java::ActiveContextClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-
-          _jobjectActiveContext = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectActiveContext));
-
-          jclass jclassActiveContext = env->GetObjectClass(_jobjectActiveContext);
-
-          jfieldID jidContextId = env->GetFieldID(jclassActiveContext, "contextId", "Ljava/lang/String;");
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectActiveContext, jidContextId)));
-
-          jfieldID jidEvaluatorId = env->GetFieldID(jclassActiveContext, "evaluatorId", "Ljava/lang/String;");
-          _jstringEvaluatorId = (jstring)env->GetObjectField(_jobjectActiveContext, jidEvaluatorId);
-          _jstringEvaluatorId = reinterpret_cast<jstring>(env->NewGlobalRef(_jstringEvaluatorId));
-
-          ManagedLog::LOGGER->LogStop("ActiveContextClr2Java::ActiveContextClr2Java");
-        }
-
-        void ActiveContextClr2Java::SubmitTask(String^ taskConfigStr) {
-          ManagedLog::LOGGER->LogStart("ActiveContextClr2Java::SubmitTask");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassActiveContext = env->GetObjectClass (_jobjectActiveContext);
-          jmethodID jmidSubmitTask = env->GetMethodID(jclassActiveContext, "submitTaskString", "(Ljava/lang/String;)V");
-
-          if (jmidSubmitTask == NULL) {
-            ManagedLog::LOGGER->Log("jmidSubmitTask is NULL");
-            return;
-          }
-          env -> CallObjectMethod(
-            _jobjectActiveContext,
-            jmidSubmitTask,
-            JavaStringFromManagedString(env, taskConfigStr));
-          ManagedLog::LOGGER->LogStop("ActiveContextClr2Java::SubmitTask");
-        }
-
-        void ActiveContextClr2Java::OnError(String^ message) {
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectActiveContext);
-        }
-
-        void ActiveContextClr2Java::Close() {
-          ManagedLog::LOGGER->LogStart("ActiveContextClr2Java::Close");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassActiveContext = env->GetObjectClass (_jobjectActiveContext);
-          jmethodID jmidClose = env->GetMethodID(jclassActiveContext, "close", "()V");
-
-          if (jmidClose == NULL) {
-            ManagedLog::LOGGER->Log("jmidClose is NULL");
-            return;
-          }
-          env -> CallObjectMethod(
-            _jobjectActiveContext,
-            jmidClose);
-          ManagedLog::LOGGER->LogStop("ActiveContextClr2Java::Close");
-        }
-
-        String^ ActiveContextClr2Java::GetId() {
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-
-        String^ ActiveContextClr2Java::GetEvaluatorId() {
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringEvaluatorId);
-        }
-
-        IEvaluatorDescriptor^ ActiveContextClr2Java::GetEvaluatorDescriptor() {
-          ManagedLog::LOGGER->LogStart("ActiveContextClr2Java::GetEvaluatorDescriptor");
-          return CommonUtilities::RetrieveEvaluatorDescriptor(_jobjectActiveContext, _jvm);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AllocatedEvaluatorClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AllocatedEvaluatorClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AllocatedEvaluatorClr2Java.cpp
deleted file mode 100644
index 11e8a80..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AllocatedEvaluatorClr2Java.cpp
+++ /dev/null
@@ -1,164 +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.
- */
-#include "Clr2JavaImpl.h"
-
-using namespace JavaClrBridge;
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        AllocatedEvaluatorClr2Java::AllocatedEvaluatorClr2Java(JNIEnv *env, jobject jallocatedEvaluator) {
-
-          ManagedLog::LOGGER->LogStart("AllocatedEvaluatorClr2Java::AllocatedEvaluatorClr2Java");
-
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectAllocatedEvaluator = reinterpret_cast<jobject>(env->NewGlobalRef(jallocatedEvaluator));
-
-          jclass jclassAllocatedEvaluator = env->GetObjectClass (_jobjectAllocatedEvaluator);
-          jfieldID jidEvaluatorId = env->GetFieldID(jclassAllocatedEvaluator, "evaluatorId", "Ljava/lang/String;");
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectAllocatedEvaluator, jidEvaluatorId)));
-
-          jfieldID jidNameServerInfo = env->GetFieldID(jclassAllocatedEvaluator, "nameServerInfo", "Ljava/lang/String;");
-          _jstringNameServerInfo = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectAllocatedEvaluator, jidNameServerInfo)));
-
-          ManagedLog::LOGGER->LogStop("AllocatedEvaluatorClr2Java::AllocatedEvaluatorClr2Java");
-        }
-
-        void AllocatedEvaluatorClr2Java::SubmitContext(String^ contextConfigStr) {
-          ManagedLog::LOGGER->LogStart("AllocatedEvaluatorClr2Java::SubmitContext");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassAllocatedEvaluator = env->GetObjectClass (_jobjectAllocatedEvaluator);
-          jmethodID jmidSubmitContext = env->GetMethodID(jclassAllocatedEvaluator, "submitContextString", "(Ljava/lang/String;)V");
-
-          if (jmidSubmitContext == NULL) {
-            ManagedLog::LOGGER->Log("jmidSubmitContext is NULL");
-            return;
-          }
-          env -> CallObjectMethod(
-            _jobjectAllocatedEvaluator,
-            jmidSubmitContext,
-            JavaStringFromManagedString(env, contextConfigStr));
-          ManagedLog::LOGGER->LogStop("AllocatedEvaluatorClr2Java::SubmitContext");
-        }
-
-        void AllocatedEvaluatorClr2Java::SubmitContextAndTask(String^ contextConfigStr, String^ taskConfigStr) {
-          ManagedLog::LOGGER->LogStart("AllocatedEvaluatorClr2Java::SubmitContextAndTask");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassAllocatedEvaluator = env->GetObjectClass (_jobjectAllocatedEvaluator);
-          jmethodID jmidSubmitContextAndTask = env->GetMethodID(jclassAllocatedEvaluator, "submitContextAndTaskString", "(Ljava/lang/String;Ljava/lang/String;)V");
-
-          if (jmidSubmitContextAndTask == NULL) {
-            ManagedLog::LOGGER->Log("jmidSubmitContextAndTask is NULL");
-            return;
-          }
-          env -> CallObjectMethod(
-            _jobjectAllocatedEvaluator,
-            jmidSubmitContextAndTask,
-            JavaStringFromManagedString(env, contextConfigStr),
-            JavaStringFromManagedString(env, taskConfigStr));
-          ManagedLog::LOGGER->LogStop("AllocatedEvaluatorClr2Java::SubmitContextAndTask");
-        }
-
-        void AllocatedEvaluatorClr2Java::SubmitContextAndService(String^ contextConfigStr, String^ serviceConfigStr) {
-          ManagedLog::LOGGER->LogStart("AllocatedEvaluatorClr2Java::SubmitContextAndService");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassAllocatedEvaluator = env->GetObjectClass (_jobjectAllocatedEvaluator);
-          jmethodID jmidSubmitContextAndService = env->GetMethodID(jclassAllocatedEvaluator, "submitContextAndServiceString", "(Ljava/lang/String;Ljava/lang/String;)V");
-
-          if (jmidSubmitContextAndService == NULL) {
-            ManagedLog::LOGGER->Log("jmidSubmitContextAndService is NULL");
-            return;
-          }
-          env -> CallObjectMethod(
-            _jobjectAllocatedEvaluator,
-            jmidSubmitContextAndService,
-            JavaStringFromManagedString(env, contextConfigStr),
-            JavaStringFromManagedString(env, serviceConfigStr));
-          ManagedLog::LOGGER->LogStop("AllocatedEvaluatorClr2Java::SubmitContextAndService");
-        }
-
-        void AllocatedEvaluatorClr2Java::SubmitContextAndServiceAndTask(String^ contextConfigStr, String^ serviceConfigStr, String^ taskConfigStr) {
-          ManagedLog::LOGGER->LogStart("AllocatedEvaluatorClr2Java::SubmitContextAndServiceAndTask");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassAllocatedEvaluator = env->GetObjectClass (_jobjectAllocatedEvaluator);
-          jmethodID jmidSubmitContextAndServiceAndTask = env->GetMethodID(jclassAllocatedEvaluator, "submitContextAndServiceAndTaskString", "(Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;)V");
-
-          if (jmidSubmitContextAndServiceAndTask == NULL) {
-            ManagedLog::LOGGER->Log("jmidSubmitContextAndServiceAndTask is NULL");
-            return;
-          }
-          env -> CallObjectMethod(
-            _jobjectAllocatedEvaluator,
-            jmidSubmitContextAndServiceAndTask,
-            JavaStringFromManagedString(env, contextConfigStr),
-            JavaStringFromManagedString(env, serviceConfigStr),
-            JavaStringFromManagedString(env, taskConfigStr));
-          ManagedLog::LOGGER->LogStop("AllocatedEvaluatorClr2Java::SubmitContextAndServiceAndTask");
-        }
-
-        void AllocatedEvaluatorClr2Java::OnError(String^ message) {
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectAllocatedEvaluator);
-        }
-
-        void AllocatedEvaluatorClr2Java::Close() {
-          ManagedLog::LOGGER->LogStart("AllocatedEvaluatorClr2Java::Close");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassAllocatedEvaluator = env->GetObjectClass (_jobjectAllocatedEvaluator);
-          jmethodID jmidClose = env->GetMethodID(jclassAllocatedEvaluator, "close", "()V");
-
-          if (jmidClose == NULL) {
-            ManagedLog::LOGGER->Log("jmidClose is NULL");
-            return;
-          }
-          env -> CallObjectMethod(
-            _jobjectAllocatedEvaluator,
-            jmidClose);
-          ManagedLog::LOGGER->LogStop("AllocatedEvaluatorClr2Java::Close");
-        }
-
-        String^ AllocatedEvaluatorClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("AllocatedEvaluatorClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-
-        String^ AllocatedEvaluatorClr2Java::GetNameServerInfo() {
-          ManagedLog::LOGGER->Log("AllocatedEvaluatorClr2Java::GetNameServerInfo");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringNameServerInfo);
-        }
-
-        IEvaluatorDescriptor^ AllocatedEvaluatorClr2Java::GetEvaluatorDescriptor() {
-          ManagedLog::LOGGER->LogStart("AllocatedEvaluatorClr2Java::GetEvaluatorDescriptor");
-          return CommonUtilities::RetrieveEvaluatorDescriptor(_jobjectAllocatedEvaluator, _jvm);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AssemblyInfo.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AssemblyInfo.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AssemblyInfo.cpp
deleted file mode 100644
index f6c3178..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AssemblyInfo.cpp
+++ /dev/null
@@ -1,50 +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.
- */
-using namespace System;
-using namespace System::Reflection;
-using namespace System::Runtime::CompilerServices;
-using namespace System::Runtime::InteropServices;
-using namespace System::Security::Permissions;
-
-//
-// General Information about an assembly is controlled through the following
-// set of attributes. Change these attribute values to modify the information
-// associated with an assembly.
-//
-[assembly:AssemblyTitleAttribute("JavaClrBridge")];
-[assembly:AssemblyProductAttribute("JavaClrBridge")];
-[assembly:AssemblyCopyrightAttribute("Copyright (c)  2014")];
-//
-// Version information for an assembly consists of the following four values:
-//
-//      Major Version
-//      Minor Version
-//      Build Number
-//      Revision
-//
-// You can specify all the value or you can default the Revision and Build Numbers
-// by using the '*' as shown below:
-
-[assembly:AssemblyVersionAttribute("1.0.*")];
-
-[assembly:ComVisible(false)];
-
-[assembly:CLSCompliantAttribute(true)];
-
-[assembly:SecurityPermission(SecurityAction::RequestMinimum, UnmanagedCode = true)];

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AssemblyUtil.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AssemblyUtil.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AssemblyUtil.cpp
deleted file mode 100644
index ce9239c..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/AssemblyUtil.cpp
+++ /dev/null
@@ -1,53 +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.
- */
-#include "InteropAssemblies.h"
-
-
-void AssemblyUtil::Add(Assembly^  myasm) {
-  if (0 == _asmCount) {
-    //asm1 = myasm;
-    AppDomain^ currentDomain = AppDomain::CurrentDomain;
-    currentDomain->AssemblyResolve += gcnew ResolveEventHandler(&MyResolveEventHandler);
-  }
-  String^ asmName = myasm->FullName->ToLower();
-  Assembly^ existingAsm = nullptr;
-  if (!asms2->TryGetValue(asmName, existingAsm)) {
-    Console::WriteLine ("AssemblyUtil:: Adding " + asmName);
-    asms2->Add(asmName , myasm);
-    ++_asmCount;
-  }
-}
-
-Assembly^ AssemblyUtil::FindAsm (String^ myasm) {
-  Assembly^ returnAsm = nullptr;
-  if (!asms2->TryGetValue(myasm->ToLower(), returnAsm)) {
-    Console::WriteLine ("AssemblyUtil:: FindAsm_Not_Found " + myasm->ToString());
-  }
-  return returnAsm;
-}
-
-Assembly^ AssemblyUtil::MyResolveEventHandler(Object^ sender, ResolveEventArgs^ args) {
-  Console::WriteLine ("AssemblyUtil:: Resolving " + args->Name);
-  Assembly^ myAsm = AssemblyUtil::FindAsm(args->Name);
-  if (nullptr != myAsm) {
-    Console::WriteLine ("AssemblyUtil:: Found " + args->Name);
-  }
-  return myAsm ;
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/BinaryUtil.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/BinaryUtil.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/BinaryUtil.cpp
deleted file mode 100644
index b7c3a2e..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/BinaryUtil.cpp
+++ /dev/null
@@ -1,102 +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.
- */
-#include "InteropUtil.h"
-#include "BinaryUtil.h"
-
-DWORD GetActualAddressFromRVA(IMAGE_SECTION_HEADER* pSectionHeader, IMAGE_NT_HEADERS* pNTHeaders, DWORD dwRVA) {
-  DWORD dwRet = 0;
-
-  for (int j = 0; j < pNTHeaders->FileHeader.NumberOfSections; j++, pSectionHeader++) {
-    DWORD cbMaxOnDisk = min( pSectionHeader->Misc.VirtualSize, pSectionHeader->SizeOfRawData );
-
-    DWORD startSectRVA, endSectRVA;
-
-    startSectRVA = pSectionHeader->VirtualAddress;
-    endSectRVA = startSectRVA + cbMaxOnDisk;
-
-    if ( (dwRVA >= startSectRVA) && (dwRVA < endSectRVA)) {
-      dwRet =  (pSectionHeader->PointerToRawData ) + (dwRVA - startSectRVA);
-      break;
-    }
-
-  }
-
-  return dwRet;
-}
-
-
-BINARY_TYPE IsManagedBinary(const wchar_t*  lpszImageName) {
-  BINARY_TYPE binaryType = BINARY_TYPE_NONE;
-  HANDLE hFile = CreateFile(lpszImageName, GENERIC_READ, FILE_SHARE_READ, NULL, OPEN_EXISTING, FILE_ATTRIBUTE_NORMAL, NULL);
-
-  if (INVALID_HANDLE_VALUE != hFile) {
-    //succeeded
-    HANDLE hOpenFileMapping = CreateFileMapping(hFile, NULL, PAGE_READONLY, 0, 0, NULL);
-    if (hOpenFileMapping) {
-      BYTE* lpBaseAddress = NULL;
-
-      lpBaseAddress = (BYTE*)MapViewOfFile(hOpenFileMapping, FILE_MAP_READ, 0, 0, 0);
-
-      if (lpBaseAddress) {
-        //having mapped the executable to our process space, now start navigating through the sections
-
-        //DOS header is straightforward. It is the topmost structure in the PE file
-        //i.e. the one at the lowest offset into the file
-        IMAGE_DOS_HEADER* pDOSHeader = (IMAGE_DOS_HEADER*)lpBaseAddress;
-
-        //the only important data in the DOS header is the e_lfanew
-        //the e_lfanew points to the offset of the beginning of NT Headers data
-        IMAGE_NT_HEADERS* pNTHeaders = (IMAGE_NT_HEADERS*)((BYTE*)pDOSHeader + pDOSHeader->e_lfanew);
-
-        IMAGE_SECTION_HEADER* pSectionHeader = (IMAGE_SECTION_HEADER*)((BYTE*)pNTHeaders + sizeof(IMAGE_NT_HEADERS));
-
-        //Now, start parsing
-        //check if it is a PE file
-
-        if (pNTHeaders->Signature == IMAGE_NT_SIGNATURE) {
-          //start parsing COM table
-
-          DWORD dwNETHeaderTableLocation = pNTHeaders->OptionalHeader.DataDirectory[IMAGE_DIRECTORY_ENTRY_COM_DESCRIPTOR].VirtualAddress;
-
-          if (dwNETHeaderTableLocation) {
-            //import data does exist for this module
-            IMAGE_COR20_HEADER* pNETHeader = (IMAGE_COR20_HEADER*)((BYTE*)pDOSHeader + GetActualAddressFromRVA(pSectionHeader, pNTHeaders, dwNETHeaderTableLocation));
-
-            if (pNETHeader) {
-              binaryType = BINARY_TYPE_CLR;
-            }
-            else {
-              binaryType = BINARY_TYPE_NATIVE;
-            }
-          }
-          else {
-            binaryType = BINARY_TYPE_NATIVE;
-          }
-        }
-        else {
-          binaryType = BINARY_TYPE_NONE;
-        }
-        UnmapViewOfFile(lpBaseAddress);
-      }
-      CloseHandle(hOpenFileMapping);
-    }
-    CloseHandle(hFile);
-  }
-  return binaryType;
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/BinaryUtil.h
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/BinaryUtil.h b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/BinaryUtil.h
deleted file mode 100644
index dc946a6..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/BinaryUtil.h
+++ /dev/null
@@ -1,26 +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.
- */
-typedef enum BINARY_TYPE {
-  BINARY_TYPE_NONE = 0,
-  BINARY_TYPE_NATIVE = 1,
-  BINARY_TYPE_CLR = 2,
-} BINARY_TYPE ;
-
-
-BINARY_TYPE IsManagedBinary(const wchar_t* lpszImageName);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ClosedContextClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ClosedContextClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ClosedContextClr2Java.cpp
deleted file mode 100644
index d87d17a..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ClosedContextClr2Java.cpp
+++ /dev/null
@@ -1,86 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-        ClosedContextClr2Java::ClosedContextClr2Java(JNIEnv *env, jobject jobjectClosedContext) {
-          ManagedLog::LOGGER->LogStart("ClosedContextClr2Java::ClosedContextClr2Java");
-
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectClosedContext = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectClosedContext));
-          jclass jclassClosedContext = env->GetObjectClass (_jobjectClosedContext);
-
-          jfieldID jidContextId = env->GetFieldID(jclassClosedContext, "contextId", "Ljava/lang/String;");
-          jfieldID jidEvaluatorId = env->GetFieldID(jclassClosedContext, "evaluatorId", "Ljava/lang/String;");
-
-          _jstringContextId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectClosedContext, jidContextId)));
-          _jstringEvaluatorId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectClosedContext, jidEvaluatorId)));
-
-          ManagedLog::LOGGER->LogStop("ClosedContextClr2Java::ClosedContextClr2Java");
-        }
-
-        IActiveContextClr2Java^ ClosedContextClr2Java::GetParentContext() {
-          ManagedLog::LOGGER->LogStart("ClosedContextClr2Java::GetParentContext");
-
-          JNIEnv *env = RetrieveEnv(_jvm);
-
-          jclass jclassClosedContext = env->GetObjectClass(_jobjectClosedContext);
-          jfieldID jidParentContext = env->GetFieldID(jclassClosedContext, "parentContext", "Lorg/apache/reef/javabridge/ActiveContextBridge;");
-          jobject jobjectParentContext = env->GetObjectField(_jobjectClosedContext, jidParentContext);
-          ManagedLog::LOGGER->LogStop("ClosedContextClr2Java::GetParentContext");
-
-          return gcnew ActiveContextClr2Java(env, jobjectParentContext);
-        }
-
-        String^ ClosedContextClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("ClosedContextClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringContextId);
-        }
-
-        String^ ClosedContextClr2Java::GetEvaluatorId() {
-          ManagedLog::LOGGER->Log("ClosedContextClr2Java::GetEvaluatorId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringEvaluatorId);
-        }
-
-        IEvaluatorDescriptor^ ClosedContextClr2Java::GetEvaluatorDescriptor() {
-          ManagedLog::LOGGER->LogStart("ClosedContextClr2Java::GetEvaluatorDescriptor");
-          return CommonUtilities::RetrieveEvaluatorDescriptor(_jobjectClosedContext, _jvm);
-        }
-
-        void ClosedContextClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("ClosedContextClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectClosedContext);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.cpp
deleted file mode 100644
index ebd1aa4..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.cpp
+++ /dev/null
@@ -1,44 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft
-{
-	namespace Reef
-	{
-		namespace Interop
-		{
-			Clr2JavaImpl::Clr2JavaImpl (JNIEnv* env, jobject  jobjectEManager, jobject  jobjectDriverManager)
-			{
-				_env = env;
-				_jobjectEManager = jobjectEManager;
-				_jclassEManager =  env->GetObjectClass (_jobjectEManager);
-				_jmidSubmit = env->GetMethodID(_jclassEManager, "submit", "([B)V");	
-				_jobjectDriverManager = jobjectDriverManager;
-				_jclassDriverManager = env->GetObjectClass (_jobjectDriverManager);
-
-			}
-
-			void Clr2JavaImpl::AllocatedEvaluatorSubmitContextAndTask(String^ contextConfigStr, String^ taskConfigStr)
-			{
-				Console::WriteLine("AllocatedEvaluatorSubmitContextAndTask not implemented for Clr2JavaImpl");
-			}
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.h
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.h b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.h
deleted file mode 100644
index b907c2a..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/Clr2JavaImpl.h
+++ /dev/null
@@ -1,220 +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.
- */
-#include "InteropUtil.h"
-#include "org_apache_reef_javabridge_NativeInterop.h"
-#include "JavaClrBridge.h"
-#include "InteropAssemblies.h"
-#using "clrhandler.dll"
-#using "Microsoft.Reef.Driver.dll"
-
-using namespace System;
-using namespace System::IO;
-using namespace System::Collections::Generic;
-using namespace System::Runtime::InteropServices;
-using namespace System::Reflection;
-using namespace Microsoft::Reef::Driver::Bridge;
-using namespace Microsoft::Reef::Driver::Evaluator;
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        public ref class CommonUtilities {
-          public:
-            static IEvaluatorDescriptor^ RetrieveEvaluatorDescriptor(jobject object, JavaVM* jvm);
-        };
-
-        public ref class AllocatedEvaluatorClr2Java : public IAllocatedEvaluaotrClr2Java {
-            jobject  _jobjectAllocatedEvaluator;
-            JavaVM* _jvm;
-            jstring _jstringId;
-            jstring _jstringNameServerInfo;
-          public:
-            AllocatedEvaluatorClr2Java(JNIEnv *env, jobject jallocatedEvaluator);
-            virtual void SubmitContextAndTask(String^ contextConfigStr, String^ taskConfigStr);
-            virtual void SubmitContext(String^ contextConfigStr);
-            virtual void SubmitContextAndService(String^ contextConfigStr, String^ serviceConfigStr);
-            virtual void SubmitContextAndServiceAndTask(String^ contextConfigStr, String^ serviceConfigStr, String^ taskConfigStr);
-            virtual void OnError(String^ message);
-            virtual void Close();
-            virtual String^ GetId();
-            virtual String^ GetNameServerInfo();
-            virtual IEvaluatorDescriptor^ GetEvaluatorDescriptor();
-        };
-
-        public ref class ActiveContextClr2Java : public IActiveContextClr2Java {
-            jobject _jobjectActiveContext;
-            jstring _jstringId;
-            jstring _jstringEvaluatorId;
-            JavaVM* _jvm;
-          public:
-            ActiveContextClr2Java(JNIEnv *env, jobject jallocatedEvaluator);
-            virtual void SubmitTask(String^ taskConfigStr);
-            virtual void Close();
-            virtual void OnError(String^ message);
-            virtual String^ GetId();
-            virtual String^ GetEvaluatorId();
-            virtual IEvaluatorDescriptor^ GetEvaluatorDescriptor();
-        };
-
-        public ref class EvaluatorRequestorClr2Java : public IEvaluatorRequestorClr2Java {
-            jobject  _jobjectEvaluatorRequestor;
-            JavaVM* _jvm;
-          public:
-            EvaluatorRequestorClr2Java(JNIEnv *env, jobject jevaluatorRequestor);
-            virtual void OnError(String^ message);
-            virtual void Submit(IEvaluatorRequest^ request);
-        };
-
-        public ref class TaskMessageClr2Java : public ITaskMessageClr2Java {
-            jobject  _jobjectTaskMessage;
-            JavaVM* _jvm;
-            jstring _jstringId;
-          public:
-            TaskMessageClr2Java(JNIEnv *env, jobject jtaskMessage);
-            virtual void OnError(String^ message);
-            virtual String^ GetId();
-        };
-
-        public ref class FailedTaskClr2Java : public IFailedTaskClr2Java {
-            jobject  _jobjectFailedTask;
-            JavaVM* _jvm;
-          public:
-            FailedTaskClr2Java(JNIEnv *env, jobject jfailedTask);
-            virtual void OnError(String^ message);
-            virtual IActiveContextClr2Java^ GetActiveContext();
-            virtual String^ GetString();
-        };
-
-        public ref class RunningTaskClr2Java : public IRunningTaskClr2Java {
-            jobject  _jobjectRunningTask;
-            JavaVM* _jvm;
-            jstring _jstringId;
-          public:
-            RunningTaskClr2Java(JNIEnv *env, jobject jrunningTask);
-            virtual void OnError(String^ message);
-            virtual IActiveContextClr2Java^ GetActiveContext();
-            virtual String^ GetId();
-            virtual void Send(array<byte>^ message);
-        };
-
-        public ref class FailedEvaluatorClr2Java : public IFailedEvaluatorClr2Java {
-            jobject  _jobjectFailedEvaluator;
-            JavaVM* _jvm;
-            jstring _jstringId;
-          public:
-            FailedEvaluatorClr2Java(JNIEnv *env, jobject jfailedEvaluator);
-            virtual void OnError(String^ message);
-            virtual IEvaluatorRequestorClr2Java^ GetEvaluatorRequestor();
-            virtual String^ GetId();
-        };
-
-        public ref class HttpServerClr2Java : public IHttpServerBridgeClr2Java {
-            jobject _jhttpServerEventBridge;
-            JavaVM* _jvm;
-          public:
-            HttpServerClr2Java(JNIEnv *env, jobject jhttpServerEventBridge);
-            virtual void OnError(String^ message);
-            virtual String^ GetQueryString();
-            virtual void SetUriSpecification(String^ uriSpecification);
-            virtual void SetQueryResult(String^ queryResult);
-            virtual array<byte>^ GetQueryRequestData();
-            virtual void SetQueryResponseData(array<byte>^ responseData);
-        };
-
-        public ref class CompletedTaskClr2Java : public ICompletedTaskClr2Java {
-            jobject  _jobjectCompletedTask;
-            JavaVM* _jvm;
-            jstring _jstringId;
-          public:
-            CompletedTaskClr2Java(JNIEnv *env, jobject jcompletedTask);
-            virtual void OnError(String^ message);
-            virtual IActiveContextClr2Java^ GetActiveContext();
-            virtual String^ GetId();
-        };
-
-        public ref class SuspendedTaskClr2Java : public ISuspendedTaskClr2Java {
-            jobject  _jobjectSuspendedTask;
-            JavaVM* _jvm;
-            jstring _jstringId;
-          public:
-            SuspendedTaskClr2Java(JNIEnv *env, jobject jobjectSuspendedTask);
-            virtual void OnError(String^ message);
-            virtual IActiveContextClr2Java^ GetActiveContext();
-            virtual String^ GetId();
-            virtual array<byte>^ Get();
-        };
-
-        public ref class CompletedEvaluatorClr2Java : public ICompletedEvaluatorClr2Java {
-            jobject  _jobjectCompletedEvaluator;
-            JavaVM* _jvm;
-            jstring _jstringId;
-          public:
-            CompletedEvaluatorClr2Java(JNIEnv *env, jobject jobjectCompletedEvaluator);
-            virtual void OnError(String^ message);
-            virtual String^ GetId();
-        };
-
-        public ref class ClosedContextClr2Java : public IClosedContextClr2Java {
-            jobject  _jobjectClosedContext;
-            JavaVM* _jvm;
-            jstring _jstringContextId;
-            jstring _jstringEvaluatorId;
-          public:
-            ClosedContextClr2Java(JNIEnv *env, jobject jobjectClosedContext);
-            virtual void OnError(String^ message);
-            virtual String^ GetId();
-            virtual String^ GetEvaluatorId();
-            virtual IEvaluatorDescriptor^ GetEvaluatorDescriptor();
-            virtual IActiveContextClr2Java^ GetParentContext();
-        };
-
-        public ref class FailedContextClr2Java : public IFailedContextClr2Java {
-            jobject  _jobjectFailedContext;
-            JavaVM* _jvm;
-            jstring _jstringContextId;
-            jstring _jstringEvaluatorId;
-            jstring _jstringParentContextId;
-          public:
-            FailedContextClr2Java(JNIEnv *env, jobject jobjectFailedContext);
-            virtual void OnError(String^ message);
-            virtual String^ GetId();
-            virtual String^ GetEvaluatorId();
-            virtual String^ GetParentId();
-            virtual IEvaluatorDescriptor^ GetEvaluatorDescriptor();
-            virtual IActiveContextClr2Java^ GetParentContext();
-        };
-
-        public ref class ContextMessageClr2Java : public IContextMessageClr2Java {
-            jobject  _jobjectContextMessage;
-            JavaVM* _jvm;
-            jbyteArray _jarrayMessage;
-            jstring _jstringId;
-            jstring _jstringSourceId;
-          public:
-            ContextMessageClr2Java(JNIEnv *env, jobject jobjectContextMessage);
-            virtual void OnError(String^ message);
-            virtual array<byte>^ Get();
-            virtual String^ GetId();
-            virtual String^ GetMessageSourceId();
-        };
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CommonUtilities.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CommonUtilities.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CommonUtilities.cpp
deleted file mode 100644
index c9b6cf1..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CommonUtilities.cpp
+++ /dev/null
@@ -1,51 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>CommonUtilities");
-        };
-
-        IEvaluatorDescriptor^ CommonUtilities::RetrieveEvaluatorDescriptor(jobject object, JavaVM* jvm) {
-          ManagedLog::LOGGER->LogStart("CommonUtilities::GetEvaluatorDescriptor");
-          JNIEnv *env = RetrieveEnv(jvm);
-          jclass jclassActiveContext = env->GetObjectClass (object);
-          jmethodID jmidGetEvaluatorDescriptor = env->GetMethodID(jclassActiveContext, "getEvaluatorDescriptorSring", "()Ljava/lang/String;");
-
-          if (jmidGetEvaluatorDescriptor == NULL) {
-            ManagedLog::LOGGER->Log("jmidGetEvaluatorDescriptor is NULL");
-            return nullptr;
-          }
-          jstring jevaluatorDescriptorString = (jstring)env -> CallObjectMethod(
-                                                 object,
-                                                 jmidGetEvaluatorDescriptor);
-          String^ evaluatorDescriptorString = ManagedStringFromJavaString(env, jevaluatorDescriptorString);
-          ManagedLog::LOGGER->LogStop("InteropUtil::GetEvaluatorDescriptor");
-
-          return gcnew EvaluatorDescriptorImpl(evaluatorDescriptorString);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CompletedEvaluatorClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CompletedEvaluatorClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CompletedEvaluatorClr2Java.cpp
deleted file mode 100644
index 296670e..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CompletedEvaluatorClr2Java.cpp
+++ /dev/null
@@ -1,58 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        CompletedEvaluatorClr2Java::CompletedEvaluatorClr2Java(JNIEnv *env, jobject jCompletedEvaluator) {
-          ManagedLog::LOGGER->LogStart("CompletedEvaluatorClr2Java::CompletedEvaluatorClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectCompletedEvaluator = reinterpret_cast<jobject>(env->NewGlobalRef(jCompletedEvaluator));
-
-          jclass jclassCompletedEvaluator = env->GetObjectClass (_jobjectCompletedEvaluator);
-          jfieldID jidEvaluatorId = env->GetFieldID(jclassCompletedEvaluator, "evaluatorId", "Ljava/lang/String;");
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectCompletedEvaluator, jidEvaluatorId)));
-          ManagedLog::LOGGER->LogStop("CompletedEvaluatorClr2Java::CompletedEvaluatorClr2Java");
-        }
-
-        void CompletedEvaluatorClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("CompletedEvaluatorClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectCompletedEvaluator);
-        }
-
-        String^ CompletedEvaluatorClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("CompletedEvaluatorClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CompletedTaskClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CompletedTaskClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CompletedTaskClr2Java.cpp
deleted file mode 100644
index 631b2ea..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/CompletedTaskClr2Java.cpp
+++ /dev/null
@@ -1,69 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        CompletedTaskClr2Java::CompletedTaskClr2Java(JNIEnv *env, jobject jobjectCompletedTask) {
-          ManagedLog::LOGGER->LogStart("CompletedTaskClr2Java::CompletedTaskClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectCompletedTask = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectCompletedTask));
-
-          jclass jclassCompletedTask = env->GetObjectClass (_jobjectCompletedTask);
-          jfieldID jidTaskId = env->GetFieldID(jclassCompletedTask, "taskId", "Ljava/lang/String;");
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectCompletedTask, jidTaskId)));
-          ManagedLog::LOGGER->LogStop("CompletedTaskClr2Java::CompletedTaskClr2Java");
-        }
-
-        void CompletedTaskClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("CompletedTaskClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectCompletedTask);
-        }
-
-        IActiveContextClr2Java^ CompletedTaskClr2Java::GetActiveContext() {
-          ManagedLog::LOGGER->LogStart("CompletedTaskClr2Java::GetActiveContext");
-          JNIEnv *env = RetrieveEnv(_jvm);
-
-          jclass jclassCompletedTask = env->GetObjectClass (_jobjectCompletedTask);
-          jfieldID jidActiveContext = env->GetFieldID(jclassCompletedTask, "jactiveContext", "Lorg/apache/reef/javabridge/ActiveContextBridge;");
-          jobject jobjectActiveContext = env->GetObjectField(_jobjectCompletedTask, jidActiveContext);
-          ManagedLog::LOGGER->LogStop("CompletedTaskClr2Java::GetActiveContext");
-          return gcnew ActiveContextClr2Java(env, jobjectActiveContext);
-        }
-
-        String^ CompletedTaskClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("CompletedTaskClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ContextMessageClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ContextMessageClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ContextMessageClr2Java.cpp
deleted file mode 100644
index 6f0834a..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ContextMessageClr2Java.cpp
+++ /dev/null
@@ -1,76 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-        ContextMessageClr2Java::ContextMessageClr2Java(JNIEnv *env, jobject jobjectContextMessage) {
-          ManagedLog::LOGGER->LogStart("ContextMessageClr2Java::ContextMessageClr2Java");
-
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectContextMessage = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectContextMessage));
-          jclass jclassContextMessage = env->GetObjectClass (_jobjectContextMessage);
-
-          jfieldID jidId = env->GetFieldID(jclassContextMessage, "contextMessageId", "Ljava/lang/String;");
-          jfieldID jidSourceId = env->GetFieldID(jclassContextMessage, "messageSourceId", "Ljava/lang/String;");
-          jfieldID jidMessage = env->GetFieldID(jclassContextMessage, "message", "()[B");
-
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectContextMessage, jidId)));
-          _jstringSourceId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectContextMessage, jidSourceId)));
-          _jarrayMessage = reinterpret_cast<jbyteArray>(env->NewGlobalRef(env->GetObjectField(_jobjectContextMessage, jidMessage)));
-
-          ManagedLog::LOGGER->LogStop("ContextMessageClr2Java::ContextMessageClr2Java");
-        }
-
-        String^ ContextMessageClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("ContextMessageClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-
-        String^ ContextMessageClr2Java::GetMessageSourceId() {
-          ManagedLog::LOGGER->Log("ContextMessageClr2Java::GetMessageSourceId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringSourceId);
-        }
-
-        array<byte>^ ContextMessageClr2Java::Get() {
-          ManagedLog::LOGGER->Log("ContextMessageClr2Java::Get");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedByteArrayFromJavaByteArray(env, _jarrayMessage);
-        }
-
-        void ContextMessageClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("ContextMessageClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectContextMessage);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file


[3/4] incubator-reef git commit: [REEF-77] Remove REEF Bridge

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/EvaluatorRequestorClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/EvaluatorRequestorClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/EvaluatorRequestorClr2Java.cpp
deleted file mode 100644
index eff8e3e..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/EvaluatorRequestorClr2Java.cpp
+++ /dev/null
@@ -1,69 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        EvaluatorRequestorClr2Java::EvaluatorRequestorClr2Java(JNIEnv *env, jobject jevaluatorRequestor) {
-          ManagedLog::LOGGER->LogStart("EvaluatorRequestorClr2Java::EvaluatorRequestorClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectEvaluatorRequestor = reinterpret_cast<jobject>(env->NewGlobalRef(jevaluatorRequestor));
-          ManagedLog::LOGGER->LogStop("EvaluatorRequestorClr2Java::EvaluatorRequestorClr2Java");
-        }
-
-        void EvaluatorRequestorClr2Java::Submit(IEvaluatorRequest^ request) {
-          ManagedLog::LOGGER->LogStart("EvaluatorRequestorClr2Java::Submit");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassEvaluatorRequestor = env->GetObjectClass (_jobjectEvaluatorRequestor);
-          jmethodID jmidSubmit = env->GetMethodID(jclassEvaluatorRequestor, "submit", "(IIILjava/lang/String;)V");
-
-          if (jmidSubmit == NULL) {
-            fprintf(stdout, " jmidSubmit is NULL\n");
-            fflush (stdout);
-            return;
-          }
-          env -> CallObjectMethod(
-            _jobjectEvaluatorRequestor,
-            jmidSubmit,
-            request -> Number,
-            request -> MemoryMegaBytes,
-			request -> VirtualCore,
-            JavaStringFromManagedString(env, request -> Rack));
-          ManagedLog::LOGGER->LogStop("EvaluatorRequestorClr2Java::Submit");
-        }
-
-        void EvaluatorRequestorClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("EvaluatorRequestorClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectEvaluatorRequestor);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedContextClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedContextClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedContextClr2Java.cpp
deleted file mode 100644
index 9a6e6d8..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedContextClr2Java.cpp
+++ /dev/null
@@ -1,94 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-        FailedContextClr2Java::FailedContextClr2Java(JNIEnv *env, jobject jobjectFailedContext) {
-          ManagedLog::LOGGER->LogStart("FailedContextClr2Java::FailedContextClr2Java");
-
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectFailedContext = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectFailedContext));
-          jclass jclassFailedContext = env->GetObjectClass (_jobjectFailedContext);
-
-          jfieldID jidContextId = env->GetFieldID(jclassFailedContext, "contextId", "Ljava/lang/String;");
-          jfieldID jidEvaluatorId = env->GetFieldID(jclassFailedContext, "evaluatorId", "Ljava/lang/String;");
-          jfieldID jidParentId = env->GetFieldID(jclassFailedContext, "parentContextId", "Ljava/lang/String;");
-
-          _jstringContextId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectFailedContext, jidContextId)));
-          _jstringEvaluatorId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectFailedContext, jidEvaluatorId)));
-          _jstringParentContextId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectFailedContext, jidParentId)));
-
-          ManagedLog::LOGGER->LogStop("FailedContextClr2Java::FailedContextClr2Java");
-        }
-
-        IActiveContextClr2Java^ FailedContextClr2Java::GetParentContext() {
-          ManagedLog::LOGGER->LogStart("FailedContextClr2Java::GetParentContext");
-
-          JNIEnv *env = RetrieveEnv(_jvm);
-
-          jclass jclassFailedContext = env->GetObjectClass(_jobjectFailedContext);
-          jfieldID jidParentContext = env->GetFieldID(jclassFailedContext, "parentContext", "Lorg/apache/reef/javabridge/ActiveContextBridge;");
-          jobject jobjectParentContext = env->GetObjectField(_jobjectFailedContext, jidParentContext);
-          ManagedLog::LOGGER->LogStop("FailedContextClr2Java::GetParentContext");
-
-          return gcnew ActiveContextClr2Java(env, jobjectParentContext);
-        }
-
-        String^ FailedContextClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("FailedContextClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringContextId);
-        }
-
-        String^ FailedContextClr2Java::GetEvaluatorId() {
-          ManagedLog::LOGGER->Log("FailedContextClr2Java::GetEvaluatorId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringEvaluatorId);
-        }
-
-        String^ FailedContextClr2Java::GetParentId() {
-          ManagedLog::LOGGER->Log("FailedContextClr2Java::GetParentId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringParentContextId);
-        }
-
-        IEvaluatorDescriptor^ FailedContextClr2Java::GetEvaluatorDescriptor() {
-          ManagedLog::LOGGER->LogStart("FailedContextClr2Java::GetEvaluatorDescriptor");
-          return CommonUtilities::RetrieveEvaluatorDescriptor(_jobjectFailedContext, _jvm);
-        }
-
-        void FailedContextClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("FailedContextClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectFailedContext);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedEvaluatorClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedEvaluatorClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedEvaluatorClr2Java.cpp
deleted file mode 100644
index d79ecce..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedEvaluatorClr2Java.cpp
+++ /dev/null
@@ -1,72 +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.
- */
-#include "Clr2JavaImpl.h"
-
-using namespace JavaClrBridge;
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        FailedEvaluatorClr2Java::FailedEvaluatorClr2Java(JNIEnv *env, jobject jobjectFailedEvaluator) {
-          ManagedLog::LOGGER->LogStart("FailedEvaluatorClr2Java::FailedEvaluatorClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectFailedEvaluator = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectFailedEvaluator));
-
-          jclass jclassFailedEvaluator = env->GetObjectClass(_jobjectFailedEvaluator);
-          jfieldID jidEvaluatorId = env->GetFieldID(jclassFailedEvaluator, "evaluatorId", "Ljava/lang/String;");
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectFailedEvaluator, jidEvaluatorId)));
-          ManagedLog::LOGGER->LogStop("FailedEvaluatorClr2Java::FailedEvaluatorClr2Java");
-        }
-
-        IEvaluatorRequestorClr2Java^ FailedEvaluatorClr2Java::GetEvaluatorRequestor() {
-          ManagedLog::LOGGER->LogStart("FailedEvaluatorClr2Java::GetEvaluatorRequestor");
-          JNIEnv *env = RetrieveEnv(_jvm);
-
-          jclass jclassFailedEvaluator = env->GetObjectClass(_jobjectFailedEvaluator);
-          jfieldID jidEvaluatorRequestor = env->GetFieldID(jclassFailedEvaluator, "evaluatorRequestorBridge", "Lorg/apache/reef/javabridge/EvaluatorRequestorBridge;");
-          jobject jobjectEvaluatorRequestor = env->GetObjectField(_jobjectFailedEvaluator, jidEvaluatorRequestor);
-          ManagedLog::LOGGER->LogStop("FailedEvaluatorClr2Java::GetEvaluatorRequestor");
-          return gcnew EvaluatorRequestorClr2Java(env, jobjectEvaluatorRequestor);
-        }
-
-        String^ FailedEvaluatorClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("FailedEvaluatorClr2Java::GetId");
-
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-
-        void FailedEvaluatorClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("FailedEvaluatorClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectFailedEvaluator);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedTaskClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedTaskClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedTaskClr2Java.cpp
deleted file mode 100644
index 32414c6..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/FailedTaskClr2Java.cpp
+++ /dev/null
@@ -1,79 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        FailedTaskClr2Java::FailedTaskClr2Java(JNIEnv *env, jobject jobjectFailedTask) {
-          ManagedLog::LOGGER->LogStart("FailedTaskClr2Java::AllocatedEvaluatorClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectFailedTask = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectFailedTask));
-          ManagedLog::LOGGER->LogStop("FailedTaskClr2Java::AllocatedEvaluatorClr2Java");
-        }
-
-        IActiveContextClr2Java^ FailedTaskClr2Java::GetActiveContext() {
-          ManagedLog::LOGGER->LogStart("FailedTaskClr2Java::GetActiveContext");
-
-          JNIEnv *env = RetrieveEnv(_jvm);
-
-          jclass jclassFailedTask = env->GetObjectClass(_jobjectFailedTask);
-          jfieldID jidActiveContext = env->GetFieldID(jclassFailedTask, "jactiveContext", "Lorg/apache/reef/javabridge/ActiveContextBridge;");
-          jobject jobjectActiveContext = env->GetObjectField(_jobjectFailedTask, jidActiveContext);
-
-          ManagedLog::LOGGER->LogStop("FailedTaskClr2Java::GetActiveContext");
-          return gcnew ActiveContextClr2Java(env, jobjectActiveContext);
-        }
-
-        String^ FailedTaskClr2Java::GetString() {
-          ManagedLog::LOGGER->LogStart("FailedTaskClr2Java::GetString");
-          JNIEnv *env = RetrieveEnv(_jvm);
-
-          jclass jclassFailedTask = env->GetObjectClass (_jobjectFailedTask);
-          jmethodID jmidGetFailedTaskString = env->GetMethodID(jclassFailedTask, "getFailedTaskString", "()Ljava/lang/String;");
-
-          if (jmidGetFailedTaskString == NULL) {
-            ManagedLog::LOGGER->LogStart("jmidGetFailedTaskString is NULL");
-            return nullptr;
-          }
-          jstring jFailedTaskString = (jstring)env -> CallObjectMethod(
-                                        _jobjectFailedTask,
-                                        jmidGetFailedTaskString);
-          ManagedLog::LOGGER->LogStop("FailedTaskClr2Java::GetString");
-          return ManagedStringFromJavaString(env, jFailedTaskString);
-        }
-
-        void FailedTaskClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("FailedTaskClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectFailedTask);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/HttpServerClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/HttpServerClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/HttpServerClr2Java.cpp
deleted file mode 100644
index 90a930f..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/HttpServerClr2Java.cpp
+++ /dev/null
@@ -1,135 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        HttpServerClr2Java::HttpServerClr2Java(JNIEnv *env, jobject jhttpServerEventBridge) {
-          ManagedLog::LOGGER->LogStart("HttpServerClr2Java::HttpServerClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jhttpServerEventBridge = reinterpret_cast<jobject>(env->NewGlobalRef(jhttpServerEventBridge));
-          ManagedLog::LOGGER->LogStop("HttpServerClr2Java::HttpServerClr2Java");
-        }
-
-        String^ HttpServerClr2Java::GetQueryString() {
-          ManagedLog::LOGGER->LogStart("HttpServerClr2Java::GetQueryString");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclasshttpServerEventBridge = env->GetObjectClass (_jhttpServerEventBridge);
-          jmethodID jmidgetQueryString = env->GetMethodID(jclasshttpServerEventBridge, "getQueryString", "()Ljava/lang/String;");
-          if (jmidgetQueryString == NULL) {
-            fprintf(stdout, " jmidgetQueryString is NULL\n");
-            fflush (stdout);
-            return nullptr;
-          }
-          jstring jQueryString = (jstring) env->CallObjectMethod(
-                                   _jhttpServerEventBridge,
-                                   jmidgetQueryString);
-
-          ManagedLog::LOGGER->LogStop("HttpServerClr2Java::GetQueryString");
-          return ManagedStringFromJavaString(env, jQueryString);
-        }
-
-        array<byte>^ HttpServerClr2Java::GetQueryRequestData() {
-          ManagedLog::LOGGER->LogStart("HttpServerClr2Java::GetQueryRequestData");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclasshttpServerEventBridge = env->GetObjectClass (_jhttpServerEventBridge);
-          jmethodID jmidgetQueryBytes = env->GetMethodID(jclasshttpServerEventBridge, "getQueryRequestData", "()[B");
-
-          if (jmidgetQueryBytes == NULL) {
-            ManagedLog::LOGGER->Log("jmidgetQueryBytes is NULL");
-            return nullptr;
-          }
-          jbyteArray jQueryBytes = (jbyteArray) env->CallObjectMethod(
-                                     _jhttpServerEventBridge,
-                                     jmidgetQueryBytes);
-
-          ManagedLog::LOGGER->LogStop("HttpServerClr2Java::GetQueryRequestData");
-          return ManagedByteArrayFromJavaByteArray(env, jQueryBytes);
-        }
-
-        void HttpServerClr2Java::SetQueryResult(String^ queryResult) {
-          ManagedLog::LOGGER->LogStart("HttpServerClr2Java::SetQueryResult");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclasshttpServerEventBridge = env->GetObjectClass (_jhttpServerEventBridge);
-          jmethodID jmidsetQueryResult = env->GetMethodID(jclasshttpServerEventBridge, "setQueryResult", "(Ljava/lang/String;)V");
-
-          if (jmidsetQueryResult == NULL) {
-            ManagedLog::LOGGER->Log("jmidsetQueryResult is NULL");
-            return;
-          }
-          env->CallObjectMethod(
-            _jhttpServerEventBridge,
-            jmidsetQueryResult,
-            JavaStringFromManagedString(env, queryResult));
-          ManagedLog::LOGGER->LogStop("HttpServerClr2Java::SetQueryResult");
-        }
-
-        void HttpServerClr2Java::SetQueryResponseData(array<byte>^ queryResponseData) {
-          ManagedLog::LOGGER->LogStart("HttpServerClr2Java::SetQueryResponseData");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclasshttpServerEventBridge = env->GetObjectClass (_jhttpServerEventBridge);
-          jmethodID jmidsetQueryResult = env->GetMethodID(jclasshttpServerEventBridge, "setQueryResponseData", "([B)V");
-
-          if (jmidsetQueryResult == NULL) {
-            ManagedLog::LOGGER->Log("jmidsetQueryResult is NULL");
-            return;
-          }
-          env->CallObjectMethod(
-            _jhttpServerEventBridge,
-            jmidsetQueryResult,
-            JavaByteArrayFromManagedByteArray(env, queryResponseData));
-          ManagedLog::LOGGER->LogStop("HttpServerClr2Java::SetQueryResponseData");
-        }
-
-        void HttpServerClr2Java::SetUriSpecification(String^ uriSpecification) {
-          ManagedLog::LOGGER->LogStart("HttpServerClr2Java::SetUriSpecification");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclasshttpServerEventBridge = env->GetObjectClass (_jhttpServerEventBridge);
-          jmethodID jmidsetUriSpecification = env->GetMethodID(jclasshttpServerEventBridge, "setUriSpecification", "(Ljava/lang/String;)V");
-
-          if (jmidsetUriSpecification == NULL) {
-            ManagedLog::LOGGER->Log("jmidsetUriSpecification is NULL");
-            return;
-          }
-          env->CallObjectMethod(
-            _jhttpServerEventBridge,
-            jmidsetUriSpecification,
-            JavaStringFromManagedString(env, uriSpecification));
-          ManagedLog::LOGGER->LogStop("HttpServerClr2Java::SetUriSpecification");
-        }
-
-        void HttpServerClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("HttpServerClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jhttpServerEventBridge);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropAssemblies.h
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropAssemblies.h b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropAssemblies.h
deleted file mode 100644
index 2e80d71..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropAssemblies.h
+++ /dev/null
@@ -1,40 +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.
- */
-#pragma once
-#define _USING_V110_SDK71_
-
-#pragma warning( push )
-#pragma warning( disable : 4793 )
-#include <jni.h>
-#pragma warning( pop )
-#include "mscoree.h"
-#include "vcclr.h"
-
-using namespace System;
-using namespace System::Reflection;
-using namespace System::Collections::Generic;
-
-public ref class AssemblyUtil {
-  public :
-    static int _asmCount = 0;
-    static Dictionary<String^, System::Reflection::Assembly^>^  asms2 = gcnew Dictionary<String^, Assembly^>();
-    static void Add(Assembly^  myasm);
-    static Assembly^ FindAsm (String^ myasm);
-    static Assembly^ MyResolveEventHandler(Object^ sender, ResolveEventArgs^ args);
-};

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropLogger.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropLogger.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropLogger.cpp
deleted file mode 100644
index 418bd55..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropLogger.cpp
+++ /dev/null
@@ -1,50 +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.
- */
-#include "InteropLogger.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Interop {
-      /// currently not being used
-      InteropLogger::InteropLogger (JNIEnv* env, jobject jobjectInteropLogger) {
-        _env = env;
-        _jobjectInteropLogger = jobjectInteropLogger;
-        _jclassInteropLogger = env->GetObjectClass(jobjectInteropLogger);
-        wchar_t formatBuf[1024];
-        if (NULL == _jclassInteropLogger) {
-          swprintf_s (formatBuf, sizeof(formatBuf) / sizeof(wchar_t), L"_jclassInteropLogger %p\n", _jclassInteropLogger);
-          fwprintf (stdout, formatBuf);
-          fflush (stdout);
-        }
-        _jmidLog  = env->GetMethodID(_jclassInteropLogger, "Log", "(ILjava/lang/String;)V");
-        if (NULL == _jmidLog) {
-          swprintf_s (formatBuf, sizeof(formatBuf) / sizeof(wchar_t), L"_jmidLog %p\n", _jmidLog);
-          fwprintf (stdout, formatBuf);
-          fflush (stdout);
-        }
-
-      }
-      void InteropLogger::Log(TraceLevel traceLevel, String^ message) {
-        pin_ptr<const wchar_t> wch = PtrToStringChars(message);
-        jstring msg = _env->NewString((const jchar*)wch, message->Length);
-        _env->CallObjectMethod(_jobjectInteropLogger, _jmidLog, (int)traceLevel, msg);
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropLogger.h
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropLogger.h b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropLogger.h
deleted file mode 100644
index c2a2e80..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropLogger.h
+++ /dev/null
@@ -1,47 +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.
- */
-#include "InteropUtil.h"
-#include "org_apache_reef_javabridge_NativeInterop.h"
-#include "JavaClrBridge.h"
-#include "InteropAssemblies.h"
-#using "clrhandler.dll"
-
-using namespace System;
-using namespace System::IO;
-using namespace System::Collections::Generic;
-using namespace System::Runtime::InteropServices;
-using namespace System::Reflection;
-using namespace Microsoft::Reef::Interop;
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Interop {
-      public ref class InteropLogger : public ILogger {
-          jobject _jobjectInteropLogger;
-          jclass  _jclassInteropLogger;
-          jmethodID _jmidLog;
-          JNIEnv* _env;
-
-        public:
-          InteropLogger (JNIEnv* env, jobject jobjectInteropLogger);
-          virtual void Log(TraceLevel traceLevel, String^ message );
-      };
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropReturnInfo.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropReturnInfo.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropReturnInfo.cpp
deleted file mode 100644
index 7a0d35a..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropReturnInfo.cpp
+++ /dev/null
@@ -1,91 +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.
- */
-#include "InteropReturnInfo.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Interop {
-      // currently not being used
-      InteropReturnInfo::InteropReturnInfo (
-        JNIEnv*     env,
-        jobject     jobjectInteropReturnInfo,
-        ILogger^    logger
-      ) {
-        _env = env;
-        _jobjectInteropReturnInfo = jobjectInteropReturnInfo;
-        jclass thisClass = env->GetObjectClass(jobjectInteropReturnInfo);
-        wchar_t formatBuf[1024];
-
-        swprintf_s (formatBuf, sizeof(formatBuf) / sizeof(wchar_t), L"zzzzzzz this should be printed by java jmid 00 %p\n", thisClass);
-        logger->Log(TraceLevel::Error, gcnew String(formatBuf));
-        _jmidAddExceptionString = env->GetMethodID(thisClass, "addExceptionString", "(Ljava/lang/String;)V");
-        if (NULL == _jmidAddExceptionString) {
-          swprintf_s (formatBuf, sizeof(formatBuf) / sizeof(wchar_t), L"_jmidAddExceptionString %p\n", _jmidAddExceptionString);
-          fwprintf (stdout, formatBuf);
-          fflush (stdout);
-        }
-
-        _jmidHasExceptions = env->GetMethodID(thisClass, "hasExceptions", "()Z");
-        if (NULL == _jmidHasExceptions) {
-          swprintf_s (formatBuf, sizeof(formatBuf) / sizeof(wchar_t), L"_jmidHasExceptions %p\n", _jmidHasExceptions);
-          fwprintf (stdout, formatBuf);
-          fflush (stdout);
-        }
-
-        _jmidsetReturnCode = env->GetMethodID(thisClass, "setReturnCode", "(I)V");
-        if (NULL == _jmidsetReturnCode) {
-          swprintf_s (formatBuf, sizeof(formatBuf) / sizeof(wchar_t), L"_jmidsetReturnCode %p\n", _jmidsetReturnCode);
-          fwprintf (stdout, formatBuf);
-          fflush (stdout);
-        }
-
-        _jmidgetReturnCode = env->GetMethodID(thisClass, "getReturnCode", "()I");
-        if (NULL == _jmidgetReturnCode) {
-          swprintf_s (formatBuf, sizeof(formatBuf) / sizeof(wchar_t), L"_jmidgetReturnCode %p\n", _jmidgetReturnCode);
-          fwprintf (stdout, formatBuf);
-          fflush (stdout);
-        }
-
-
-      }
-
-      void InteropReturnInfo::AddExceptionString(String^ exceptionString) {
-        HasExceptions();
-        pin_ptr<const wchar_t> wch = PtrToStringChars(exceptionString);
-        jstring ret = _env->NewString((const jchar*)wch, exceptionString->Length);
-        _env->CallObjectMethod(_jobjectInteropReturnInfo, _jmidAddExceptionString, ret);
-        HasExceptions();
-      }
-
-      Boolean InteropReturnInfo::HasExceptions() {
-        jobject obj = _env->CallObjectMethod(_jobjectInteropReturnInfo, _jmidHasExceptions);
-        return ((int)obj) != 0;
-      }
-      void InteropReturnInfo::SetReturnCode(int rc) {
-        _env->CallObjectMethod(_jobjectInteropReturnInfo, _jmidsetReturnCode, rc);
-        GetReturnCode();
-      }
-      int InteropReturnInfo::GetReturnCode() {
-        jobject obj = _env->CallObjectMethod(_jobjectInteropReturnInfo, _jmidgetReturnCode);
-        return (int)obj;
-      }
-
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropReturnInfo.h
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropReturnInfo.h b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropReturnInfo.h
deleted file mode 100644
index 1278516..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropReturnInfo.h
+++ /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.
- */
-#include "InteropUtil.h"
-#include "org_apache_reef_javabridge_NativeInterop.h"
-#include "JavaClrBridge.h"
-#include "InteropAssemblies.h"
-#using "clrhandler.dll"
-
-using namespace System;
-using namespace System::IO;
-using namespace System::Collections::Generic;
-using namespace System::Runtime::InteropServices;
-using namespace System::Reflection;
-using namespace Microsoft::Reef::Interop;
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Interop {
-      public ref class InteropReturnInfo : public IInteropReturnInfo {
-          JNIEnv* _env;
-          jobject   _jobjectInteropReturnInfo;
-
-          jmethodID _jmidAddExceptionString;
-          jmethodID _jmidHasExceptions;
-          jmethodID _jmidsetReturnCode;
-          jmethodID _jmidgetReturnCode;
-
-        public:
-          InteropReturnInfo  (
-            JNIEnv* env,
-            jobject     jobjectInteropReturnInfo,
-            ILogger^    logger
-          );
-          virtual void AddExceptionString(String^ exceptionString);
-          virtual Boolean HasExceptions();
-          virtual void SetReturnCode(int rc);
-          virtual int GetReturnCode();
-      };
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropUtil.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropUtil.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropUtil.cpp
deleted file mode 100644
index be24f32..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropUtil.cpp
+++ /dev/null
@@ -1,129 +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.
- */
-#include "InteropUtil.h"
-#include "Clr2JavaImpl.h"
-
-using namespace System::Runtime::InteropServices;
-
-ref class ManagedLog {
-  internal:
-    static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>InteropUtil");
-};
-
-const wchar_t* UnicodeCppStringFromJavaString (
-  JNIEnv *env,
-  jstring javaString) {
-  const wchar_t* cppString = NULL;
-  if (NULL != javaString) {
-    cppString = (const wchar_t* )env->GetStringChars( javaString, 0);
-  }
-  return cppString;
-}
-
-void ReleaseUnicodeCppString (
-  JNIEnv*     env,
-  jstring     javaString,
-  jchar*      cppString) {
-  if (NULL != cppString) {
-    env->ReleaseStringChars(javaString, (jchar *)cppString);
-  }
-}
-
-String^ ManagedStringFromJavaString (
-  JNIEnv *env,
-  jstring javaString) {
-  if (javaString != NULL) {
-    int len = env->GetStringLength(javaString);
-    const wchar_t* wcsStr = UnicodeCppStringFromJavaString (env, javaString);
-    String^ managedStr = (NULL == wcsStr || 0 == len) ? nullptr : Marshal::PtrToStringUni((IntPtr)(unsigned short*)wcsStr, len);
-    ReleaseUnicodeCppString (env, javaString, (jchar*)wcsStr);
-    return managedStr;
-  }
-  return nullptr;
-}
-
-jstring JavaStringFromManagedString(
-  JNIEnv *env,
-  String^ managedString) {
-  pin_ptr<const wchar_t> wch = PtrToStringChars(managedString);
-  return env->NewString((const jchar*)wch, managedString->Length);
-}
-
-void HandleClr2JavaError(
-  JNIEnv *env,
-  String^ errorMessage,
-  jobject javaObject) {
-  ManagedLog::LOGGER->LogStart("InteropUtil::HandleClr2JavaError");
-
-  jclass javaClass = env->GetObjectClass (javaObject);
-  jmethodID jmidOnError = env->GetMethodID(javaClass, "onError", "(Ljava/lang/String;)V");
-
-  if (jmidOnError == NULL) {
-    ManagedLog::LOGGER->Log("jmidOnError is NULL");
-    return;
-  }
-  env -> CallObjectMethod(
-    javaObject,
-    jmidOnError,
-    JavaStringFromManagedString(env, errorMessage));
-  ManagedLog::LOGGER->LogStop("InteropUtil::HandleClr2JavaError");
-}
-
-array<byte>^ ManagedByteArrayFromJavaByteArray(
-  JNIEnv *env,
-  jbyteArray javaByteArray) {
-  if (javaByteArray != NULL) {
-    byte* bytes = (byte*)env->GetByteArrayElements (javaByteArray, FALSE);
-    int len = env->GetArrayLength(javaByteArray);
-    array<byte>^  managedByteArray = gcnew array<byte>(len);
-    //System::Array
-    for (int i = 0; i < len; i++) {
-      managedByteArray[i] = bytes[i];
-    }
-    return managedByteArray;
-  }
-  return nullptr;
-}
-
-jbyteArray JavaByteArrayFromManagedByteArray(
-  JNIEnv *env,
-  array<byte>^ managedByteArray) {
-  jbyteArray javaByteArray = env->NewByteArray(managedByteArray->Length);
-  pin_ptr<Byte> p = &managedByteArray[0];
-  env->SetByteArrayRegion(javaByteArray, 0, managedByteArray->Length, (jbyte*) p);
-  return javaByteArray;
-}
-
-jlongArray JavaLongArrayFromManagedLongArray(
-  JNIEnv *env,
-  array<unsigned long long>^ managedLongArray) {
-  jlongArray javaLongArray = env->NewLongArray(managedLongArray->Length);
-  pin_ptr<unsigned long long> p = &managedLongArray[0];
-  env->SetLongArrayRegion(javaLongArray, 0, managedLongArray->Length, (jlong*) p);
-  return javaLongArray;
-}
-
-JNIEnv* RetrieveEnv(JavaVM* jvm) {
-  JNIEnv *env;
-  if (jvm->AttachCurrentThread((void **) &env, NULL) != 0) {
-    ManagedLog::LOGGER->Log("cannot attach jni env to current jvm thread.");
-    throw;
-  }
-  return env;
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropUtil.h
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropUtil.h b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropUtil.h
deleted file mode 100644
index 2d95bcc..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/InteropUtil.h
+++ /dev/null
@@ -1,65 +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.
- */
-#pragma once
-#define _USING_V110_SDK71_
-
-#pragma warning( push )
-#pragma warning( disable : 4793 )
-#include <jni.h>
-#pragma warning( pop )
-#include "mscoree.h"
-#include "vcclr.h"
-
-using namespace System;
-
-const wchar_t* UnicodeCppStringFromJavaString (
-  JNIEnv *env,
-  jstring javaString);
-
-void ReleaseUnicodeCppString (
-  JNIEnv*     env,
-  jstring     javaString,
-  jchar*      cppString);
-
-String^ ManagedStringFromJavaString (
-  JNIEnv *env,
-  jstring javaString);
-
-jstring JavaStringFromManagedString(
-  JNIEnv *env,
-  String^ managedString);
-
-array<byte>^ ManagedByteArrayFromJavaByteArray(
-  JNIEnv *env,
-  jbyteArray javaByteArray);
-
-jbyteArray JavaByteArrayFromManagedByteArray(
-  JNIEnv *env,
-  array<byte>^ managedByteArray);
-
-jlongArray JavaLongArrayFromManagedLongArray(
-  JNIEnv *env,
-  array<unsigned long long>^ managedLongArray);
-
-JNIEnv* RetrieveEnv(JavaVM* jvm);
-
-void HandleClr2JavaError(
-  JNIEnv *env,
-  String^ errorMessage,
-  jobject javaObject);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.cpp
deleted file mode 100644
index 0a9353d..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.cpp
+++ /dev/null
@@ -1,492 +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.
- */
-#include "InteropUtil.h"
-#include "org_apache_reef_javabridge_NativeInterop.h"
-#include "JavaClrBridge.h"
-#include "InteropAssemblies.h"
-#include "InteropReturnInfo.h"
-#include "Clr2JavaImpl.h"
-#include "InteropLogger.h"
-#include "BinaryUtil.h"
-#include "malloc.h"
-
-using namespace System;
-using namespace System::IO;
-using namespace System::Collections::Generic;
-using namespace System::Runtime::InteropServices;
-using namespace System::Reflection;
-using namespace Microsoft::Reef::Driver::Bridge;
-
-ref class ManagedLog {
-  internal:
-    static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-};
-
-static void MarshalErrorToJava (
-  JNIEnv *env,
-  jobject  jerrorInfo,
-  int errorNo,
-  String^ exceptionString
-) {
-  jclass objectClass;
-  jfieldID fieldID;
-
-  objectClass = env->GetObjectClass(jerrorInfo);
-  fieldID = env->GetFieldID(objectClass, "errorNo", "I");
-  env->SetIntField (jerrorInfo, fieldID, errorNo);
-
-  pin_ptr<const wchar_t> wchExceptionString = PtrToStringChars(exceptionString);
-  jstring jexceptionString = env->NewString((const jchar*)wchExceptionString, exceptionString->Length);
-  fieldID = env->GetFieldID(objectClass, "exceptionString", "Ljava/lang/String;");
-  env->SetObjectField(jerrorInfo, fieldID, jexceptionString);
-}
-
-
-// Loading Clr Assembly. Note that we do not use ManagerLogger in this method since the
-// logger assembly needs to be loaded by this method before it can be used.
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_loadClrAssembly
-(
-  JNIEnv *env,
-  jclass  tobj,
-  jstring jfileName) {
-  try {
-    Console::Write("+Java_org_apache_reef_javabridge_NativeInterop_loadClrAssembly: ");
-    const wchar_t* charAsmName = UnicodeCppStringFromJavaString (env, jfileName);
-    int len = env->GetStringLength(jfileName);
-    wchar_t* fileName = (wchar_t* )_alloca((len + 2) * sizeof(wchar_t));
-    memcpy(fileName, charAsmName, (len + 2)* sizeof(wchar_t));
-    fileName[len] = 0;
-    String^ asmName = ManagedStringFromJavaString(env, jfileName);
-    Console::WriteLine("loading " + asmName);
-
-    BINARY_TYPE binaryType = IsManagedBinary(fileName);
-    if (binaryType == BINARY_TYPE_CLR) {
-      System::Reflection::Assembly^ asm1 = Assembly::LoadFrom(asmName);
-      AssemblyUtil::Add(asm1);
-    }
-    else if (binaryType == BINARY_TYPE_NATIVE) {
-      HANDLE handle = LoadLibraryW(fileName);
-    }
-  }
-  catch (System::Exception^ ex) {
-    // We do not propagate the exception back to Java to stop driver here
-    // since failure to load an assembly is not necesary devastating
-    Console::Write("Exceptions in Java_org_apache_reef_javabridge_NativeInterop_loadClrAssembly");
-    Console::Write(ex->Message);
-    Console::Write(ex->StackTrace);
-  }
-
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    CallClrSystemOnStartHandler
- * Signature: (Ljava/lang/String;)V
- */
-JNIEXPORT jlongArray JNICALL Java_org_apache_reef_javabridge_NativeInterop_CallClrSystemOnStartHandler
-(JNIEnv * env, jclass jclassx, jstring dateTimeString, jstring httpServerPort) {
-  try {
-    ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_CallClrSystemOnStartHandler");
-    const wchar_t* charConfig = UnicodeCppStringFromJavaString (env, dateTimeString);
-    int lenConfig = env->GetStringLength(dateTimeString);
-    String^  strConfig = Marshal::PtrToStringUni((IntPtr)(unsigned short*) charConfig, lenConfig);
-    DateTime dt = DateTime::Now;
-
-	const wchar_t* charPort = UnicodeCppStringFromJavaString (env, httpServerPort);
-    int lenPort = env->GetStringLength(httpServerPort);
-    String^  strPort = Marshal::PtrToStringUni((IntPtr)(unsigned short*) charPort, lenPort);
-
-    array<unsigned long long>^ handlers = ClrSystemHandlerWrapper::Call_ClrSystemStartHandler_OnStart(dt, strPort);
-    return JavaLongArrayFromManagedLongArray(env, handlers);
-  }
-  catch (System::Exception^ ex) {
-    // we cannot get error back to java here since we don't have an object to call back (although we idealy should...)
-    ManagedLog::LOGGER->LogError("Exceptions in Java_org_apache_reef_javabridge_NativeInterop_CallClrSystemOnStartHandler", ex);
-    return NULL;
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemAllocatedEvaluatorHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/AllocatedEvaluatorBridge;Lorg/apache/reef/javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemAllocatedEvaluatorHandlerOnNext
-(JNIEnv *env, jclass cls, jlong handle, jobject jallocatedEvaluatorBridge, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemAllocatedEvaluatorHandlerOnNext:");
-  AllocatedEvaluatorClr2Java^ allocatedEval = gcnew AllocatedEvaluatorClr2Java(env, jallocatedEvaluatorBridge);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemAllocatedEvaluatorHandler_OnNext(handle, allocatedEval);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemAllocatedEvaluatorHandler_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    allocatedEval -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemActiveContextHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/ActiveContextBridge;Lorg/apache/reef/javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemActiveContextHandlerOnNext
-(JNIEnv *env, jclass cls, jlong handle, jobject jactiveContextBridge, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemActiveContextHandlerOnNext");
-  ActiveContextClr2Java^ activeContextBrdige = gcnew ActiveContextClr2Java(env, jactiveContextBridge);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemActiveContextHandler_OnNext(handle, activeContextBrdige);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemActiveContextHandler_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    activeContextBrdige -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemEvaluatorRequstorHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/EvaluatorRequstorBridge;Lorg/apache/reef/javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemEvaluatorRequstorHandlerOnNext
-(JNIEnv *env, jclass cls, jlong handle, jobject jevaluatorRequestorBridge, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemEvaluatorRequstorHandlerOnNext");
-  EvaluatorRequestorClr2Java^ evaluatorRequestorBridge = gcnew EvaluatorRequestorClr2Java(env, jevaluatorRequestorBridge);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemEvaluatorRequestor_OnNext(handle, evaluatorRequestorBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemEvaluatorRequestor_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    evaluatorRequestorBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemTaskMessageHandlerOnNext
- * Signature: (J[BLorg/apache/reef/javabridge/TaskMessageBridge;Lorg/apache/reef/javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemTaskMessageHandlerOnNext
-(JNIEnv *env, jclass cls, jlong handle, jbyteArray jmessage, jobject jtaskMessageBridge, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemTaskMessageHandlerOnNext");
-  TaskMessageClr2Java^ taskMesageBridge = gcnew TaskMessageClr2Java(env, jtaskMessageBridge);
-  array<byte>^ message = ManagedByteArrayFromJavaByteArray(env, jmessage);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemTaskMessage_OnNext(handle, taskMesageBridge, message);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemTaskMessage_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    taskMesageBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSysteFailedTaskHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/FailedTaskBridge;Lorg/apache/reef/javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemFailedTaskHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jfailedTask, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemFailedTaskHandlerOnNext");
-  FailedTaskClr2Java^ failedTaskBridge = gcnew FailedTaskClr2Java(env, jfailedTask);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemFailedTask_OnNext(handler, failedTaskBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemTaskMessage_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    failedTaskBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSysteFailedTaskHandlerOnNext
- * Signature: (JLorg.apache.reef.javabridge/FailedTaskBridge;Lorg.apache.reef.javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemRunningTaskHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jrunningTask, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemRunningTaskHandlerOnNext");
-  RunningTaskClr2Java^ runningTaskBridge = gcnew RunningTaskClr2Java(env, jrunningTask);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemRunningTask_OnNext(handler, runningTaskBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemRunningTask_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    runningTaskBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemFailedEvaluatorHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/FailedEvaluatorBridge;Lorg/apache/reef/javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemFailedEvaluatorHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jfailedEvaluator, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemFailedEvaluatorHandlerOnNext");
-  FailedEvaluatorClr2Java^ failedEvaluatorBridge = gcnew FailedEvaluatorClr2Java(env, jfailedEvaluator);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemFailedEvaluator_OnNext(handler, failedEvaluatorBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemFailedEvaluator_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    failedEvaluatorBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemHttpServerEventHandlerOnHttpRequest
- * Signature: (JLorg/apache/reef/javabridge/HttpServerEventBridge;Lorg/apache/reef/javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemHttpServerHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jhttpServerEventBridge, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemHttpServerHandlerOnNext");
-  HttpServerClr2Java^ httpServerClr2Java = gcnew HttpServerClr2Java(env, jhttpServerEventBridge);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemHttpServer_OnNext(handler, httpServerClr2Java);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemHttpServer_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    httpServerClr2Java -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemCompletedTaskHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/CompletedTaskBridge;Lorg/apache/reef/javabridge/InteropLogger;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemCompletedTaskHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jcompletedTask, jobject jlogger) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemCompletedTaskHandlerOnNext");
-  CompletedTaskClr2Java^ completedTaskBridge = gcnew CompletedTaskClr2Java(env, jcompletedTask);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemCompletedTask_OnNext(handler, completedTaskBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemCompletedTask_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    completedTaskBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrBufferedLog
- * Signature: (ILjava/lang/String;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrBufferedLog
-(JNIEnv *env, jclass cls, jint logLevel, jstring message) {
-  try {
-    if (!JavaClrBridge::LoggerWrapper::initialized) {
-      ManagedLog::LOGGER->Log("Initializing CLRBufferedLogHandler in java bridge...");
-      JavaClrBridge::LoggerWrapper::logger->Listeners->Add(gcnew System::Diagnostics::ConsoleTraceListener());
-      JavaClrBridge::LoggerWrapper::initialized = true;
-    }
-
-    System::Diagnostics::TraceEventType eventType;
-    switch (logLevel) {
-    case 0:
-      eventType = System::Diagnostics::TraceEventType::Stop;
-      break;
-    case 1:
-      eventType = System::Diagnostics::TraceEventType::Error;
-      break;
-    case 2:
-      eventType = System::Diagnostics::TraceEventType::Warning;
-      break;
-    case 3:
-      eventType = System::Diagnostics::TraceEventType::Information;
-      break;
-    case 4:
-      eventType = System::Diagnostics::TraceEventType::Verbose;
-      break;
-    default:
-      eventType = System::Diagnostics::TraceEventType::Information;
-      break;
-
-    }
-
-    String^ msg = ManagedStringFromJavaString(env, message);
-    msg = System::String::Concat(System::DateTime::Now, msg);
-    JavaClrBridge::LoggerWrapper::logger->TraceEvent(eventType, 0, msg);
-  }
-  catch (System::Exception^ ex) {
-    ManagedLog::LOGGER->LogError("Exception in Java_javabridge_NativeInterop_ClrBufferedLog", ex);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemSupendedTaskHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/SuspendedTaskBridge;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemSupendedTaskHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jsuspendedTask) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemSupendedTaskHandlerOnNext");
-  SuspendedTaskClr2Java^ suspendedTaskBridge = gcnew SuspendedTaskClr2Java(env, jsuspendedTask);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemSuspendedTask_OnNext(handler, suspendedTaskBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemSuspendedTask_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    suspendedTaskBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemCompletdEvaluatorHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/CompletedEvaluatorBridge;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemCompletdEvaluatorHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jcompletedEvaluator) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemCompletdEvaluatorHandlerOnNext");
-  CompletedEvaluatorClr2Java^ completedEvaluatorBridge = gcnew CompletedEvaluatorClr2Java(env, jcompletedEvaluator);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemCompletedEvaluator_OnNext(handler, completedEvaluatorBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemSuspendedTask_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    completedEvaluatorBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemClosedContextHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/ClosedContextBridge;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemClosedContextHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jclosedContext) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemClosedContextHandlerOnNext");
-  ClosedContextClr2Java^ closedContextBridge = gcnew ClosedContextClr2Java(env, jclosedContext);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemClosedContext_OnNext(handler, closedContextBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemClosedContext_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    closedContextBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemFailedContextHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/FailedContextBridge;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemFailedContextHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jfailedContext) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemFailedContextHandlerOnNext");
-  FailedContextClr2Java^ failedContextBridge = gcnew FailedContextClr2Java(env, jfailedContext);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemFailedContext_OnNext(handler, failedContextBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemFailedContext_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    failedContextBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemContextMessageHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/ContextMessageBridge;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemContextMessageHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jcontextMessage) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemContextMessageHandlerOnNext");
-  ContextMessageClr2Java^ contextMessageBridge = gcnew ContextMessageClr2Java(env, jcontextMessage);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemContextMessage_OnNext(handler, contextMessageBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemContextMessage_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    contextMessageBridge -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemDriverRestartHandlerOnNext
- * Signature: (J)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemDriverRestartHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemDriverRestartHandlerOnNext");
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemDriverRestart_OnNext(handler);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemContextMessage_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    // we do not call back to Java for exception in .NET restart handler
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemDriverRestartActiveContextHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/ActiveContextBridge;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemDriverRestartActiveContextHandlerOnNext
-(JNIEnv *env, jclass cls, jlong handle, jobject jactiveContextBridge) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemDriverRestartActiveContextHandlerOnNext");
-  ActiveContextClr2Java^ activeContextBrdige = gcnew ActiveContextClr2Java(env, jactiveContextBridge);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemDriverRestartActiveContextHandler_OnNext(handle, activeContextBrdige);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemDriverRestartActiveContextHandler_OnNext";
-    ManagedLog::LOGGER -> LogError(errorMessage, ex);
-    activeContextBrdige -> OnError(errorMessage);
-  }
-}
-
-/*
- * Class:     org_apache_reef_javabridge_NativeInterop
- * Method:    ClrSystemDriverRestartRunningTaskHandlerOnNext
- * Signature: (JLorg/apache/reef/javabridge/RunningTaskBridge;)V
- */
-JNIEXPORT void JNICALL Java_org_apache_reef_javabridge_NativeInterop_ClrSystemDriverRestartRunningTaskHandlerOnNext
-(JNIEnv *env , jclass cls, jlong handler, jobject jrunningTask) {
-  ManagedLog::LOGGER->Log("+Java_org_apache_reef_javabridge_NativeInterop_ClrSystemDriverRestartRunningTaskHandlerOnNext");
-  RunningTaskClr2Java^ runningTaskBridge = gcnew RunningTaskClr2Java(env, jrunningTask);
-  try {
-    ClrSystemHandlerWrapper::Call_ClrSystemDriverRestartRunningTask_OnNext(handler, runningTaskBridge);
-  }
-  catch (System::Exception^ ex) {
-    String^ errorMessage = "Exception in Call_ClrSystemDriverRestartRunningTask_OnNext";
-    ManagedLog::LOGGER->LogError(errorMessage, ex);
-    runningTaskBridge -> OnError(errorMessage);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.h
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.h b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.h
deleted file mode 100644
index 61d9d0a..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.h
+++ /dev/null
@@ -1,33 +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.
- */
-// JavaClrBridge.h
-
-#pragma once
-
-using namespace System;
-
-namespace JavaClrBridge {
-    ref class LoggerWrapper
-    {
-    public:
-        static System::Diagnostics::TraceSource^ logger = 
-            gcnew System::Diagnostics::TraceSource("JavaCLRBridgeLogger", System::Diagnostics::SourceLevels::All);
-        static bool initialized = false;
-    };
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.sln
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.sln b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.sln
deleted file mode 100644
index d4b2aec..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.sln
+++ /dev/null
@@ -1,56 +0,0 @@
-
-Microsoft Visual Studio Solution File, Format Version 12.00
-# Visual Studio 2012
-Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "JavaClrBridge", "JavaClrBridge.vcxproj", "{2825FD53-350B-4294-8CFC-8DD2F4F4F285}"
-	ProjectSection(ProjectDependencies) = postProject
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD} = {443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}
-	EndProjectSection
-EndProject
-Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "ClrHandler", "..\..\..\CSharp\CSharp\ClrHandler\ClrHandler.csproj", "{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}"
-EndProject
-Global
-	GlobalSection(SolutionConfigurationPlatforms) = preSolution
-		Debug|Any CPU = Debug|Any CPU
-		Debug|Mixed Platforms = Debug|Mixed Platforms
-		Debug|Win32 = Debug|Win32
-		Debug|x64 = Debug|x64
-		Release|Any CPU = Release|Any CPU
-		Release|Mixed Platforms = Release|Mixed Platforms
-		Release|Win32 = Release|Win32
-		Release|x64 = Release|x64
-	EndGlobalSection
-	GlobalSection(ProjectConfigurationPlatforms) = postSolution
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Debug|Any CPU.ActiveCfg = Debug|Win32
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Debug|Mixed Platforms.ActiveCfg = Release|x64
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Debug|Mixed Platforms.Build.0 = Release|x64
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Debug|Mixed Platforms.Deploy.0 = Release|x64
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Debug|Win32.ActiveCfg = Debug|Win32
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Debug|Win32.Build.0 = Debug|Win32
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Debug|x64.ActiveCfg = Debug|x64
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Debug|x64.Build.0 = Debug|x64
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Release|Any CPU.ActiveCfg = Release|Win32
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Release|Mixed Platforms.ActiveCfg = Release|x64
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Release|Mixed Platforms.Build.0 = Release|x64
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Release|Win32.ActiveCfg = Release|Win32
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Release|Win32.Build.0 = Release|Win32
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Release|x64.ActiveCfg = Release|x64
-		{2825FD53-350B-4294-8CFC-8DD2F4F4F285}.Release|x64.Build.0 = Release|x64
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Debug|Any CPU.Build.0 = Debug|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Debug|Mixed Platforms.ActiveCfg = Debug|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Debug|Mixed Platforms.Build.0 = Debug|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Debug|Win32.ActiveCfg = Debug|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Debug|x64.ActiveCfg = Debug|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Debug|x64.Build.0 = Debug|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Release|Any CPU.ActiveCfg = Release|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Release|Any CPU.Build.0 = Release|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Release|Mixed Platforms.ActiveCfg = Release|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Release|Mixed Platforms.Build.0 = Release|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Release|Win32.ActiveCfg = Release|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Release|x64.ActiveCfg = Release|Any CPU
-		{443A7B61-5C91-4F67-9FCD-81BC6FABFDBD}.Release|x64.Build.0 = Release|Any CPU
-	EndGlobalSection
-	GlobalSection(SolutionProperties) = preSolution
-		HideSolutionNode = FALSE
-	EndGlobalSection
-EndGlobal

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj
deleted file mode 100644
index 08412d1..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj
+++ /dev/null
@@ -1,173 +0,0 @@
-<?xml version="1.0" encoding="utf-8"?>
-<Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
-  <PropertyGroup Condition="'$(Configuration)'==''">
-    <Configuration>Release</Configuration>
-  </PropertyGroup>
-  <ItemGroup Label="ProjectConfigurations">
-    <ProjectConfiguration Include="Release|x64">
-      <Configuration>Release</Configuration>
-      <Platform>x64</Platform>
-    </ProjectConfiguration>
-    <ProjectConfiguration Include="Debug|Win32">
-      <Configuration>Debug</Configuration>
-      <Platform>Win32</Platform>
-    </ProjectConfiguration>
-    <ProjectConfiguration Include="Debug|x64">
-      <Configuration>Debug</Configuration>
-      <Platform>x64</Platform>
-    </ProjectConfiguration>
-    <ProjectConfiguration Include="Release|Win32">
-      <Configuration>Release</Configuration>
-      <Platform>Win32</Platform>
-    </ProjectConfiguration>
-  </ItemGroup>
-  <PropertyGroup Label="DefaultDirectories">
-    <JAVA_HOME Condition=" '$(JAVA_HOME)' == '' ">c:\progra~1\java\jdk1.7.0_40</JAVA_HOME>
-  </PropertyGroup>
-  <PropertyGroup Label="Globals">
-    <ProjectGuid>{2825FD53-350B-4294-8CFC-8DD2F4F4F285}</ProjectGuid>
-    <TargetFrameworkVersion>v4.5</TargetFrameworkVersion>
-    <Keyword>ManagedCProj</Keyword>
-    <RootNamespace>Microsoft.Reef.Interop</RootNamespace>
-  </PropertyGroup>
-  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
-    <ConfigurationType>DynamicLibrary</ConfigurationType>
-    <UseDebugLibraries>false</UseDebugLibraries>
-    <PlatformToolset>v110</PlatformToolset>
-    <CLRSupport>true</CLRSupport>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="Configuration">
-    <ConfigurationType>DynamicLibrary</ConfigurationType>
-    <UseDebugLibraries>true</UseDebugLibraries>
-    <PlatformToolset>v110</PlatformToolset>
-    <CLRSupport>true</CLRSupport>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="Configuration">
-    <ConfigurationType>DynamicLibrary</ConfigurationType>
-    <UseDebugLibraries>true</UseDebugLibraries>
-    <PlatformToolset>v110</PlatformToolset>
-    <CLRSupport>true</CLRSupport>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
-    <ConfigurationType>DynamicLibrary</ConfigurationType>
-    <UseDebugLibraries>false</UseDebugLibraries>
-    <PlatformToolset>v110</PlatformToolset>
-    <CLRSupport>true</CLRSupport>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
-  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
-  <ImportGroup Label="ExtensionSettings">
-  </ImportGroup>
-  <ImportGroup Label="PropertySheets" Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
-  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="PropertySheets">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
-  <ImportGroup Label="PropertySheets" Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
-  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
-  <PropertyGroup Label="UserMacros" />
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <LinkIncremental>true</LinkIncremental>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
-    <LinkIncremental>true</LinkIncremental>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <LinkIncremental>false</LinkIncremental>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
-    <LinkIncremental>false</LinkIncremental>
-  </PropertyGroup>
-  <ItemDefinitionGroup>
-    <ClCompile>
-      <AdditionalUsingDirectories>..\..\..\..\..\target\classes</AdditionalUsingDirectories>
-      <AdditionalIncludeDirectories>..\..\..\..\..\..\reef-bridge-java\target\classes;$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32</AdditionalIncludeDirectories>
-    </ClCompile>
-    <Link>
-      <GenerateDebugInformation>false</GenerateDebugInformation>
-      <OutputFile>..\..\..\..\..\target\classes\$(TargetName)$(TargetExt)</OutputFile>
-    </Link>
-  </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <ClCompile>
-      <WarningLevel>Level3</WarningLevel>
-      <Optimization>Disabled</Optimization>
-      <PreprocessorDefinitions>WIN32;_DEBUG;%(PreprocessorDefinitions)</PreprocessorDefinitions>
-      <PrecompiledHeader>Use</PrecompiledHeader>
-    </ClCompile>
-  </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
-    <ClCompile>
-      <WarningLevel>Level3</WarningLevel>
-      <Optimization>Disabled</Optimization>
-      <PreprocessorDefinitions>WIN32;_DEBUG;%(PreprocessorDefinitions)</PreprocessorDefinitions>
-      <PrecompiledHeader>NotUsing</PrecompiledHeader>
-    </ClCompile>
-  </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <ClCompile>
-      <WarningLevel>Level3</WarningLevel>
-      <PreprocessorDefinitions>WIN32;NDEBUG;%(PreprocessorDefinitions)</PreprocessorDefinitions>
-      <PrecompiledHeader>NotUsing</PrecompiledHeader>
-    </ClCompile>
-  </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
-    <ClCompile>
-      <WarningLevel>Level3</WarningLevel>
-      <PreprocessorDefinitions>WIN32;%(PreprocessorDefinitions)</PreprocessorDefinitions>
-      <PrecompiledHeader>NotUsing</PrecompiledHeader>
-    </ClCompile>
-  </ItemDefinitionGroup>
-  <ItemGroup>
-    <Reference Include="System" />
-    <Reference Include="System.Data" />
-    <Reference Include="System.Xml" />
-  </ItemGroup>
-  <ItemGroup>
-    <ClInclude Include="BinaryUtil.h" />
-    <ClInclude Include="Clr2JavaImpl.h" />
-    <ClInclude Include="InteropAssemblies.h" />
-    <ClInclude Include="InteropLogger.h" />
-    <ClInclude Include="InteropReturnInfo.h" />
-    <ClInclude Include="InteropUtil.h" />
-  </ItemGroup>
-  <ItemGroup>
-    <ClCompile Include="ActiveContextClr2Java.cpp" />
-    <ClCompile Include="AllocatedEvaluatorClr2Java.cpp" />
-    <ClCompile Include="AssemblyInfo.cpp" />
-    <ClCompile Include="AssemblyUtil.cpp" />
-    <ClCompile Include="BinaryUtil.cpp" />
-    <ClCompile Include="ClosedContextClr2Java.cpp" />
-    <ClCompile Include="CommonUtilities.cpp" />
-    <ClCompile Include="CompletedEvaluatorClr2Java.cpp" />
-    <ClCompile Include="CompletedTaskClr2Java.cpp" />
-    <ClCompile Include="ContextMessageClr2Java.cpp" />
-    <ClCompile Include="EvaluatorRequestorClr2Java.cpp" />
-    <ClCompile Include="FailedContextClr2Java.cpp" />
-    <ClCompile Include="FailedEvaluatorClr2Java.cpp" />
-    <ClCompile Include="FailedTaskClr2Java.cpp" />
-    <ClCompile Include="HttpServerClr2Java.cpp" />
-    <ClCompile Include="InteropLogger.cpp" />
-    <ClCompile Include="InteropReturnInfo.cpp" />
-    <ClCompile Include="InteropUtil.cpp" />
-    <ClCompile Include="JavaClrBridge.cpp" />
-    <ClCompile Include="RunningTaskClr2Java.cpp" />
-    <ClCompile Include="SuspendedTaskClr2Java.cpp" />
-    <ClCompile Include="TaskMessageClr2Java.cpp" />
-  </ItemGroup>
-  <ItemGroup>
-    <Text Include="ReadMe.txt" />
-  </ItemGroup>
-  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
-  <ImportGroup Label="ExtensionTargets">
-  </ImportGroup>
-</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj.filters
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj.filters b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj.filters
deleted file mode 100644
index 5421846..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/JavaClrBridge.vcxproj.filters
+++ /dev/null
@@ -1,104 +0,0 @@
-<?xml version="1.0" encoding="utf-8"?>
-<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
-  <ItemGroup>
-    <Filter Include="Source Files">
-      <UniqueIdentifier>{4FC737F1-C7A5-4376-A066-2A32D752A2FF}</UniqueIdentifier>
-      <Extensions>cpp;c;cc;cxx;def;odl;idl;hpj;bat;asm;asmx</Extensions>
-    </Filter>
-    <Filter Include="Header Files">
-      <UniqueIdentifier>{93995380-89BD-4b04-88EB-625FBE52EBFB}</UniqueIdentifier>
-      <Extensions>h;hpp;hxx;hm;inl;inc;xsd</Extensions>
-    </Filter>
-  </ItemGroup>
-  <ItemGroup>
-    <ClInclude Include="InteropUtil.h">
-      <Filter>Header Files</Filter>
-    </ClInclude>
-    <ClInclude Include="InteropAssemblies.h">
-      <Filter>Header Files</Filter>
-    </ClInclude>
-    <ClInclude Include="InteropReturnInfo.h">
-      <Filter>Header Files</Filter>
-    </ClInclude>
-    <ClInclude Include="InteropLogger.h">
-      <Filter>Header Files</Filter>
-    </ClInclude>
-    <ClInclude Include="Clr2JavaImpl.h">
-      <Filter>Header Files</Filter>
-    </ClInclude>
-    <ClInclude Include="BinaryUtil.h">
-      <Filter>Header Files</Filter>
-    </ClInclude>
-  </ItemGroup>
-  <ItemGroup>
-    <ClCompile Include="JavaClrBridge.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="AssemblyInfo.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="InteropUtil.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="AssemblyUtil.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="InteropReturnInfo.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="InteropLogger.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="AllocatedEvaluatorClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="ActiveContextClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="EvaluatorRequestorClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="TaskMessageClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="BinaryUtil.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="FailedTaskClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="FailedEvaluatorClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="HttpServerClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="CompletedTaskClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="RunningTaskClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="SuspendedTaskClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="CompletedEvaluatorClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="ClosedContextClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="CommonUtilities.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="FailedContextClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-    <ClCompile Include="ContextMessageClr2Java.cpp">
-      <Filter>Source Files</Filter>
-    </ClCompile>
-  </ItemGroup>
-  <ItemGroup>
-    <Text Include="ReadMe.txt" />
-  </ItemGroup>
-</Project>
\ No newline at end of file


[2/4] incubator-reef git commit: [REEF-77] Remove REEF Bridge

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ManagedLogger.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ManagedLogger.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ManagedLogger.cpp
deleted file mode 100644
index 62e30b3..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ManagedLogger.cpp
+++ /dev/null
@@ -1,47 +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.
- */
-#include "Clr2JavaImpl.h"
-
-using namespace JavaClrBridge;
-
-namespace Microsoft
-{
-	namespace Reef
-	{
-		namespace Driver
-		{
-			namespace Bridge
-			{				
-				ManagedLogger::ManagedLogger(String^ className)
-				{
-					_logger = BridgeLogger::GetLogger(className);	
-				}
-				BridgeLogger^  ManagedLogger::GetLogger(String^ className)
-				{
-					if(_logger == nullptr)
-					{
-						_logger = BridgeLogger::GetLogger(className);
-					}
-					return _logger;
-				}
-
-			}
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ReadMe.txt
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ReadMe.txt b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ReadMe.txt
deleted file mode 100644
index 4e1b52f..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/ReadMe.txt
+++ /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.
-====
-
-========================================================================
-    DYNAMIC LINK LIBRARY : JavaClrBridge Project Overview
-========================================================================
-
-AppWizard has created this JavaClrBridge DLL for you.  
-
-This file contains a summary of what you will find in each of the files that
-make up your JavaClrBridge application.
-
-JavaClrBridge.vcxproj
-    This is the main project file for VC++ projects generated using an Application Wizard. 
-    It contains information about the version of Visual C++ that generated the file, and 
-    information about the platforms, configurations, and project features selected with the
-    Application Wizard.
-
-JavaClrBridge.vcxproj.filters
-    This is the filters file for VC++ projects generated using an Application Wizard. 
-    It contains information about the association between the files in your project 
-    and the filters. This association is used in the IDE to show grouping of files with
-    similar extensions under a specific node (for e.g. ".cpp" files are associated with the
-    "Source Files" filter).
-
-JavaClrBridge.cpp
-    This is the main DLL source file.
-
-JavaClrBridge.h
-    This file contains a class declaration.
-
-AssemblyInfo.cpp
-	Contains custom attributes for modifying assembly metadata.
-
-/////////////////////////////////////////////////////////////////////////////
-Other notes:
-
-AppWizard uses "TODO:" to indicate parts of the source code you
-should add to or customize.
-
-/////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/RunningTaskClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/RunningTaskClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/RunningTaskClr2Java.cpp
deleted file mode 100644
index 7ef6f08..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/RunningTaskClr2Java.cpp
+++ /dev/null
@@ -1,90 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-        RunningTaskClr2Java::RunningTaskClr2Java(JNIEnv *env, jobject jobjectRunningTask) {
-          ManagedLog::LOGGER->LogStart("RunningTaskClr2Java::RunningTaskClr2Java");
-
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectRunningTask = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectRunningTask));
-
-          jclass jclassRunningTask = env->GetObjectClass (_jobjectRunningTask);
-          jmethodID jmidGetId = env->GetMethodID(jclassRunningTask, "getId", "()Ljava/lang/String;");
-
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env -> CallObjectMethod(_jobjectRunningTask, jmidGetId)));
-          ManagedLog::LOGGER->LogStop("RunningTaskClr2Java::RunningTaskClr2Java");
-        }
-
-        IActiveContextClr2Java^ RunningTaskClr2Java::GetActiveContext() {
-          ManagedLog::LOGGER->LogStart("RunningTaskClr2Java::GetActiveContext");
-
-          JNIEnv *env = RetrieveEnv(_jvm);
-
-          jclass jclassRunningTask = env->GetObjectClass(_jobjectRunningTask);
-          jfieldID jidActiveContext = env->GetFieldID(jclassRunningTask, "jactiveContext", "Lorg/apache/reef/javabridge/ActiveContextBridge;");
-          jobject jobjectActiveContext = env->GetObjectField(_jobjectRunningTask, jidActiveContext);
-          ManagedLog::LOGGER->LogStop("RunningTaskClr2Java::GetActiveContext");
-
-          return gcnew ActiveContextClr2Java(env, jobjectActiveContext);
-        }
-
-        String^ RunningTaskClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("RunningTaskClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-
-        void RunningTaskClr2Java::Send(array<byte>^ message) {
-          ManagedLog::LOGGER->LogStart("RunningTaskClr2Java::Send");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassRunningTask = env->GetObjectClass(_jobjectRunningTask);
-          jmethodID jmidSend = env->GetMethodID(jclassRunningTask, "send", "([B)V");
-
-
-          if (jmidSend == NULL) {
-            ManagedLog::LOGGER->Log("jmidSend is NULL");
-            return;
-          }
-          env->CallObjectMethod(
-            _jobjectRunningTask,
-            jmidSend,
-            JavaByteArrayFromManagedByteArray(env, message));
-          ManagedLog::LOGGER->LogStop("RunningTaskClr2Java::Send");
-        }
-
-        void RunningTaskClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("RunningTaskClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectRunningTask);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/SuspendedTaskClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/SuspendedTaskClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/SuspendedTaskClr2Java.cpp
deleted file mode 100644
index 695e2b3..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/SuspendedTaskClr2Java.cpp
+++ /dev/null
@@ -1,83 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        SuspendedTaskClr2Java::SuspendedTaskClr2Java(JNIEnv *env, jobject jobjectSuspendedTask) {
-          ManagedLog::LOGGER->LogStart("SuspendedTaskClr2Java::SuspendedTaskClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectSuspendedTask = reinterpret_cast<jobject>(env->NewGlobalRef(jobjectSuspendedTask));
-
-          jclass jclassSuspendedTask = env->GetObjectClass (_jobjectSuspendedTask);
-          jfieldID jidTaskId = env->GetFieldID(jclassSuspendedTask, "taskId", "Ljava/lang/String;");
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectSuspendedTask, jidTaskId)));
-          ManagedLog::LOGGER->LogStop("SuspendedTaskClr2Java::SuspendedTaskClr2Java");
-        }
-
-        IActiveContextClr2Java^ SuspendedTaskClr2Java::GetActiveContext() {
-          ManagedLog::LOGGER->LogStart("SuspendedTaskClr2Java::GetActiveContext");
-          JNIEnv *env = RetrieveEnv(_jvm);
-
-          jclass jclassSuspendedTask = env->GetObjectClass (_jobjectSuspendedTask);
-          jfieldID jidActiveContext = env->GetFieldID(jclassSuspendedTask, "jactiveContext", "Lorg/apache/reef/javabridge/ActiveContextBridge;");
-          jobject jobjectActiveContext = env->GetObjectField(_jobjectSuspendedTask, jidActiveContext);
-          ManagedLog::LOGGER->LogStop("SuspendedTaskClr2Java::GetActiveContext");
-          return gcnew ActiveContextClr2Java(env, jobjectActiveContext);
-        }
-
-        String^ SuspendedTaskClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("SuspendedTaskClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-
-        array<byte>^ SuspendedTaskClr2Java::Get() {
-          ManagedLog::LOGGER->Log("SuspendedTaskClr2Java::Get");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          jclass jclassSuspendedTask = env->GetObjectClass (_jobjectSuspendedTask);
-          jmethodID jmidGet = env->GetMethodID(jclassSuspendedTask, "get", "()[B");
-
-          if (jmidGet == NULL) {
-            ManagedLog::LOGGER->Log("jmidGet is NULL");
-            return nullptr;
-          }
-          jbyteArray jMessage = (jbyteArray) env->CallObjectMethod(_jobjectSuspendedTask, jmidGet);
-          return ManagedByteArrayFromJavaByteArray(env, jMessage);
-        }
-
-        void SuspendedTaskClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("SuspendedTaskClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectSuspendedTask);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/TaskMessageClr2Java.cpp
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/TaskMessageClr2Java.cpp b/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/TaskMessageClr2Java.cpp
deleted file mode 100644
index 01d9471..0000000
--- a/reef-bridge-project/reef-bridge-clr/src/main/Cpp/CppBridge/JavaClrBridge/TaskMessageClr2Java.cpp
+++ /dev/null
@@ -1,58 +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.
- */
-#include "Clr2JavaImpl.h"
-
-namespace Microsoft {
-  namespace Reef {
-    namespace Driver {
-      namespace Bridge {
-        ref class ManagedLog {
-          internal:
-            static BridgeLogger^ LOGGER = BridgeLogger::GetLogger("<C++>");
-        };
-
-        TaskMessageClr2Java::TaskMessageClr2Java(JNIEnv *env, jobject jtaskMessage) {
-          ManagedLog::LOGGER->LogStart("TaskMessageClr2Java::TaskMessageClr2Java");
-          pin_ptr<JavaVM*> pJavaVm = &_jvm;
-          if (env->GetJavaVM(pJavaVm) != 0) {
-            ManagedLog::LOGGER->LogError("Failed to get JavaVM", nullptr);
-          }
-          _jobjectTaskMessage = reinterpret_cast<jobject>(env->NewGlobalRef(jtaskMessage));
-
-          jclass jclassTaskMessage = env->GetObjectClass (_jobjectTaskMessage);
-          jfieldID jidTaskId = env->GetFieldID(jclassTaskMessage, "taskId", "Ljava/lang/String;");
-          _jstringId = reinterpret_cast<jstring>(env->NewGlobalRef(env->GetObjectField(_jobjectTaskMessage, jidTaskId)));
-          ManagedLog::LOGGER->LogStop("TaskMessageClr2Java::TaskMessageClr2Java");
-        }
-
-        void TaskMessageClr2Java::OnError(String^ message) {
-          ManagedLog::LOGGER->Log("TaskMessageClr2Java::OnError");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          HandleClr2JavaError(env, message, _jobjectTaskMessage);
-        }
-
-        String^ TaskMessageClr2Java::GetId() {
-          ManagedLog::LOGGER->Log("TaskMessageClr2Java::GetId");
-          JNIEnv *env = RetrieveEnv(_jvm);
-          return ManagedStringFromJavaString(env, _jstringId);
-        }
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/pom.xml
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/pom.xml b/reef-bridge-project/reef-bridge-java/pom.xml
deleted file mode 100644
index c16e816..0000000
--- a/reef-bridge-project/reef-bridge-java/pom.xml
+++ /dev/null
@@ -1,116 +0,0 @@
-<?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:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
-         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>
-    <artifactId>reef-bridge-java</artifactId>
-    <name>REEF Bridge Java</name>
-    <description>Bridge between JVM and CLR.</description>
-
-
-    <parent>
-        <groupId>org.apache.reef</groupId>
-        <artifactId>reef-bridge-project</artifactId>
-        <version>0.10-incubating-SNAPSHOT</version>
-    </parent>
-
-    <dependencies>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-common</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-runtime-local</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-runtime-yarn</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-io</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-checkpoint</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>reef-webserver</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro</artifactId>
-        </dependency>
-    </dependencies>
-
-    <build>
-        <plugins>
-            <plugin>
-                <artifactId>maven-jar-plugin</artifactId>
-                <configuration>
-                    <archive>
-                        <manifest>
-                            <addClasspath>false</addClasspath>
-                            <classpathPrefix>lib/</classpathPrefix>
-                            <mainClass>org.apache.reef.javabridge.JavaBridge</mainClass>
-                        </manifest>
-                    </archive>
-                </configuration>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-antrun-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <phase>process-classes</phase>
-                        <goals>
-                            <goal>run</goal>
-                        </goals>
-                        <configuration>
-                            <exportAntProperties>true</exportAntProperties>
-                            <target>
-                                <property name="runtime_classpath" refid="maven.compile.classpath"/>
-                                <exec executable="javah">
-                                    <arg value="-cp"/>
-                                    <arg value="${runtime_classpath}"/>
-                                    <arg value="-d"/>
-                                    <arg value="${project.build.directory}/classes"/>
-                                    <arg value="org.apache.reef.javabridge.NativeInterop"/>
-                                </exec>
-                            </target>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-compiler-plugin</artifactId>
-            </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ActiveContextBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ActiveContextBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ActiveContextBridge.java
deleted file mode 100644
index a0dedf5..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ActiveContextBridge.java
+++ /dev/null
@@ -1,80 +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.javabridge;
-
-import org.apache.reef.driver.context.ActiveContext;
-import org.apache.reef.io.naming.Identifiable;
-import org.apache.reef.tang.ClassHierarchy;
-import org.apache.reef.tang.Configuration;
-import org.apache.reef.tang.formats.AvroConfigurationSerializer;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class ActiveContextBridge extends NativeBridge implements Identifiable {
-  private static final Logger LOG = Logger.getLogger(ActiveContextBridge.class.getName());
-
-  private ActiveContext jactiveContext;
-
-  private AvroConfigurationSerializer serializer;
-
-  private String contextId;
-
-  private String evaluatorId;
-
-  public ActiveContextBridge(ActiveContext activeContext) {
-    jactiveContext = activeContext;
-    serializer = new AvroConfigurationSerializer();
-    contextId = activeContext.getId();
-    evaluatorId = activeContext.getEvaluatorId();
-  }
-
-  public void submitTaskString(final String taskConfigurationString) {
-
-    if (taskConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty taskConfigurationString provided.");
-    }
-    ClassHierarchy clrClassHierarchy = Utilities.loadClassHierarchy(NativeInterop.CLASS_HIERARCHY_FILENAME);
-    Configuration taskConfiguration;
-    try {
-      taskConfiguration = serializer.fromString(taskConfigurationString, clrClassHierarchy);
-    } catch (final Exception e) {
-      final String message = "Unable to de-serialize CLR  task configurations using class hierarchy.";
-      LOG.log(Level.SEVERE, message, e);
-      throw new RuntimeException(message, e);
-    }
-    jactiveContext.submitTask(taskConfiguration);
-  }
-
-  public String getEvaluatorDescriptorSring() {
-    final String descriptorString = Utilities.getEvaluatorDescriptorString(jactiveContext.getEvaluatorDescriptor());
-    LOG.log(Level.FINE, "active context - serialized evaluator descriptor: " + descriptorString);
-    return descriptorString;
-  }
-
-  @Override
-  public void close() {
-    jactiveContext.close();
-  }
-
-  @Override
-  public String getId() {
-    return contextId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/AllocatedEvaluatorBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/AllocatedEvaluatorBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/AllocatedEvaluatorBridge.java
deleted file mode 100644
index 5d88355..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/AllocatedEvaluatorBridge.java
+++ /dev/null
@@ -1,141 +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.javabridge;
-
-import org.apache.reef.driver.evaluator.AllocatedEvaluator;
-import org.apache.reef.tang.ClassHierarchy;
-import org.apache.reef.tang.Configuration;
-import org.apache.reef.tang.formats.AvroConfigurationSerializer;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class AllocatedEvaluatorBridge extends NativeBridge {
-
-  private static final Logger LOG = Logger.getLogger(AllocatedEvaluatorBridge.class.getName());
-
-  private final AllocatedEvaluator jallocatedEvaluator;
-  private final AvroConfigurationSerializer serializer;
-  private final ClassHierarchy clrClassHierarchy;
-  private final String evaluatorId;
-  private final String nameServerInfo;
-
-  public AllocatedEvaluatorBridge(final AllocatedEvaluator allocatedEvaluator, final String serverInfo) {
-    jallocatedEvaluator = allocatedEvaluator;
-    serializer = new AvroConfigurationSerializer();
-    clrClassHierarchy = Utilities.loadClassHierarchy(NativeInterop.CLASS_HIERARCHY_FILENAME);
-    evaluatorId = allocatedEvaluator.getId();
-    nameServerInfo = serverInfo;
-  }
-
-  public void submitContextAndTaskString(final String contextConfigurationString, final String taskConfigurationString) {
-    if (contextConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty contextConfigurationString provided.");
-    }
-    if (taskConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty taskConfigurationString provided.");
-    }
-    Configuration contextConfiguration;
-    Configuration taskConfiguration;
-    try {
-      contextConfiguration = serializer.fromString(contextConfigurationString, clrClassHierarchy);
-      taskConfiguration = serializer.fromString(taskConfigurationString, clrClassHierarchy);
-    } catch (final Exception e) {
-      final String message = "Unable to de-serialize CLR context or task configurations using class hierarchy.";
-      LOG.log(Level.SEVERE, message, e);
-      throw new RuntimeException(message, e);
-    }
-    jallocatedEvaluator.submitContextAndTask(contextConfiguration, taskConfiguration);
-  }
-
-  public void submitContextString(final String contextConfigurationString) {
-    if (contextConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty contextConfigurationString provided.");
-    }
-    Configuration contextConfiguration;
-    try {
-      contextConfiguration = serializer.fromString(contextConfigurationString, clrClassHierarchy);
-    } catch (final Exception e) {
-      final String message = "Unable to de-serialize CLR context configurations using class hierarchy.";
-      LOG.log(Level.SEVERE, message, e);
-      throw new RuntimeException(message, e);
-    }
-    jallocatedEvaluator.submitContext(contextConfiguration);
-  }
-
-  public void submitContextAndServiceString(final String contextConfigurationString, final String serviceConfigurationString) {
-    if (contextConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty contextConfigurationString provided.");
-    }
-    if (serviceConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty serviceConfigurationString provided.");
-    }
-
-    Configuration contextConfiguration;
-    Configuration servicetConfiguration;
-    try {
-      contextConfiguration = serializer.fromString(contextConfigurationString, clrClassHierarchy);
-      servicetConfiguration = serializer.fromString(serviceConfigurationString, clrClassHierarchy);
-    } catch (final Exception e) {
-      final String message = "Unable to de-serialize CLR context or service  configurations using class hierarchy.";
-      LOG.log(Level.SEVERE, message, e);
-      throw new RuntimeException(message, e);
-    }
-    jallocatedEvaluator.submitContextAndService(contextConfiguration, servicetConfiguration);
-  }
-
-  public void submitContextAndServiceAndTaskString(
-      final String contextConfigurationString,
-      final String serviceConfigurationString,
-      final String taskConfigurationString) {
-    if (contextConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty contextConfigurationString provided.");
-    }
-    if (serviceConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty serviceConfigurationString provided.");
-    }
-    if (taskConfigurationString.isEmpty()) {
-      throw new RuntimeException("empty taskConfigurationString provided.");
-    }
-    Configuration contextConfiguration;
-    Configuration servicetConfiguration;
-    Configuration taskConfiguration;
-    try {
-      contextConfiguration = serializer.fromString(contextConfigurationString, clrClassHierarchy);
-      servicetConfiguration = serializer.fromString(serviceConfigurationString, clrClassHierarchy);
-      taskConfiguration = serializer.fromString(taskConfigurationString, clrClassHierarchy);
-    } catch (final Exception e) {
-      final String message = "Unable to de-serialize CLR context or service or task configurations using class hierarchy.";
-      LOG.log(Level.SEVERE, message, e);
-      throw new RuntimeException(message, e);
-    }
-    jallocatedEvaluator.submitContextAndServiceAndTask(contextConfiguration, servicetConfiguration, taskConfiguration);
-  }
-
-  public String getEvaluatorDescriptorSring() {
-    String descriptorString = Utilities.getEvaluatorDescriptorString(jallocatedEvaluator.getEvaluatorDescriptor());
-    LOG.log(Level.INFO, "allocated evaluator - serialized evaluator descriptor: " + descriptorString);
-    return descriptorString;
-  }
-
-  @Override
-  public void close() {
-    jallocatedEvaluator.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ClosedContextBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ClosedContextBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ClosedContextBridge.java
deleted file mode 100644
index 62f9ce7..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ClosedContextBridge.java
+++ /dev/null
@@ -1,81 +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.javabridge;
-
-import org.apache.reef.driver.context.ActiveContext;
-import org.apache.reef.driver.context.ClosedContext;
-import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
-import org.apache.reef.util.Optional;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class ClosedContextBridge extends NativeBridge implements ClosedContext {
-
-  private static final Logger LOG = Logger.getLogger(ClosedContextBridge.class.getName());
-
-  private final ClosedContext jcloseContext;
-  private final ActiveContextBridge parentContext;
-  private final String contextId;
-  private final String evaluatorId;
-  private final EvaluatorDescriptor evaluatorDescriptor;
-
-  public ClosedContextBridge(final ClosedContext closedContext) {
-    jcloseContext = closedContext;
-    parentContext = new ActiveContextBridge(closedContext.getParentContext());
-    contextId = closedContext.getId();
-    evaluatorId = closedContext.getEvaluatorId();
-    evaluatorDescriptor = closedContext.getEvaluatorDescriptor();
-  }
-
-  @Override
-  public String getId() {
-    return contextId;
-  }
-
-  @Override
-  public String getEvaluatorId() {
-    return evaluatorId;
-  }
-
-  @Override
-  public Optional<String> getParentId() {
-    return Optional.of(parentContext.getId());
-  }
-
-  @Override
-  public EvaluatorDescriptor getEvaluatorDescriptor() {
-    return evaluatorDescriptor;
-  }
-
-  @Override
-  public void close() throws Exception {
-  }
-
-  public String getEvaluatorDescriptorSring() {
-    String descriptorString = Utilities.getEvaluatorDescriptorString(evaluatorDescriptor);
-    LOG.log(Level.INFO, "Closed Context - serialized evaluator descriptor: " + descriptorString);
-    return descriptorString;
-  }
-
-  @Override
-  public ActiveContext getParentContext() {
-    return jcloseContext.getParentContext();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/CompletedEvaluatorBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/CompletedEvaluatorBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/CompletedEvaluatorBridge.java
deleted file mode 100644
index 0e300fd..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/CompletedEvaluatorBridge.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.javabridge;
-
-import org.apache.reef.driver.evaluator.CompletedEvaluator;
-import org.apache.reef.io.naming.Identifiable;
-
-public class CompletedEvaluatorBridge extends NativeBridge implements Identifiable {
-
-  private final CompletedEvaluator jcompletedEvaluator;
-
-  private final String evaluatorId;
-
-  public CompletedEvaluatorBridge(CompletedEvaluator completedEvaluator) {
-    jcompletedEvaluator = completedEvaluator;
-    evaluatorId = completedEvaluator.getId();
-  }
-
-  @Override
-  public String getId() {
-    return evaluatorId;
-  }
-
-  @Override
-  public void close() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/CompletedTaskBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/CompletedTaskBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/CompletedTaskBridge.java
deleted file mode 100644
index c95ca14..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/CompletedTaskBridge.java
+++ /dev/null
@@ -1,40 +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.javabridge;
-
-import org.apache.reef.driver.task.CompletedTask;
-
-public class CompletedTaskBridge extends NativeBridge {
-
-  private CompletedTask jcompletedTask;
-
-  private String taskId;
-
-  private ActiveContextBridge jactiveContext;
-
-  public CompletedTaskBridge(CompletedTask completedTask) {
-    jcompletedTask = completedTask;
-    taskId = completedTask.getId();
-    jactiveContext = new ActiveContextBridge(completedTask.getActiveContext());
-  }
-
-  @Override
-  public void close() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ContextMessageBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ContextMessageBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ContextMessageBridge.java
deleted file mode 100644
index eca4ba8..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/ContextMessageBridge.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.javabridge;
-
-import org.apache.reef.driver.context.ContextMessage;
-
-public class ContextMessageBridge extends NativeBridge implements ContextMessage {
-
-  private ContextMessage jcontextMessage;
-  private String contextMessageId;
-  private String messageSourceId;
-  private byte[] message;
-
-  public ContextMessageBridge(ContextMessage contextMessage) {
-    jcontextMessage = contextMessage;
-    contextMessageId = contextMessage.getId();
-    messageSourceId = contextMessage.getMessageSourceID();
-    message = contextMessage.get();
-  }
-
-  @Override
-  public void close() throws Exception {
-
-  }
-
-  @Override
-  public byte[] get() {
-    return message;
-  }
-
-  @Override
-  public String getId() {
-    return contextMessageId;
-  }
-
-  @Override
-  public String getMessageSourceID() {
-    return messageSourceId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/EvaluatorRequestorBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/EvaluatorRequestorBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/EvaluatorRequestorBridge.java
deleted file mode 100644
index a712fc4..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/EvaluatorRequestorBridge.java
+++ /dev/null
@@ -1,76 +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.javabridge;
-
-import org.apache.reef.driver.evaluator.EvaluatorRequest;
-import org.apache.reef.driver.evaluator.EvaluatorRequestor;
-import org.apache.reef.util.logging.LoggingScope;
-import org.apache.reef.util.logging.LoggingScopeFactory;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public final class EvaluatorRequestorBridge extends NativeBridge {
-  private static final Logger LOG = Logger.getLogger(EvaluatorRequestorBridge.class.getName());
-  private final boolean isBlocked;
-  private final EvaluatorRequestor jevaluatorRequestor;
-  private final LoggingScopeFactory loggingScopeFactory;
-
-  // accumulate how many evaluators have been submitted through this instance
-  // of EvaluatorRequestorBridge
-  private int clrEvaluatorsNumber;
-
-  public EvaluatorRequestorBridge(final EvaluatorRequestor evaluatorRequestor, final boolean isBlocked, final LoggingScopeFactory loggingScopeFactory) {
-    this.jevaluatorRequestor = evaluatorRequestor;
-    this.clrEvaluatorsNumber = 0;
-    this.isBlocked = isBlocked;
-    this.loggingScopeFactory = loggingScopeFactory;
-  }
-
-  public void submit(final int evaluatorsNumber, final int memory, final int virtualCore, final String rack) {
-    if (this.isBlocked) {
-      throw new RuntimeException("Cannot request additional Evaluator, this is probably because the Driver has crashed and restarted, and cannot ask for new container due to YARN-2433.");
-    }
-
-    if (rack != null && !rack.isEmpty()) {
-      LOG.log(Level.WARNING, "Ignoring rack preference.");
-    }
-
-    try (final LoggingScope ls = loggingScopeFactory.evaluatorRequestSubmitToJavaDriver(evaluatorsNumber)) {
-      clrEvaluatorsNumber += evaluatorsNumber;
-
-      final EvaluatorRequest request = EvaluatorRequest.newBuilder()
-        .setNumber(evaluatorsNumber)
-        .setMemory(memory)
-        .setNumberOfCores(virtualCore)
-        .build();
-
-      LOG.log(Level.FINE, "submitting evaluator request {0}", request);
-      jevaluatorRequestor.submit(request);
-    }
-  }
-
-  public int getEvaluatorNumber() {
-    return clrEvaluatorsNumber;
-  }
-
-  @Override
-  public void close() {
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedContextBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedContextBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedContextBridge.java
deleted file mode 100644
index dfed7f7..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedContextBridge.java
+++ /dev/null
@@ -1,83 +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.javabridge;
-
-import org.apache.reef.driver.context.ContextBase;
-import org.apache.reef.driver.context.FailedContext;
-import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
-import org.apache.reef.util.Optional;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class FailedContextBridge extends NativeBridge implements ContextBase {
-
-  private static final Logger LOG = Logger.getLogger(FailedContextBridge.class.getName());
-
-  private final ActiveContextBridge parentContext;
-  private final EvaluatorDescriptor evaluatorDescriptor;
-  private final String evaluatorId;
-  private final String contextId;
-  private final String parentContextId;
-  private final FailedContext jfailedContext;
-
-  public FailedContextBridge(final FailedContext failedContext) {
-    jfailedContext = failedContext;
-    evaluatorDescriptor = failedContext.getEvaluatorDescriptor();
-    evaluatorId = failedContext.getEvaluatorId();
-    contextId = failedContext.getId();
-    parentContext = failedContext.getParentContext().isPresent() ?
-        new ActiveContextBridge(failedContext.getParentContext().get()) : null;
-    parentContextId = parentContext != null ? parentContext.getId() : null;
-  }
-
-  @Override
-  public void close() throws Exception {
-  }
-
-  @Override
-  public String getId() {
-    return contextId;
-  }
-
-  @Override
-  public String getEvaluatorId() {
-    return evaluatorId;
-  }
-
-  @Override
-  public Optional<String> getParentId() {
-    if (parentContextId != null) {
-      return Optional.of(parentContextId);
-    } else {
-      return Optional.empty();
-    }
-  }
-
-  @Override
-  public EvaluatorDescriptor getEvaluatorDescriptor() {
-    return evaluatorDescriptor;
-  }
-
-  public String getEvaluatorDescriptorSring() {
-    String descriptorString = Utilities.getEvaluatorDescriptorString(evaluatorDescriptor);
-    LOG.log(Level.INFO, "Failed Context - serialized evaluator descriptor: " + descriptorString);
-    return descriptorString;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedEvaluatorBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedEvaluatorBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedEvaluatorBridge.java
deleted file mode 100644
index bae4946..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedEvaluatorBridge.java
+++ /dev/null
@@ -1,47 +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.javabridge;
-
-import org.apache.reef.driver.evaluator.EvaluatorRequestor;
-import org.apache.reef.driver.evaluator.FailedEvaluator;
-import org.apache.reef.util.logging.LoggingScopeFactory;
-
-import java.util.logging.Logger;
-
-public class FailedEvaluatorBridge extends NativeBridge {
-  private static final Logger LOG = Logger.getLogger(FailedEvaluatorBridge.class.getName());
-  private FailedEvaluator jfailedEvaluator;
-  private EvaluatorRequestorBridge evaluatorRequestorBridge;
-  private String evaluatorId;
-
-  public FailedEvaluatorBridge(FailedEvaluator failedEvaluator, EvaluatorRequestor evaluatorRequestor, boolean blockedForAdditionalEvaluator, final LoggingScopeFactory loggingScopeFactory) {
-    jfailedEvaluator = failedEvaluator;
-    evaluatorId = failedEvaluator.getId();
-    evaluatorRequestorBridge = new EvaluatorRequestorBridge(evaluatorRequestor, blockedForAdditionalEvaluator, loggingScopeFactory);
-  }
-
-  public int getNewlyRequestedEvaluatorNumber() {
-    return evaluatorRequestorBridge.getEvaluatorNumber();
-  }
-
-  @Override
-  public void close() {
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedTaskBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedTaskBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedTaskBridge.java
deleted file mode 100644
index 30383ca..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/FailedTaskBridge.java
+++ /dev/null
@@ -1,60 +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.javabridge;
-
-import org.apache.reef.driver.context.ActiveContext;
-import org.apache.reef.driver.task.FailedTask;
-import org.apache.reef.util.Optional;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class FailedTaskBridge extends NativeBridge {
-  private static final Logger LOG = Logger.getLogger(FailedTaskBridge.class.getName());
-
-  private FailedTask jfailedTask;
-  private ActiveContextBridge jactiveContext;
-
-  public FailedTaskBridge(FailedTask failedTask) {
-    jfailedTask = failedTask;
-    Optional<ActiveContext> activeContext = failedTask.getActiveContext();
-    jactiveContext = activeContext.isPresent() ? new ActiveContextBridge(activeContext.get()) : null;
-  }
-
-  public String getFailedTaskString() {
-    final String description = jfailedTask.getDescription().isPresent() ? jfailedTask.getDescription().get().replace("=", "").replace(",", "") : "";
-    final String cause = jfailedTask.getReason().isPresent() ? jfailedTask.getReason().get().toString().replace("=", "").replace(",", "") : "";
-    final String data = jfailedTask.getData().isPresent() ? new String(jfailedTask.getData().get()).replace("=", "").replace(",", "") : "";
-
-    // TODO: deserialize/serialize with proper Avro schema
-    final String poorSerializedString = "Identifier=" + jfailedTask.getId().replace("=", "").replace(",", "")
-        + ", Message=" + jfailedTask.getMessage().replace("=", "").replace(",", "")
-        + ", Description=" + description
-        + ", Cause=" + cause
-        + ", Data=" + data;
-
-    LOG.log(Level.INFO, "serialized failed task " + poorSerializedString);
-    return poorSerializedString;
-  }
-
-  @Override
-  public void close() {
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/HttpServerEventBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/HttpServerEventBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/HttpServerEventBridge.java
deleted file mode 100644
index 3e8a4e5..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/HttpServerEventBridge.java
+++ /dev/null
@@ -1,79 +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.javabridge;
-
-public final class HttpServerEventBridge extends NativeBridge {
-  private String queryString;
-  private byte[] queryRequestData;
-  private byte[] queryResponseData;
-  private String queryResult;
-  private String uriSpecification;
-
-  public HttpServerEventBridge(final String queryStr) {
-    this.queryString = queryStr;
-  }
-
-  public HttpServerEventBridge(final byte[] queryRequestData) {
-    this.queryRequestData = queryRequestData;
-  }
-
-  public final String getQueryString() {
-    return queryString;
-  }
-
-  public final void setQueryString(final String queryStr) {
-    this.queryString = queryStr;
-  }
-
-  public final String getQueryResult() {
-    return queryResult;
-  }
-
-  public final void setQueryResult(final String queryResult) {
-    this.queryResult = queryResult;
-  }
-
-  public final String getUriSpecification() {
-    return uriSpecification;
-  }
-
-  public final void setUriSpecification(final String uriSpecification) {
-    this.uriSpecification = uriSpecification;
-  }
-
-  public final byte[] getQueryRequestData() {
-    return queryRequestData;
-  }
-
-  public final void setQueryRequestData(final byte[] queryRequestData) {
-    this.queryRequestData = queryRequestData;
-  }
-
-  public final byte[] getQueryResponseData() {
-    return queryResponseData;
-  }
-
-  public final void setQueryResponseData(final byte[] responseData) {
-    queryResponseData = responseData;
-  }
-
-  @Override
-  public void close() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/InteropLogger.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/InteropLogger.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/InteropLogger.java
deleted file mode 100644
index 8bfbdfa..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/InteropLogger.java
+++ /dev/null
@@ -1,55 +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.javabridge;
-
-import java.util.HashMap;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class InteropLogger {
-  private static final Logger LOG = Logger.getLogger("InteropLogger");
-  HashMap<Integer, Level> levelHashMap;
-
-  {
-    levelHashMap = new HashMap<Integer, Level>();
-    levelHashMap.put(Level.OFF.intValue(), Level.OFF);
-    levelHashMap.put(Level.SEVERE.intValue(), Level.SEVERE);
-    levelHashMap.put(Level.WARNING.intValue(), Level.WARNING);
-    levelHashMap.put(Level.INFO.intValue(), Level.INFO);
-
-    levelHashMap.put(Level.CONFIG.intValue(), Level.CONFIG);
-    levelHashMap.put(Level.FINE.intValue(), Level.FINE);
-    levelHashMap.put(Level.FINER.intValue(), Level.FINER);
-
-    levelHashMap.put(Level.FINEST.intValue(), Level.FINEST);
-    levelHashMap.put(Level.ALL.intValue(), Level.ALL);
-  }
-
-  public void Log(int intLevel, String message) {
-    if (levelHashMap.containsKey(intLevel)) {
-      Level level = levelHashMap.get(intLevel);
-      LOG.log(level, message);
-    } else {
-
-      LOG.log(Level.WARNING, "Level " + intLevel + " is not a valid Log level");
-      LOG.log(Level.WARNING, message);
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/InteropReturnInfo.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/InteropReturnInfo.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/InteropReturnInfo.java
deleted file mode 100644
index 8ef59d6..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/InteropReturnInfo.java
+++ /dev/null
@@ -1,52 +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.javabridge;
-
-import java.util.ArrayList;
-
-public class InteropReturnInfo {
-
-  int returnCode;
-  ArrayList<String> exceptionList = new ArrayList<String>();
-
-  public void addExceptionString(String exceptionString) {
-    exceptionList.add(exceptionString);
-  }
-
-  public boolean hasExceptions() {
-    return !exceptionList.isEmpty();
-  }
-
-  public ArrayList<String> getExceptionList() {
-    return exceptionList;
-  }
-
-  public int getReturnCode() {
-    return returnCode;
-  }
-
-  public void setReturnCode(int rc) {
-    returnCode = rc;
-  }
-
-  public void reset() {
-    exceptionList = new ArrayList<String>();
-    returnCode = 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/JavaBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/JavaBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/JavaBridge.java
deleted file mode 100644
index ba438d8..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/JavaBridge.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.javabridge;
-
-public class JavaBridge {
-  private final static String CPP_BRIDGE = "JavaClrBridge";
-
-  static {
-    try {
-      System.loadLibrary(CPP_BRIDGE);
-    } catch (UnsatisfiedLinkError e) {
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/NativeBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/NativeBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/NativeBridge.java
deleted file mode 100644
index 4249ba7..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/NativeBridge.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.javabridge;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public abstract class NativeBridge implements AutoCloseable {
-
-  private static final Logger LOG = Logger.getLogger(ActiveContextBridge.class.getName());
-
-  public void onError(String errorMessage) {
-    LOG.log(Level.SEVERE, "Bridge received error from CLR: " + errorMessage);
-    throw new RuntimeException("Bridge received error from CLR: " + errorMessage);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/NativeInterop.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/NativeInterop.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/NativeInterop.java
deleted file mode 100644
index 5d99a3c..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/NativeInterop.java
+++ /dev/null
@@ -1,262 +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.javabridge;
-
-import org.apache.commons.compress.utils.IOUtils;
-
-import java.io.*;
-import java.util.Date;
-import java.util.HashMap;
-
-public class NativeInterop {
-
-  public static final String CLASS_HIERARCHY_FILENAME = "clrClassHierarchy.bin";
-  public static final String GLOBAL_LIBRARIES_FILENAME = "userSuppliedGlobalLibraries.txt";
-  public static final String EvaluatorRequestorKey = "EvaluatorRequestor";
-  public static final String AllocatedEvaluatorKey = "AllocatedEvaluator";
-  public static final String ActiveContextKey = "ActiveContext";
-  public static final String TaskMessageKey = "TaskMessage";
-  public static final String FailedTaskKey = "FailedTask";
-  public static final String FailedEvaluatorKey = "FailedEvaluator";
-  public static final String HttpServerKey = "HttpServerKey";
-  public static final String CompletedTaskKey = "CompletedTask";
-  public static final String RunningTaskKey = "RunningTask";
-  public static final String SuspendedTaskKey = "SuspendedTask";
-  public static final String CompletedEvaluatorKey = "CompletedEvaluator";
-  public static final String ClosedContextKey = "ClosedContext";
-  public static final String FailedContextKey = "FailedContext";
-  public static final String ContextMessageKey = "ContextMessage";
-  public static final String DriverRestartKey = "DriverRestart";
-  public static final String DriverRestartActiveContextKey = "DriverRestartActiveContext";
-  public static final String DriverRestartRunningTaskKey = "DriverRestartRunningTask";
-  public static final HashMap<String, Integer> Handlers = new HashMap<String, Integer>() {
-    {
-      put(EvaluatorRequestorKey, 0);
-      put(AllocatedEvaluatorKey, 1);
-      put(ActiveContextKey, 2);
-      put(TaskMessageKey, 3);
-      put(FailedTaskKey, 4);
-      put(FailedEvaluatorKey, 5);
-      put(HttpServerKey, 6);
-      put(CompletedTaskKey, 7);
-      put(RunningTaskKey, 8);
-      put(SuspendedTaskKey, 9);
-      put(CompletedEvaluatorKey, 10);
-      put(ClosedContextKey, 11);
-      put(FailedContextKey, 12);
-      put(ContextMessageKey, 13);
-      put(DriverRestartKey, 14);
-      put(DriverRestartActiveContextKey, 15);
-      put(DriverRestartRunningTaskKey, 16);
-    }
-  };
-
-
-  public static final int nHandlers = 17;
-  private static final String LIB_BIN = "/";
-  private static final String DLL_EXTENSION = ".dll";
-  private static final String CPP_BRIDGE = "JavaClrBridge";
-  private static final String tmpLoadingDirectory = System.getProperty("user.dir") + "/reef/CLRLoadingDirectory";
-  private static final String[] managedDlls = {
-      "ClrHandler",
-      "msvcr110",
-  };
-
-  static {
-    System.out.println("============== Driver Bridge initiated, loading DLLs at time " + new Date().toString() + "============== ");
-    new File(tmpLoadingDirectory).mkdir();
-    loadFromJar();
-    System.out.println("================== Done loading dlls for Driver at time " + new Date().toString() + " ================== \n");
-  }
-
-  public static native void loadClrAssembly(String filePath);
-
-  public static native void ClrBufferedLog(int level, String message);
-
-  public static native long[] CallClrSystemOnStartHandler(String dateTime, String httpServerPortNumber);
-
-  public static native void ClrSystemAllocatedEvaluatorHandlerOnNext(
-      long handle,
-      AllocatedEvaluatorBridge javaEvaluatorBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemActiveContextHandlerOnNext(
-      long handle,
-      ActiveContextBridge javaActiveContextBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemEvaluatorRequstorHandlerOnNext(
-      long handle,
-      EvaluatorRequestorBridge javaEvluatorRequstorBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemTaskMessageHandlerOnNext(
-      long handle,
-      byte[] mesage,
-      TaskMessageBridge javaTaskMessageBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemFailedTaskHandlerOnNext(
-      long handle,
-      FailedTaskBridge failedTaskBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemHttpServerHandlerOnNext(
-      long handle,
-      HttpServerEventBridge httpServerEventBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemFailedEvaluatorHandlerOnNext(
-      long handle,
-      FailedEvaluatorBridge failedEvaluatorBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemCompletedTaskHandlerOnNext(
-      long handle,
-      CompletedTaskBridge completedTaskBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemRunningTaskHandlerOnNext(
-      long handle,
-      RunningTaskBridge runningTaskBridge,
-      InteropLogger interopLogger
-  );
-
-  public static native void ClrSystemSupendedTaskHandlerOnNext(
-      long handle,
-      SuspendedTaskBridge suspendedTaskBridge
-  );
-
-  public static native void ClrSystemCompletdEvaluatorHandlerOnNext(
-      long handle,
-      CompletedEvaluatorBridge completedEvaluatorBridge
-  );
-
-  public static native void ClrSystemClosedContextHandlerOnNext(
-      long handle,
-      ClosedContextBridge closedContextBridge
-  );
-
-  public static native void ClrSystemFailedContextHandlerOnNext(
-      long handle,
-      FailedContextBridge failedContextBridge
-  );
-
-  public static native void ClrSystemContextMessageHandlerOnNext(
-      long handle,
-      ContextMessageBridge contextMessageBridge
-  );
-
-  public static native void ClrSystemDriverRestartHandlerOnNext(
-      long handle
-  );
-
-  public static native void ClrSystemDriverRestartActiveContextHandlerOnNext(
-      long handle,
-      ActiveContextBridge activeContextBridge
-  );
-
-  public static native void ClrSystemDriverRestartRunningTaskHandlerOnNext(
-      long handle,
-      RunningTaskBridge runningTaskBridge
-  );
-
-  private static void loadFromJar() {
-    // we need to put both DLLs to temp dir
-    loadLib(CPP_BRIDGE, false);
-    final File[] files = new File(System.getProperty("user.dir") + "/reef/global").listFiles(new FilenameFilter() {
-      public boolean accept(File dir, String name) {
-        return name.toLowerCase().endsWith(DLL_EXTENSION);
-      }
-    });
-    //System.out.println("Total dll files to load from " + System.getProperty("user.dir") + "/reef/global" + "  are: " + files.length );
-
-    for (int i = 0; i < files.length; i++) {
-      try {
-        final String fileName = files[i].getName();
-        String fileNameWithoutExtension = fileName;
-        if (fileName.indexOf(".") > 0) {
-          fileNameWithoutExtension = fileName.substring(0, fileName.lastIndexOf("."));
-        }
-        loadLib(fileNameWithoutExtension, true);
-      } catch (final Exception e) {
-        System.out.println("exception lading dll library " + e);
-        throw e;
-      }
-    }
-
-    for (int i = 0; i < managedDlls.length; i++) {
-      loadLib(managedDlls[i], true);
-    }
-  }
-
-  /**
-   * Puts library to temp dir and loads to memory
-   */
-
-  private static void loadLib(String name, final boolean copyOnly) {
-    name = name + DLL_EXTENSION;
-    //System.out.println("LOADING " + System.getProperty("user.dir") + "/reef/global/" + name );
-
-    try {
-      final String path = "/ReefDriverAppDlls/" + name;
-      //System.out.println("trying to load: " +  NativeInterop.class.getClass().getResource(path).getPath());
-      final InputStream in = NativeInterop.class.getResourceAsStream(path);
-      // always write to different location
-      final File fileOut = new File(tmpLoadingDirectory + LIB_BIN + name);
-      final OutputStream out = new FileOutputStream(fileOut);
-      //System.out.println("after new FileOutputStream(fileOut)");
-      if (null == in) {
-        // System.out.println("Cannot find " + path);
-        return;
-      }
-      if (out == null) {
-        System.out.println("** out is null");
-      }
-
-      IOUtils.copy(in, out);
-      in.close();
-      out.close();
-
-      if (false == copyOnly) {
-        //System.out.println("Loading DLL not copyonly");
-        System.load(fileOut.toString());
-        //System.out.println("Loading DLL not copyonly done");
-      } else {
-        //System.out.println("Loading DLL copyonly");
-        if (null == fileOut) {
-          System.out.println("fileOut is NULL");
-        }
-        //System.out.println("fileOut.toString() " + fileOut.toString());
-        NativeInterop.loadClrAssembly(fileOut.toString());
-        //System.out.println("Done Loading DLL " +  fileOut.toString());
-      }
-    } catch (final Exception e) {
-      throw new UnsatisfiedLinkError("Failed to load required DLL " + name);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/RunningTaskBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/RunningTaskBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/RunningTaskBridge.java
deleted file mode 100644
index 301c4fc..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/RunningTaskBridge.java
+++ /dev/null
@@ -1,49 +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.javabridge;
-
-import org.apache.reef.driver.context.ActiveContext;
-import org.apache.reef.driver.task.RunningTask;
-
-import java.util.logging.Logger;
-
-public class RunningTaskBridge extends NativeBridge {
-  private static final Logger LOG = Logger.getLogger(RunningTaskBridge.class.getName());
-
-  final private RunningTask jrunningTask;
-  final private ActiveContextBridge jactiveContext;
-
-  public RunningTaskBridge(RunningTask runningTask) {
-    jrunningTask = runningTask;
-    final ActiveContext activeContext = runningTask.getActiveContext();
-    jactiveContext = new ActiveContextBridge(activeContext);
-  }
-
-  public final String getId() {
-    return jrunningTask.getId();
-  }
-
-  public final void send(final byte[] message) {
-    jrunningTask.send(message);
-  }
-
-  @Override
-  public void close() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/SuspendedTaskBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/SuspendedTaskBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/SuspendedTaskBridge.java
deleted file mode 100644
index 16fa3d3..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/SuspendedTaskBridge.java
+++ /dev/null
@@ -1,54 +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.javabridge;
-
-import org.apache.reef.driver.task.SuspendedTask;
-import org.apache.reef.io.Message;
-import org.apache.reef.io.naming.Identifiable;
-
-public class SuspendedTaskBridge extends NativeBridge implements Identifiable, Message {
-
-  private final SuspendedTask jsuspendedTask;
-  private final String taskId;
-  private final ActiveContextBridge jactiveContext;
-
-  public SuspendedTaskBridge(SuspendedTask suspendedTask) {
-    jsuspendedTask = suspendedTask;
-    taskId = suspendedTask.getId();
-    jactiveContext = new ActiveContextBridge(jsuspendedTask.getActiveContext());
-  }
-
-  public ActiveContextBridge getActiveContext() {
-    return jactiveContext;
-  }
-
-  @Override
-  public void close() {
-  }
-
-  @Override
-  public String getId() {
-    return taskId;
-  }
-
-  @Override
-  public byte[] get() {
-    return jsuspendedTask.get();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/TaskMessageBridge.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/TaskMessageBridge.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/TaskMessageBridge.java
deleted file mode 100644
index 25b0478..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/TaskMessageBridge.java
+++ /dev/null
@@ -1,36 +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.javabridge;
-
-import org.apache.reef.driver.task.TaskMessage;
-
-public class TaskMessageBridge extends NativeBridge {
-  private TaskMessage jtaskMessage;
-  private String taskId;
-
-  // we don't really need to pass this around, just have this as place holder for future.
-  public TaskMessageBridge(TaskMessage taskMessage) {
-    jtaskMessage = taskMessage;
-    taskId = taskMessage.getId();
-  }
-
-  @Override
-  public void close() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/dee0a76f/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.java
----------------------------------------------------------------------
diff --git a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.java b/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.java
deleted file mode 100644
index e6d0849..0000000
--- a/reef-bridge-project/reef-bridge-java/src/main/java/org/apache/reef/javabridge/Utilities.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.javabridge;
-
-import org.apache.reef.driver.evaluator.EvaluatorDescriptor;
-import org.apache.reef.tang.ClassHierarchy;
-import org.apache.reef.tang.implementation.protobuf.ProtocolBufferClassHierarchy;
-import org.apache.reef.tang.proto.ClassHierarchyProto;
-
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.InetSocketAddress;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
-public class Utilities {
-  public static ClassHierarchy loadClassHierarchy(String classHierarchyFile) {
-    Path p = Paths.get(classHierarchyFile);
-    if (!Files.exists(p)) {
-      p = Paths.get(System.getProperty("user.dir") + "/reef/global/" + classHierarchyFile);
-    }
-    if (!Files.exists(p)) {
-      throw new RuntimeException("cannot find file " + p.toAbsolutePath());
-    }
-    try (final InputStream chin = new FileInputStream(p.toAbsolutePath().toString())) {
-      final ClassHierarchyProto.Node root = ClassHierarchyProto.Node.parseFrom(chin);
-      final ClassHierarchy ch = new ProtocolBufferClassHierarchy(root);
-      return ch;
-    } catch (final IOException e) {
-      final String message = "Unable to load class hierarchy from " + classHierarchyFile;
-      throw new RuntimeException(message, e);
-    }
-  }
-
-  public static String getEvaluatorDescriptorString(EvaluatorDescriptor evaluatorDescriptor) {
-    InetSocketAddress socketAddress = evaluatorDescriptor.getNodeDescriptor().getInetSocketAddress();
-    return "IP=" + socketAddress.getAddress() + ", Port=" + socketAddress.getPort() + ", HostName=" + socketAddress.getHostName() + ", Memory=" + evaluatorDescriptor.getMemory() + ", Core=" + evaluatorDescriptor.getNumberOfCores();
-  }
-}