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

[2/2] incubator-reef git commit: [REEF-228] and [REEF-225]: A Client API and Local implementation

[REEF-228] and [REEF-225]: A Client API and Local implementation

This PR contains a client API for REEF.NET in `Org.Apache.REEF.Client`.
It also adds the `Org.Apache.REEF.Client.Local` implementation. That
implementation is backed by a Java Tool
`org.apache.reef.bridge.client.LocalClient` which gets called in
`Org.Apache.REEF.Client.Local.LocalClient.Submit()`. The API can be seen
in action in the new project `Org.Apache.REEF.Examples.HelloREEF`.

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

Pull Request:
  This closes #134


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

Branch: refs/heads/master
Commit: 561e7a0addfa0d59d7c7ea15f34ff8766a68a176
Parents: 85bf1da
Author: Markus Weimer <we...@apache.org>
Authored: Wed Apr 1 19:14:10 2015 -0700
Committer: Julia Wang <jw...@yahoo.com>
Committed: Tue Apr 7 14:36:52 2015 -0700

----------------------------------------------------------------------
 .../Org.Apache.REEF.Bridge.JAR.csproj           |  17 +-
 .../Org.Apache.REEF.Bridge.JAR.nuspec           |   3 +-
 .../Org.Apache.REEF.Client/API/ClientFactory.cs |  48 ++++
 .../API/Exceptions/ClasspathException.cs        |  37 +++
 .../API/Exceptions/JavaNotFoundException.cs     |  37 +++
 .../Org.Apache.REEF.Client/API/IREEFClient.cs   |  34 +++
 .../Org.Apache.REEF.Client/API/JobSubmission.cs | 190 ++++++++++++++
 .../Common/ClientConstants.cs                   |  37 +++
 .../Common/DriverFolderPreparationHelper.cs     | 143 +++++++++++
 .../Org.Apache.REEF.Client/Common/FileSets.cs   | 166 ++++++++++++
 .../Common/JavaClientLauncher.cs                | 166 ++++++++++++
 .../Org.Apache.REEF.Client/Local/LocalClient.cs | 101 ++++++++
 .../Local/LocalRuntimeClientConfiguration.cs    |  54 ++++
 .../Local/Parameters/LocalRuntimeDirectory.cs   |  27 ++
 .../Local/Parameters/NumberOfEvaluators.cs      |  27 ++
 .../Org.Apache.REEF.Client.csproj               |  23 ++
 lang/cs/Org.Apache.REEF.Client/packages.config  |  24 ++
 lang/cs/Org.Apache.REEF.Common/Constants.cs     |   3 +
 .../Files/REEFFileNames.cs                      | 252 +++++++++++++++++++
 .../Org.Apache.REEF.Common.csproj               |   1 +
 .../Bridge/BridgeConfigurationProvider.cs       | 133 ++++++++++
 .../Bridge/ClrHandlerHelper.cs                  |  25 +-
 .../Bridge/ClrSystemHandlerWrapper.cs           |  51 ++--
 lang/cs/Org.Apache.REEF.Driver/Constants.cs     |   2 +
 .../Org.Apache.REEF.Driver.csproj               |   1 +
 .../App.config                                  |  24 ++
 .../HelloDriver.cs                              | 120 +++++++++
 .../HelloREEF.cs                                |  87 +++++++
 .../HelloTask.cs                                |  47 ++++
 .../Org.Apache.REEF.Examples.HelloREEF.csproj   |  77 ++++++
 .../Properties/AssemblyInfo.cs                  |  54 ++++
 .../Readme.md                                   |   9 +
 .../Functional/Bridge/TestBridgeClient.cs       |   2 +-
 lang/cs/Org.Apache.REEF.Tests/app.config        |  29 +++
 lang/cs/Org.Apache.REEF.sln                     | Bin 40196 -> 40556 bytes
 lang/java/reef-bridge-client/pom.xml            | 136 ++++++++++
 .../apache/reef/bridge/client/Constants.java    |  92 +++++++
 .../apache/reef/bridge/client/LocalClient.java  |  96 +++++++
 .../apache/reef/bridge/client/package-info.java |  22 ++
 .../reef/javabridge/generic/JobDriver.java      |  34 +--
 .../local/client/LocalJobSubmissionHandler.java |  41 +--
 .../client/PreparedDriverFolderLauncher.java    | 100 ++++++++
 pom.xml                                         |   5 +-
 43 files changed, 2476 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj b/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj
index b1f1159..e6f921b 100644
--- a/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj
+++ b/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.csproj
@@ -59,24 +59,33 @@ under the License.
           DependsOnTargets="CheckMavenInstall;CheckJavaInstall;CheckProtocInstall" 
           Outputs="$(OutputPath)">
     <PropertyGroup>
-      <!--The shaded jar of the bridge -->
+      <!--The shaded jar of the bridge (driver side) -->
       <Bridge_JAR_Name>reef-bridge-java-$(REEF_Version)-shaded.jar</Bridge_JAR_Name>
       <Bridge_JAR>$(REEF_Source_Folder)\lang\java\reef-bridge-java\target\$(Bridge_JAR_Name)</Bridge_JAR>
+      <!--The shaded jar of the bridge (client side) -->
+      <Client_JAR_Name>reef-bridge-client-$(REEF_Version)-shaded.jar</Client_JAR_Name>
+      <Client_JAR>$(REEF_Source_Folder)\lang\java\reef-bridge-client\target\$(Client_JAR_Name)</Client_JAR>
     </PropertyGroup>
     <Exec Command="$(M2_HOME)\bin\mvn -TC1 -DskipTests install" Condition="!Exists('$(Bridge_JAR)')" WorkingDirectory="$(REEF_Source_Folder)" />
     <Copy DestinationFolder="$(OutputPath)" SourceFiles="$(Bridge_JAR)" />
+    <Copy DestinationFolder="$(OutputPath)" SourceFiles="$(Client_JAR)" />
   </Target>
   <!--
     Executes 'mvn clean', if the JAR exists.
   -->
   <Target Name="Clean" DependsOnTargets="CheckMavenInstall;CheckJavaInstall;CheckProtocInstall">
     <PropertyGroup>
-      <!--The shaded jar of the bridge -->
+      <!--The shaded jar of the bridge (driver side) -->
       <Bridge_JAR_Name>reef-bridge-java-$(REEF_Version)-shaded.jar</Bridge_JAR_Name>
       <Bridge_JAR>$(REEF_Source_Folder)\lang\java\reef-bridge-java\target\$(Bridge_JAR_Name)</Bridge_JAR>
+      <!--The shaded jar of the bridge (client side) -->
+      <Client_JAR_Name>reef-bridge-client-$(REEF_Version)-shaded.jar</Client_JAR_Name>
+      <Client_JAR>$(REEF_Source_Folder)\lang\java\reef-bridge-client\target\$(Client_JAR_Name)</Client_JAR>
     </PropertyGroup>
     <Exec Command="$(M2_HOME)\bin\mvn -TC1 -DskipTests clean" Condition="Exists('$(Bridge_JAR)')" WorkingDirectory="$(REEF_Source_Folder)" />
     <Delete Files="$(OutputPath)\$(Bridge_JAR_Name)" />
+    <Delete Files="$(OutputPath)\$(Client_JAR_Name)" />
+
   </Target>
   <!--
     Standard Rebuild target: Clean, then build
@@ -93,6 +102,10 @@ under the License.
     <Line Include="line02"><Text>$file1 = $project.ProjectItems.Item("reef-bridge-java-$(ReefVer)-incubating-SNAPSHOT-shaded.jar")</Text></Line>
     <Line Include="line03"><Text>$copyToOutput1 = $file1.Properties.Item("CopyToOutputDirectory")</Text></Line>
     <Line Include="line04"><Text>$copyToOutput1.Value = 2</Text></Line>
+    <!--Copy the client JAR-->
+    <Line Include="line02"><Text>$file1 = $project.ProjectItems.Item("reef-bridge-client-$(ReefVer)-incubating-SNAPSHOT-shaded.jar")</Text></Line>
+    <Line Include="line03"><Text>$copyToOutput1 = $file1.Properties.Item("CopyToOutputDirectory")</Text></Line>
+    <Line Include="line04"><Text>$copyToOutput1.Value = 2</Text></Line>
     <LineText Include="%(Line.Text)" />
   </ItemGroup>
   <WriteLinesToFile

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.nuspec
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.nuspec b/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.nuspec
index c5c5131..9b0d5b9 100644
--- a/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.nuspec
+++ b/lang/cs/Org.Apache.REEF.Bridge.JAR/Org.Apache.REEF.Bridge.JAR.nuspec
@@ -29,7 +29,8 @@ under the License.
     <copyright>The Apache Software Foundation and the respective owners of the packaged libraries.</copyright>
   </metadata>
   <files>
-      <file src="..\bin\$Platform$\$Configuration$\Org.Apache.REEF.Bridge.JAR\reef-bridge-java-$REEF_Version$-shaded.jar" target="content" />
+    <file src="..\bin\$Platform$\$Configuration$\Org.Apache.REEF.Bridge.JAR\reef-bridge-java-$REEF_Version$-shaded.jar" target="content" />
+    <file src="..\bin\$Platform$\$Configuration$\Org.Apache.REEF.Bridge.JAR\reef-bridge-client-$REEF_Version$-shaded.jar" target="content" />
 	  <file src="..\bin\$Platform$\$Configuration$\Org.Apache.REEF.Bridge.JAR\install.ps1" target="tools" />
   </files>
 </package>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/API/ClientFactory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/ClientFactory.cs b/lang/cs/Org.Apache.REEF.Client/API/ClientFactory.cs
new file mode 100644
index 0000000..1c35b4a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/ClientFactory.cs
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.REEF.Client.Local;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+
+namespace Org.Apache.REEF.Client.API
+{
+    /// <summary>
+    /// Instantiates the client based on IConfiguration for the chosen runtime.
+    /// </summary>
+    public static class ClientFactory
+    {
+        /// <summary>
+        /// Creates a new instance of IREEFClient, based on the given Configuration.
+        /// </summary>
+        /// <remarks>
+        /// If the client itself uses Tang, it is a better design to have the IREEFClient injected into it. In order to make
+        /// that happen, mix in the appropriate runtime configuration into the client configuration.
+        /// </remarks>
+        /// <param name="runtimeClientConfiguration">
+        /// The client configuration. Typically, this will be created via e.g.
+        /// <seealso cref="LocalRuntimeClientConfiguration" />
+        /// </param>
+        /// <returns></returns>
+        public static IREEFClient GetClient(IConfiguration runtimeClientConfiguration)
+        {
+            return TangFactory.GetTang().NewInjector(runtimeClientConfiguration).GetInstance<IREEFClient>();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/API/Exceptions/ClasspathException.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Exceptions/ClasspathException.cs b/lang/cs/Org.Apache.REEF.Client/API/Exceptions/ClasspathException.cs
new file mode 100644
index 0000000..2c4e460
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Exceptions/ClasspathException.cs
@@ -0,0 +1,37 @@
+using System;
+
+/**
+ * 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.
+ */
+
+namespace Org.Apache.REEF.Client.API.Exceptions
+{
+    /// <summary>
+    /// Thrown when the client fails to assemble the classpath for the Java part of the client.
+    /// </summary>
+    public sealed class ClasspathException : Exception
+    {
+        public ClasspathException(string message) : base(message)
+        {
+        }
+
+        public ClasspathException(string message, Exception innerException) : base(message, innerException)
+        {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/API/Exceptions/JavaNotFoundException.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Exceptions/JavaNotFoundException.cs b/lang/cs/Org.Apache.REEF.Client/API/Exceptions/JavaNotFoundException.cs
new file mode 100644
index 0000000..493c08c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Exceptions/JavaNotFoundException.cs
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System.IO;
+
+namespace Org.Apache.REEF.Client.API.Exceptions
+{
+    /// <summary>
+    /// Thrown when the java installation cannot be found on the client.
+    /// </summary>
+    public sealed class JavaNotFoundException : FileNotFoundException
+    {
+        public JavaNotFoundException(string message) : base(message)
+        {
+        }
+
+        public JavaNotFoundException(string message, string fileName) : base(message, fileName)
+        {
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/API/IREEFClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/IREEFClient.cs b/lang/cs/Org.Apache.REEF.Client/API/IREEFClient.cs
new file mode 100644
index 0000000..56b0042
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/IREEFClient.cs
@@ -0,0 +1,34 @@
+/**
+ * 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.
+ */
+
+namespace Org.Apache.REEF.Client.API
+{
+    /// <summary>
+    /// Interface for job submission on a REEF cluster
+    /// </summary>
+    // ReSharper disable once InconsistentNaming
+    public interface IREEFClient
+    {
+        /// <summary>
+        /// Submit the job described in jobSubmission to the cluster.
+        /// </summary>
+        /// <param name="jobSubmission"></param>
+        void Submit(JobSubmission jobSubmission);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/API/JobSubmission.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/JobSubmission.cs b/lang/cs/Org.Apache.REEF.Client/API/JobSubmission.cs
new file mode 100644
index 0000000..440755e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobSubmission.cs
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.IO;
+using Org.Apache.REEF.Tang.Interface;
+
+namespace Org.Apache.REEF.Client.API
+{
+    /// <summary>
+    /// Captures a submission of a REEF Job to a cluster.
+    /// </summary>
+    public sealed class JobSubmission
+    {
+        private readonly ISet<IConfiguration> _driverConfigurations = new HashSet<IConfiguration>();
+        private readonly ISet<string> _globalAssemblies = new HashSet<string>();
+        private readonly ISet<string> _globalFiles = new HashSet<string>();
+        private readonly ISet<string> _localAssemblies = new HashSet<string>();
+        private readonly ISet<string> _localFiles = new HashSet<string>();
+        private int _driverMemory = 512;
+
+        /// <summary>
+        /// The assemblies to be made available to all containers.
+        /// </summary>
+        internal ISet<string> GlobalAssemblies
+        {
+            get { return _globalAssemblies; }
+        }
+
+        /// <summary>
+        /// The driver configurations
+        /// </summary>
+        internal ISet<IConfiguration> DriverConfigurations
+        {
+            get { return _driverConfigurations; }
+        }
+
+        internal ISet<string> GlobalFiles
+        {
+            get { return _globalFiles; }
+        }
+
+        internal ISet<string> LocalAssemblies
+        {
+            get { return _localAssemblies; }
+        }
+
+        internal ISet<string> LocalFiles
+        {
+            get { return _localFiles; }
+        }
+
+        internal int DriverMemory
+        {
+            get { return _driverMemory; }
+        }
+
+        /// <summary>
+        /// The Job's identifier
+        /// </summary>
+        public string JobIdentifier { get; private set; }
+
+        /// <summary>
+        /// Add a file to be made available in all containers.
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        public JobSubmission AddGlobalFile(string fileName)
+        {
+            _globalFiles.Add(fileName);
+            return this;
+        }
+
+        /// <summary>
+        /// Add a file to be made available only on the driver.
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        public JobSubmission AddLocalFile(string fileName)
+        {
+            _localFiles.Add(fileName);
+            return this;
+        }
+
+        /// <summary>
+        /// Add an assembly to be made available on all containers.
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        public JobSubmission AddGlobalAssembly(string fileName)
+        {
+            _globalAssemblies.Add(fileName);
+            return this;
+        }
+
+        /// <summary>
+        /// Add an assembly to the driver only.
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        public JobSubmission AddLocalAssembly(string fileName)
+        {
+            _localAssemblies.Add(fileName);
+            return this;
+        }
+
+        /// <summary>
+        /// Add a Configuration to the Driver.
+        /// </summary>
+        /// <param name="configuration"></param>
+        /// <returns></returns>
+        public JobSubmission AddDriverConfiguration(IConfiguration configuration)
+        {
+            _driverConfigurations.Add(configuration);
+            return this;
+        }
+
+        /// <summary>
+        /// Add the assembly needed for the given Type to the driver.
+        /// </summary>
+        /// <param name="type"></param>
+        /// <returns></returns>
+        public JobSubmission AddLocalAssemblyForType(Type type)
+        {
+            AddLocalAssembly(GetAssemblyPathForType(type));
+            return this;
+        }
+
+        /// <summary>
+        /// Add the assembly needed for the given Type to all containers.
+        /// </summary>
+        /// <param name="type"></param>
+        /// <returns></returns>
+        public JobSubmission AddGlobalAssemblyForType(Type type)
+        {
+            AddGlobalAssembly(GetAssemblyPathForType(type));
+            return this;
+        }
+
+        /// <summary>
+        /// Gives the job an identifier.
+        /// </summary>
+        /// <param name="id"></param>
+        /// <returns></returns>
+        public JobSubmission SetJobIdentifier(string id)
+        {
+            JobIdentifier = id;
+            return this;
+        }
+
+        /// <summary>
+        /// Sets the amount of memory (in MB) to allocate for the Driver.
+        /// </summary>
+        /// <param name="driverMemoryInMb">The amount of memory (in MB) to allocate for the Driver.</param>
+        /// <returns>this</returns>
+        public JobSubmission SetDriverMemory(int driverMemoryInMb)
+        {
+            _driverMemory = driverMemoryInMb;
+            return this;
+        }
+
+        /// <summary>
+        /// Finds the path to the assembly the given Type was loaded from.
+        /// </summary>
+        /// <param name="type"></param>
+        /// <returns></returns>
+        private static string GetAssemblyPathForType(Type type)
+        {
+            var path = Uri.UnescapeDataString(new UriBuilder(type.Assembly.CodeBase).Path);
+            return Path.GetFullPath(path);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/Common/ClientConstants.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/ClientConstants.cs b/lang/cs/Org.Apache.REEF.Client/Common/ClientConstants.cs
new file mode 100644
index 0000000..a5963d3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Common/ClientConstants.cs
@@ -0,0 +1,37 @@
+/**
+ * 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.
+ */
+
+namespace Org.Apache.REEF.Client.Common
+{
+    /// <summary>
+    /// Constants used by the O.A.R.Client project.
+    /// </summary>
+    internal static class ClientConstants
+    {
+        /// <summary>
+        /// The prefix of the JAR file containing the client logic.
+        /// </summary>
+        internal const string ClientJarFilePrefix = "reef-bridge-client-";
+
+        /// <summary>
+        /// The prefix of the JAR file to be copied to the driver.
+        /// </summary>
+        internal const string DriverJarFilePrefix = "reef-bridge-java-";
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs b/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
new file mode 100644
index 0000000..7e155b8
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
@@ -0,0 +1,143 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+using System;
+using System.IO;
+using System.Linq;
+using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Common;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Implementations.Configuration;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Client.Common
+{
+    /// <summary>
+    /// Helps prepare the driver folder.
+    /// </summary>
+    internal sealed class DriverFolderPreparationHelper
+    {
+        private const string DLLFileNameExtension = ".dll";
+        private const string EXEFileNameExtension = ".exe";
+        private static readonly Logger Logger = Logger.GetLogger(typeof(DriverFolderPreparationHelper));
+        private readonly AvroConfigurationSerializer _configurationSerializer;
+        private readonly REEFFileNames _fileNames;
+        private readonly FileSets _fileSets;
+
+        [Inject]
+        internal DriverFolderPreparationHelper(
+            REEFFileNames fileNames,
+            AvroConfigurationSerializer configurationSerializer,
+            FileSets fileSets)
+        {
+            _fileNames = fileNames;
+            _configurationSerializer = configurationSerializer;
+            _fileSets = fileSets;
+        }
+
+        /// <summary>
+        /// Prepares the working directory for a Driver in driverFolderPath.
+        /// </summary>
+        /// <param name="jobSubmission"></param>
+        /// <param name="driverFolderPath"></param>
+        internal void PrepareDriverFolder(JobSubmission jobSubmission, string driverFolderPath)
+        {
+            Logger.Log(Level.Info, "Preparing Driver filesystem layout in " + driverFolderPath);
+
+            // Setup the folder structure
+            CreateDefaultFolderStructure(driverFolderPath);
+
+            // Add the jobSubmission into that folder structure
+            _fileSets.AddJobFiles(jobSubmission);
+
+            // Create the driver configuration
+            CreateDriverConfiguration(jobSubmission, driverFolderPath);
+
+            // Add the REEF assemblies
+            AddAssemblies();
+
+            // Initiate the final copy
+            _fileSets.CopyToDriverFolder(driverFolderPath);
+
+            Logger.Log(Level.Info, "Done preparing Driver filesystem layout in " + driverFolderPath);
+        }
+
+        /// <summary>
+        /// Merges the Configurations in jobSubmission and serializes them into the right place within driverFolderPath,
+        /// assuming
+        /// that points to a Driver's working directory.
+        /// </summary>
+        /// <param name="jobSubmission"></param>
+        /// <param name="driverFolderPath"></param>
+        internal void CreateDriverConfiguration(JobSubmission jobSubmission, string driverFolderPath)
+        {
+            var driverConfiguration = Configurations.Merge(jobSubmission.DriverConfigurations.ToArray());
+
+            _configurationSerializer.ToFile(driverConfiguration,
+                Path.Combine(driverFolderPath, _fileNames.GetClrDriverConfigurationPath()));
+
+            // TODO: Remove once we cleaned up the Evaluator to not expect this [REEF-216]
+            _configurationSerializer.ToFile(driverConfiguration,
+                Path.Combine(driverFolderPath, _fileNames.GetGlobalFolderPath(), Constants.ClrBridgeRuntimeConfiguration));
+        }
+
+        /// <summary>
+        /// Creates the driver folder structure in this given folder as the root
+        /// </summary>
+        /// <param name="driverFolderPath"></param>
+        internal void CreateDefaultFolderStructure(string driverFolderPath)
+        {
+            Directory.CreateDirectory(Path.Combine(driverFolderPath, _fileNames.GetReefFolderName()));
+            Directory.CreateDirectory(Path.Combine(driverFolderPath, _fileNames.GetLocalFolderPath()));
+            Directory.CreateDirectory(Path.Combine(driverFolderPath, _fileNames.GetGlobalFolderPath()));
+        }
+
+        /// <summary>
+        /// Adds all Assemlies to the Global folder in the Driver.
+        /// </summary>
+        private void AddAssemblies()
+        {
+            // TODO: Be more precise, e.g. copy the JAR only to the driver.
+            var assemblies = Directory.GetFiles(@".\").Where(IsAssemblyToCopy);
+            _fileSets.AddToGlobalFiles(assemblies);
+        }
+
+        /// <summary>
+        /// Returns true, if the given file path references a DLL or EXE or JAR.
+        /// </summary>
+        /// <param name="filePath"></param>
+        /// <returns></returns>
+        private static Boolean IsAssemblyToCopy(string filePath)
+
+        {
+            var fileName = Path.GetFileName(filePath);
+            if (string.IsNullOrWhiteSpace(fileName))
+            {
+                return false;
+            }
+            var lowerCasePath = fileName.ToLower();
+            return lowerCasePath.EndsWith(DLLFileNameExtension) ||
+                   lowerCasePath.EndsWith(EXEFileNameExtension) ||
+                   lowerCasePath.StartsWith(ClientConstants.DriverJarFilePrefix);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/Common/FileSets.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/FileSets.cs b/lang/cs/Org.Apache.REEF.Client/Common/FileSets.cs
new file mode 100644
index 0000000..e337112
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Common/FileSets.cs
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System.Collections.Generic;
+using System.IO;
+using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Client.Common
+{
+    /// <summary>
+    /// Manages the files to be copied to driver folder.
+    /// </summary>
+    internal sealed class FileSets
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof (FileSets));
+        private readonly REEFFileNames _fileNames;
+
+        private readonly ISet<string> _globalFileSet =
+            new HashSet<string>(CaseIgnoringStringEqualityComparer.TheInstance);
+
+        private readonly ISet<string> _localFileSet = new HashSet<string>(CaseIgnoringStringEqualityComparer.TheInstance);
+
+        [Inject]
+        internal FileSets(REEFFileNames fileNames)
+        {
+            _fileNames = fileNames;
+        }
+
+        /// <summary>
+        /// Adds a file to be added to the driver's local folder.
+        /// </summary>
+        /// <remarks>Empty strings are quietly ignored.</remarks>
+        /// <param name="path"></param>
+        internal void AddToLocalFiles(string path)
+        {
+            if (string.IsNullOrWhiteSpace(path))
+            {
+                Logger.Log(Level.Warning, "Skipping: " + path);
+            }
+            else
+            {
+                _localFileSet.Add(Path.GetFullPath(path));
+            }
+        }
+
+        /// <summary>
+        /// Adds all the referenced files to the driver's local folder.
+        /// </summary>
+        /// <remarks>Empty strings are quietly ignored.</remarks>
+        /// <param name="paths"></param>
+        internal void AddToLocalFiles(IEnumerable<string> paths)
+        {
+            foreach (var path in paths)
+            {
+                AddToLocalFiles(path);
+            }
+        }
+
+        /// <summary>
+        /// Adds a file to be added to the driver's global folder.
+        /// </summary>
+        /// <remarks>Empty strings are quietly ignored.</remarks>
+        /// <param name="path"></param>
+        internal void AddToGlobalFiles(string path)
+        {
+            if (string.IsNullOrWhiteSpace(path))
+            {
+                Logger.Log(Level.Warning, "Skipping: " + path);
+            }
+            else
+            {
+                _globalFileSet.Add(Path.GetFullPath(path));
+            }
+        }
+
+        /// <summary>
+        /// Adds all the referenced files to the driver's global folder.
+        /// </summary>
+        /// <remarks>Empty strings are quietly ignored.</remarks>
+        /// <param name="paths"></param>
+        internal void AddToGlobalFiles(IEnumerable<string> paths)
+        {
+            foreach (var path in paths)
+            {
+                AddToGlobalFiles(path);
+            }
+        }
+
+        /// <summary>
+        /// Copies the files captured to the right places, given that driverFolderPath points to the root folder of the driver.
+        /// </summary>
+        /// <param name="driverFolderPath"></param>
+        internal void CopyToDriverFolder(string driverFolderPath)
+        {
+            var localFolderPath = Path.Combine(driverFolderPath, _fileNames.GetLocalFolderPath());
+            CopyAllToFolder(_localFileSet, localFolderPath);
+            var globalFolderPath = Path.Combine(driverFolderPath, _fileNames.GetGlobalFolderPath());
+            CopyAllToFolder(_globalFileSet, globalFolderPath);
+        }
+
+        /// <summary>
+        /// Adds all the files referenced in the given JobSubmission
+        /// </summary>
+        /// <param name="submission"></param>
+        internal void AddJobFiles(JobSubmission submission)
+        {
+            AddToGlobalFiles(submission.GlobalFiles);
+            AddToGlobalFiles(submission.GlobalAssemblies);
+            AddToLocalFiles(submission.LocalFiles);
+            AddToLocalFiles(submission.LocalAssemblies);
+        }
+
+        /// <summary>
+        /// Copies all the files referenced in sourcePaths to destinationPath
+        /// </summary>
+        /// <param name="sourcePaths"></param>
+        /// <param name="destinationPath"></param>
+        private static void CopyAllToFolder(IEnumerable<string> sourcePaths, string destinationPath)
+        {
+            foreach (var file in sourcePaths)
+            {
+                var destinationFileName = Path.GetFileName(file);
+                var destination = Path.Combine(destinationPath, destinationFileName);
+                Logger.Log(Level.Verbose, "Copying {0} to {1}", file, destination);
+                File.Copy(file, destination);
+            }
+        }
+    }
+
+    /// <summary>
+    /// An EqualityComparer for strings that ignores the case when comparing.
+    /// </summary>
+    internal sealed class CaseIgnoringStringEqualityComparer : IEqualityComparer<string>
+    {
+        internal static CaseIgnoringStringEqualityComparer TheInstance = new CaseIgnoringStringEqualityComparer();
+
+        public bool Equals(string stringOne, string stringTwo)
+        {
+            return stringOne.ToLower().Equals(stringTwo.ToLower());
+        }
+
+        public int GetHashCode(string s)
+        {
+            return s.ToLower().GetHashCode();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs b/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs
new file mode 100644
index 0000000..2fa83fc
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Common/JavaClientLauncher.cs
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.IO;
+using System.Linq;
+using Org.Apache.REEF.Client.API.Exceptions;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Diagnostics;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Client.Common
+{
+    /// <summary>
+    /// Helper class to launch the java side of the various clients.
+    /// </summary>
+    internal class JavaClientLauncher
+    {
+        /// <summary>
+        /// The folder in which we search for the client jar.
+        /// </summary>
+        private const string JarFolder = "./";
+
+        private static readonly Logger Logger = Logger.GetLogger(typeof (JavaClientLauncher));
+
+        [Inject]
+        private JavaClientLauncher()
+        {
+        }
+
+        internal void Launch(string javaClassName, params string[] parameters)
+        {
+            var startInfo = new ProcessStartInfo
+            {
+                Arguments = AssembleArguments(javaClassName, parameters),
+                FileName = GetJavaCommand(),
+                UseShellExecute = false,
+                RedirectStandardOutput = true,
+                RedirectStandardError = true
+            };
+            var process = Process.Start(startInfo);
+            if (process != null)
+            {
+                process.OutputDataReceived += delegate(object sender, DataReceivedEventArgs e)
+                {
+                    if (!string.IsNullOrWhiteSpace(e.Data))
+                    {
+                        Logger.Log(Level.Info, e.Data);
+                    }
+                };
+                process.ErrorDataReceived += delegate(object sender, DataReceivedEventArgs e)
+                {
+                    if (!string.IsNullOrWhiteSpace(e.Data))
+                    {
+                        Logger.Log(Level.Error, e.Data);
+                    }
+                };
+                process.BeginErrorReadLine();
+                process.BeginOutputReadLine();
+                process.WaitForExit();
+            }
+            else
+            {
+                Exceptions.Throw(new Exception("Java client process didn't start."), Logger);
+            }
+        }
+
+        /// <summary>
+        /// Assembles the command line arguments. Used by Launch()
+        /// </summary>
+        /// <param name="javaClassName"></param>
+        /// <param name="parameters"></param>
+        /// <returns></returns>
+        private string AssembleArguments(string javaClassName, params string[] parameters)
+        {
+            IList<string> arguments = new List<string>();
+            arguments.Add("-cp");
+            arguments.Add(GetClientClasspath());
+            arguments.Add(javaClassName);
+            foreach (var parameter in parameters)
+            {
+                arguments.Add(parameter);
+            }
+            return string.Join(" ", arguments);
+        }
+
+        /// <summary>
+        /// Find the `java` command on this machine and returns its path.
+        /// </summary>
+        /// <exception cref="JavaNotFoundException">If the java binary couldn't be found.</exception>
+        /// <returns>The path of the `java` command on this machine.</returns>
+        private static string GetJavaCommand()
+        {
+            var javaHomePath = Environment.GetEnvironmentVariable("JAVA_HOME");
+            if (string.IsNullOrWhiteSpace(javaHomePath))
+            {
+                // TODO: Attempt to find java via the registry.
+                Exceptions.Throw(
+                    new JavaNotFoundException("JAVA_HOME isn't set. Please install Java and make set JAVA_HOME"), Logger);
+            }
+
+            if (!Directory.Exists(javaHomePath))
+            {
+                Exceptions.Throw(
+                    new JavaNotFoundException("JAVA_HOME references a folder that doesn't exist.", javaHomePath), Logger);
+            }
+
+            var javaBinPath = Path.Combine(javaHomePath, "bin");
+            if (!Directory.Exists(javaBinPath))
+            {
+                throw new JavaNotFoundException(
+                    "JAVA_HOME references a folder that doesn't contain a `bin` folder. Please adjust JAVA_HOME",
+                    javaHomePath);
+            }
+
+            var javaPath = Path.Combine(javaBinPath, "java.exe");
+            if (!File.Exists(javaPath))
+            {
+                Exceptions.Throw(
+                    new JavaNotFoundException(
+                        "Could not find java.exe on this machine. Is Java installed and JAVA_HOME set?", javaPath),
+                    Logger);
+            }
+            return javaPath;
+        }
+
+        /// <summary>
+        /// Assembles the classpath for the side process
+        /// </summary>
+        /// <exception cref="ClasspathException">If the classpath would be empty.</exception>
+        /// <returns></returns>
+        private string GetClientClasspath()
+        {
+            var files = Directory.GetFiles(JarFolder)
+                .Where(x => (!string.IsNullOrWhiteSpace(x)))
+                .Where(e => Path.GetFileName(e).ToLower().StartsWith(ClientConstants.ClientJarFilePrefix))
+                .ToArray();
+
+            if (null == files || files.Length == 0)
+            {
+                Exceptions.Throw(new ClasspathException(
+                    "Unable to assemble classpath. Make sure the REEF JAR is in the current working directory."), Logger);
+            }
+            var classPath = string.Join(";", files);
+            return classPath;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs b/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
new file mode 100644
index 0000000..f5aa561
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System;
+using System.IO;
+using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.Common;
+using Org.Apache.REEF.Client.Local.Parameters;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Client.Local
+{
+    /// <summary>
+    /// An implementation of the REEF interface using an external Java program
+    /// </summary>
+    public sealed class LocalClient : IREEFClient
+    {
+        /// <summary>
+        /// The class name that contains the Java counterpart for this client.
+        /// </summary>
+        private const string JavaClassName = "org.apache.reef.bridge.client.LocalClient";
+
+        /// <summary>
+        /// The name of the folder in the job's working directory that houses the driver.
+        /// </summary>
+        private const string DriverFolderName = "driver";
+
+        private static readonly Logger Logger = Logger.GetLogger(typeof (LocalClient));
+        private readonly DriverFolderPreparationHelper _driverFolderPreparationHelper;
+        private readonly JavaClientLauncher _javaClientLauncher;
+        private readonly int _numberOfEvaluators;
+        private readonly string _runtimeFolder;
+
+        [Inject]
+        private LocalClient(DriverFolderPreparationHelper driverFolderPreparationHelper,
+            [Parameter(typeof (LocalRuntimeDirectory))] string runtimeFolder,
+            [Parameter(typeof (NumberOfEvaluators))] int numberOfEvaluators, JavaClientLauncher javaClientLauncher)
+        {
+            _driverFolderPreparationHelper = driverFolderPreparationHelper;
+            _runtimeFolder = runtimeFolder;
+            _numberOfEvaluators = numberOfEvaluators;
+            _javaClientLauncher = javaClientLauncher;
+        }
+
+        /// <summary>
+        /// Uses Path.GetTempPath() as the runtime execution folder.
+        /// </summary>
+        /// <param name="driverFolderPreparationHelper"></param>
+        /// <param name="reefJarPath"></param>
+        /// <param name="numberOfEvaluators"></param>
+        [Inject]
+        private LocalClient(
+            DriverFolderPreparationHelper driverFolderPreparationHelper,
+            [Parameter(typeof (NumberOfEvaluators))] int numberOfEvaluators, JavaClientLauncher javaClientLauncher)
+            : this(driverFolderPreparationHelper, Path.GetTempPath(), numberOfEvaluators, javaClientLauncher)
+        {
+            // Intentionally left blank.
+        }
+
+        public void Submit(JobSubmission jobSubmission)
+        {
+            // Prepare the job submission folder
+            var jobFolder = CreateJobFolder(jobSubmission.JobIdentifier);
+            var driverFolder = Path.Combine(jobFolder, DriverFolderName);
+            Logger.Log(Level.Info, "Preparing driver folder in " + driverFolder);
+
+            _driverFolderPreparationHelper.PrepareDriverFolder(jobSubmission, driverFolder);
+
+            _javaClientLauncher.Launch(JavaClassName, driverFolder, jobSubmission.JobIdentifier,
+                _numberOfEvaluators.ToString());
+            Logger.Log(Level.Info, "Submitted the Driver for execution.");
+        }
+
+        /// <summary>
+        /// Creates the temporary directory to hold the job submission.
+        /// </summary>
+        /// <returns></returns>
+        private string CreateJobFolder(string jobId)
+        {
+            var timestamp = DateTime.Now.ToString("yyyyMMddHHmmssfff");
+            return Path.Combine(_runtimeFolder, string.Join("-", "reef", jobId, timestamp));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/Local/LocalRuntimeClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/LocalRuntimeClientConfiguration.cs b/lang/cs/Org.Apache.REEF.Client/Local/LocalRuntimeClientConfiguration.cs
new file mode 100644
index 0000000..20c3c98
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalRuntimeClientConfiguration.cs
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.Local.Parameters;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.Client.Local
+{
+    public sealed class LocalRuntimeClientConfiguration : ConfigurationModuleBuilder
+    {
+        /// <summary>
+        /// The number of threads or evaluators available to the resourcemanager.
+        /// </summary>
+        /// <remarks>
+        /// This is the upper limit on the number of
+        /// Evaluators that the local resourcemanager will hand out concurrently. This simulates the size of a physical cluster
+        /// in terms of the number of slots available on it with one important caveat: The Driver is not counted against this
+        /// number.
+        /// </remarks>
+        public static readonly OptionalParameter<int> NumberOfEvaluators = new OptionalParameter<int>();
+
+        /// <summary>
+        /// The folder in which the sub-folders, one per job, will be created.
+        /// </summary>
+        /// <remarks>
+        /// If none is given, the temp directory is used.
+        /// </remarks>
+        public static readonly OptionalParameter<string> RuntimeFolder = new OptionalParameter<string>();
+
+        public static ConfigurationModule ConfigurationModule = new LocalRuntimeClientConfiguration()
+            .BindImplementation(GenericType<IREEFClient>.Class, GenericType<LocalClient>.Class)
+            .BindNamedParameter(GenericType<LocalRuntimeDirectory>.Class, RuntimeFolder)
+            .BindNamedParameter(GenericType<NumberOfEvaluators>.Class, NumberOfEvaluators)
+            .Build();
+    }
+}
\ No newline at end of file

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

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

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
index eb9f995..f86e0dc 100644
--- a/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
+++ b/lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj
@@ -35,12 +35,28 @@ under the License.
   <ItemGroup>
     <Reference Include="System" />
     <Reference Include="System.Core" />
+    <Reference Include="System.IO.Compression" />
+    <Reference Include="System.IO.Compression.FileSystem" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="API\ClientFactory.cs" />
+    <Compile Include="API\Exceptions\ClasspathException.cs" />
+    <Compile Include="API\Exceptions\JavaNotFoundException.cs" />
+    <Compile Include="API\IREEFClient.cs" />
+    <Compile Include="API\JobSubmission.cs" />
     <Compile Include="CLRBridgeClient.cs" />
+    <Compile Include="Common\ClientConstants.cs" />
+    <Compile Include="Common\DriverFolderPreparationHelper.cs" />
+    <Compile Include="Common\FileSets.cs" />
+    <Compile Include="Common\JavaClientLauncher.cs" />
+    <Compile Include="Local\LocalClient.cs" />
+    <Compile Include="Local\LocalRuntimeClientConfiguration.cs" />
+    <Compile Include="Local\Parameters\LocalRuntimeDirectory.cs" />
+    <Compile Include="Local\Parameters\NumberOfEvaluators.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
   </ItemGroup>
   <ItemGroup>
+    <None Include="packages.config" />
     <None Include="run.cmd" />
   </ItemGroup>
   <ItemGroup>
@@ -78,6 +94,13 @@ under the License.
     </ProjectReference>
   </ItemGroup>
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
+  <Target Name="EnsureNuGetPackageBuildImports" BeforeTargets="PrepareForBuild">
+    <PropertyGroup>
+      <ErrorText>This project references NuGet package(s) that are missing on this computer. Enable NuGet Package Restore to download them.  For more information, see http://go.microsoft.com/fwlink/?LinkID=322105. The missing file is {0}.</ErrorText>
+    </PropertyGroup>
+    <Error Condition="!Exists('$(SolutionDir)\.nuget\NuGet.targets')" Text="$([System.String]::Format('$(ErrorText)', '$(SolutionDir)\.nuget\NuGet.targets'))" />
+  </Target>
   <!-- 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">

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Client/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/packages.config b/lang/cs/Org.Apache.REEF.Client/packages.config
new file mode 100644
index 0000000..933b7e1
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/packages.config
@@ -0,0 +1,24 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+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.
+-->
+<packages>
+  <package id="Microsoft.Hadoop.Avro" version="1.4.0.0" targetFramework="net45" />
+  <package id="Newtonsoft.Json" version="6.0.8" targetFramework="net45" />
+  <package id="protobuf-net" version="2.0.0.668" targetFramework="net45" />
+</packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Common/Constants.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Constants.cs b/lang/cs/Org.Apache.REEF.Common/Constants.cs
index 27e185e..913d810 100644
--- a/lang/cs/Org.Apache.REEF.Common/Constants.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Constants.cs
@@ -17,10 +17,13 @@
  * under the License.
  */
 
+using System;
+
 namespace Org.Apache.REEF.Common
 {
     public class Constants
     {
+        [Obsolete(message:"Use REEFFileNames instead.")]
         public const string ClrBridgeRuntimeConfiguration = "clrBridge.config";
 
         // if 8080 port is not used, then query would fail, 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs b/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
new file mode 100644
index 0000000..4b6d26e
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
@@ -0,0 +1,252 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+using System.Diagnostics.CodeAnalysis;
+using System.IO;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Common.Files
+{
+    /// <summary>
+    /// Contains all file system constants used by REEF.
+    /// </summary>
+    /// <remarks>
+    /// Whenever editing this, make sure you also edit org.apache.reef.runtime.common.files.REEFFileNames in the java
+    /// code base.
+    /// </remarks>
+    [SuppressMessage("ReSharper", "InconsistentNaming",
+        Justification = "The names are all taken from the Java codebase.")]
+    public sealed class REEFFileNames
+    {
+        private const string JAR_FILE_SUFFIX = ".jar";
+        private const string JOB_FOLDER_PREFIX = "reef-job-";
+        private const string EVALUATOR_FOLDER_PREFIX = "reef-evaluator-";
+        private const string DRIVER_STDERR = "driver.stderr";
+        private const string DRIVER_STDOUT = "driver.stdout";
+        private const string EVALUATOR_STDERR = "evaluator.stderr";
+        private const string EVALUATOR_STDOUT = "evaluator.stdout";
+        private const string CPP_BRIDGE = "JavaClrBridge";
+        private const string REEF_DRIVER_APPDLL_DIR = "/ReefDriverAppDlls/";
+        private const string TMP_LOAD_DIR = "/reef/CLRLoadingDirectory";
+        private const string REEF_BASE_FOLDER = "reef";
+        private const string GLOBAL_FOLDER = "global";
+        private const string LOCAL_FOLDER = "local";
+        private const string DRIVER_CONFIGURATION_NAME = "driver.conf";
+        private const string EVALUATOR_CONFIGURATION_NAME = "evaluator.conf";
+        private const string CLR_DRIVER_CONFIGURATION_NAME = "clrdriver.conf";
+        private const string BRIDGE_DLL_NAME = "Org.Apache.REEF.Bridge.dll";
+
+        [Inject]
+        public REEFFileNames()
+        {
+        }
+
+        /// <summary>
+        /// The name of the REEF folder inside of the working directory of an Evaluator or Driver
+        /// </summary>
+        /// <returns></returns>
+        public string GetReefFolderName()
+        {
+            return REEF_BASE_FOLDER;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>the name of the folder inside of REEF_BASE_FOLDER that houses the global files.</returns>
+        public string GetGlobalFolderName()
+        {
+            return GLOBAL_FOLDER;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>the path to the global folder: REEF_BASE_FOLDER/GLOBAL_FOLDER</returns>
+        public string GetGlobalFolderPath()
+        {
+            return GLOBAL_FOLDER_PATH;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>the name of the folder inside of REEF_BASE_FOLDER that houses the local files.</returns>
+        public string GetLocalFolderName()
+        {
+            return LOCAL_FOLDER;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>the path to the local folder: REEF_BASE_FOLDER/LOCAL_FOLDER</returns>
+        public string GetLocalFolderPath()
+        {
+            return LOCAL_FOLDER_PATH;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The name under which the driver configuration will be stored in REEF_BASE_FOLDER/LOCAL_FOLDER</returns>
+        public string GetDriverConfigurationName()
+        {
+            return DRIVER_CONFIGURATION_NAME;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The name under which the driver configuration will be stored in REEF_BASE_FOLDER/LOCAL_FOLDER</returns>
+        public string GetClrDriverConfigurationName()
+        {
+            return CLR_DRIVER_CONFIGURATION_NAME;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The path to the driver configuration: GLOBAL_FOLDER/LOCAL_FOLDER/DRIVER_CONFIGURATION_NAME</returns>
+        public string GetDriverConfigurationPath()
+        {
+            return DRIVER_CONFIGURATION_PATH;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The path to the CLR driver configuration: GLOBAL_FOLDER/LOCAL_FOLDER/CLR_DRIVER_CONFIGURATION_NAME</returns>
+        public string GetClrDriverConfigurationPath()
+        {
+            return CLR_DRIVER_CONFIGURATION_PATH;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>REEF_BASE_FOLDER/LOCAL_FOLDER</returns>
+        public string GetEvaluatorConfigurationName()
+        {
+            return EVALUATOR_CONFIGURATION_NAME;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>the path to the evaluator configuration.</returns>
+        public string GetEvaluatorConfigurationPath()
+        {
+            return EVALUATOR_CONFIGURATION_PATH;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns> The suffix used for JAR files, including the "."</returns>
+        public string GetJarFileSuffix()
+        {
+            return JAR_FILE_SUFFIX;
+        }
+
+        /// <summary>
+        /// The prefix used whenever REEF is asked to create a job folder, on (H)DFS or locally. This prefix is also used with
+        /// JAR files created to represent a job.
+        /// </summary>
+        /// <returns></returns>
+        public string GetJobFolderPrefix()
+        {
+            return JOB_FOLDER_PREFIX;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The name used within the current working directory of the driver to redirect standard error to.</returns>
+        public string GetDriverStderrFileName()
+        {
+            return DRIVER_STDERR;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The name used within the current working directory of the driver to redirect standard out to.</returns>
+        public string GetDriverStdoutFileName()
+        {
+            return DRIVER_STDOUT;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The prefix used whenever REEF is asked to create an Evaluator folder, e.g. for staging.</returns>
+        public string GetEvaluatorFolderPrefix()
+        {
+            return EVALUATOR_FOLDER_PREFIX;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The name used within the current working directory of the driver to redirect standard error to.</returns>
+        public string GetEvaluatorStderrFileName()
+        {
+            return EVALUATOR_STDERR;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>The name used within the current working directory of the driver to redirect standard out to.</returns>
+        public string GetEvaluatorStdoutFileName()
+        {
+            return EVALUATOR_STDOUT;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>the name of cpp bridge file</returns>
+        public string GetCppBridge()
+        {
+            return CPP_BRIDGE;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>reef driver app dll directory</returns>
+        public string GetReefDriverAppDllDir()
+        {
+            return REEF_DRIVER_APPDLL_DIR;
+        }
+
+        /// <summary>
+        /// The name of the Bridge DLL.
+        /// </summary>
+        /// <returns>The name of the Bridge DLL.</returns>
+        public string GetBridgeDLLName()
+        {
+            return BRIDGE_DLL_NAME;
+        }
+
+        /// <summary>
+        /// </summary>
+        /// <returns>temp load directory</returns>
+        public string GetLoadDir()
+        {
+            return TMP_LOAD_DIR;
+        }
+
+        private static readonly string GLOBAL_FOLDER_PATH = Path.Combine(REEF_BASE_FOLDER, GLOBAL_FOLDER);
+        private static readonly string LOCAL_FOLDER_PATH = Path.Combine(REEF_BASE_FOLDER, LOCAL_FOLDER);
+
+        private static readonly string DRIVER_CONFIGURATION_PATH = Path.Combine(LOCAL_FOLDER_PATH,
+            DRIVER_CONFIGURATION_NAME);
+
+        private static readonly string CLR_DRIVER_CONFIGURATION_PATH = Path.Combine(LOCAL_FOLDER_PATH,
+            CLR_DRIVER_CONFIGURATION_NAME);
+
+        private static readonly string EVALUATOR_CONFIGURATION_PATH =
+            Path.Combine(LOCAL_FOLDER_PATH, EVALUATOR_CONFIGURATION_NAME);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj b/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj
index 20f2b8f..556b182 100644
--- a/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj
+++ b/lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj
@@ -109,6 +109,7 @@ under the License.
     <Compile Include="Protobuf\ReefProtocol\ReefProtocol.pb.cs" />
     <Compile Include="Protobuf\ReefProtocol\ReefService.pb.cs" />
     <Compile Include="Protobuf\ReefProtocol\Serializer.cs" />
+    <Compile Include="Files\REEFFileNames.cs" />
     <Compile Include="Runtime\Evaluator\Constants.cs" />
     <Compile Include="Runtime\Evaluator\Context\ContextClientCodeException.cs" />
     <Compile Include="Runtime\Evaluator\Context\ContextConfiguration.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Driver/Bridge/BridgeConfigurationProvider.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/BridgeConfigurationProvider.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/BridgeConfigurationProvider.cs
new file mode 100644
index 0000000..4f15b7c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/BridgeConfigurationProvider.cs
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+using System.IO;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities.Diagnostics;
+using Org.Apache.REEF.Utilities.Logging;
+
+namespace Org.Apache.REEF.Driver.Bridge
+{
+    /// <summary>
+    /// Helper class that provides the Bridge Configuration.
+    /// </summary>
+    /// <remarks>
+    /// The Bridge configuration file moved in the evolution of REEF (see [REEF-228]). This class hides the actual location
+    /// from client code and does the appropriate logging of its location.
+    /// </remarks>
+    internal sealed class BridgeConfigurationProvider
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(BridgeConfigurationProvider));
+        private readonly REEFFileNames _fileNames;
+
+        [Inject]
+        internal BridgeConfigurationProvider(REEFFileNames fileNames)
+        {
+            _fileNames = fileNames;
+        }
+
+        /// <summary>
+        /// Finds the path to the bridge configuration
+        /// </summary>
+        /// <remarks>
+        /// It tries both the new and the legacy locations and gives preference to the new locations. Warnings will be logged
+        /// when both are present as well as when the configuration is only present in the legacy location.
+        /// </remarks>
+        /// <exception cref="FileNotFoundException">When neither the legacy nor the new file exists.</exception>
+        /// <returns>The path to the bridge configuration</returns>
+        internal string GetBridgeConfigurationPath()
+        {
+            var newBridgeConfigurationPath = Path.GetFullPath(Path.Combine(Directory.GetCurrentDirectory(),
+                _fileNames.GetClrDriverConfigurationPath()));
+            var legacyBridgeConfigurationPath =
+                Path.GetFullPath(Path.Combine(Directory.GetCurrentDirectory(), "reef", "global",
+                    "clrBridge.config"));
+
+            var newExists = File.Exists(newBridgeConfigurationPath);
+            var oldExists = File.Exists(legacyBridgeConfigurationPath);
+
+            if (newExists && oldExists)
+            {
+                Logger.Log(Level.Warning, "Found configurations in both the legacy location (" +
+                                          legacyBridgeConfigurationPath + ") and the new location (" +
+                                          newBridgeConfigurationPath +
+                                          "). Loading only the one found in the new location."
+                    );
+            }
+            if (newExists)
+            {
+                return newBridgeConfigurationPath;
+            }
+            if (oldExists)
+            {
+                Logger.Log(Level.Warning, "Only found configuration in the legacy location (" +
+                                          legacyBridgeConfigurationPath + ") and not the new location (" +
+                                          newBridgeConfigurationPath +
+                                          "). Loading only the one found in the legacy location.");
+                return legacyBridgeConfigurationPath;
+            }
+            // If we reached this, we weren't able to find the configuration file.
+            var message = "Unable to find brigde configuration. Paths checked: ['" +
+                          newBridgeConfigurationPath + "', '" +
+                          legacyBridgeConfigurationPath + "']";
+
+            Logger.Log(Level.Error, message);
+            var exception = new FileNotFoundException(message);
+            Exceptions.Throw(exception, Logger);
+            throw exception;
+        }
+
+        /// <summary>
+        /// Loads the bridge configuration from disk.
+        /// </summary>
+        /// <remarks>
+        /// It tries both the new and the legacy locations and gives preference to the new locations. Warnings will be logged
+        /// when both are present as well as when the configuration is read from the legacy location.
+        /// </remarks>
+        /// <exception cref="FileNotFoundException">When neither the legacy nor the new file exists.</exception>
+        /// <returns>The bridge Configuration loaded from disk</returns>
+        internal IConfiguration LoadBridgeConfiguration()
+        {
+            var bridgeConfigurationPath = GetBridgeConfigurationPath();
+            Logger.Log(Level.Info, "Loading configuration '" + bridgeConfigurationPath + "'.");
+            return new AvroConfigurationSerializer().FromFile(bridgeConfigurationPath);
+        }
+
+        /// <summary>
+        /// Same as LoadBridgeConfiguration for use in cases where Tang isn't available.
+        /// </summary>
+        /// <returns></returns>
+        internal static IConfiguration GetBridgeConfiguration()
+        {
+            return new BridgeConfigurationProvider(new REEFFileNames()).LoadBridgeConfiguration();
+        }
+
+        /// <summary>
+        /// Instantiates an IInjector using the bridge configuration.
+        /// </summary>
+        /// <returns></returns>
+        internal static IInjector GetBridgeInjector()
+        {
+            return TangFactory.GetTang().NewInjector(GetBridgeConfiguration());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrHandlerHelper.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrHandlerHelper.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrHandlerHelper.cs
index 575519c..bce5ce4 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrHandlerHelper.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrHandlerHelper.cs
@@ -24,7 +24,6 @@ using System.IO;
 using System.Linq;
 using System.Runtime.InteropServices;
 using Org.Apache.REEF.Tang.Exceptions;
-using Org.Apache.REEF.Tang.Formats;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Tang.Protobuf;
@@ -80,30 +79,16 @@ namespace Org.Apache.REEF.Driver.Bridge
         {
             using (LOGGER.LogFunction("ClrHandlerHelper::GetCommandLineArguments"))
             {
-                string bridgeConfiguration = Path.Combine(Directory.GetCurrentDirectory(), "reef", "global",
-                                                          Constants.DriverBridgeConfiguration);
-
-                if (!File.Exists(bridgeConfiguration))
-                {
-                    string error = "Configuraiton file not found: " + bridgeConfiguration;
-                    LOGGER.Log(Level.Error, error);
-                    Exceptions.Throw(new InvalidOperationException(error), LOGGER);
-                }
                 CommandLineArguments arguments;
-                IInjector injector;
                 try
-                {
-                    IConfiguration driverBridgeConfiguration =
-                        new AvroConfigurationSerializer().FromFile(bridgeConfiguration);
-                    injector = TangFactory.GetTang().NewInjector(driverBridgeConfiguration);
-                    arguments = injector.GetInstance<CommandLineArguments>();
+                {                       
+                    arguments = BridgeConfigurationProvider.GetBridgeInjector().GetInstance<CommandLineArguments>();
                 }
                 catch (InjectionException e)
                 {
                     string error = "Cannot inject command line arguments from driver bridge configuration. ";
-                    Exceptions.Caught(e, Level.Error, error, LOGGER);
-                    // return empty string set
-                    return new HashSet<string>();
+                    Exceptions.CaughtAndThrow(e, Level.Error, error, LOGGER);
+                    throw e;
                 }
                 return arguments.Arguments;
             }
@@ -116,7 +101,7 @@ namespace Org.Apache.REEF.Driver.Bridge
             File.WriteAllText(path, string.Join(",", classPaths));
         }
 
-        public static void GenerateClassHierarchy(HashSet<string> clrDlls)
+        public static void GenerateClassHierarchy(ISet<string> clrDlls)
         {
             using (LOGGER.LogFunction("ClrHandlerHelper::GenerateClassHierarchy"))
             {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrSystemHandlerWrapper.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrSystemHandlerWrapper.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrSystemHandlerWrapper.cs
index 80d22f5..3a65103 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrSystemHandlerWrapper.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrSystemHandlerWrapper.cs
@@ -18,10 +18,13 @@
  */
 
 using System;
+using System.CodeDom;
 using System.Globalization;
 using System.IO;
 using System.Runtime.InteropServices;
+using Org.Apache.REEF.Common;
 using Org.Apache.REEF.Common.Context;
+using Org.Apache.REEF.Common.Files;
 using Org.Apache.REEF.Driver.Bridge.Clr2java;
 using Org.Apache.REEF.Driver.Bridge.Events;
 using Org.Apache.REEF.Driver.Context;
@@ -167,10 +170,22 @@ namespace Org.Apache.REEF.Driver.Bridge
         {
             using (LOGGER.LogFunction("ClrSystemHandlerWrapper::Call_ClrSystemHttpServer_OnNext"))
             {
-                GCHandle gc = GCHandle.FromIntPtr((IntPtr)handle);
-                ClrSystemHandler<IHttpMessage> obj = (ClrSystemHandler<IHttpMessage>)gc.Target;
-                obj.OnNext(new HttpMessage(clr2Java));
-            }      
+                try
+                {
+                    GCHandle gc = GCHandle.FromIntPtr((IntPtr) handle);
+                    if (!gc.IsAllocated)
+                    {
+                        LOGGER.Log(Level.Warning, "gc is not allocated.");
+                    } 
+                    ClrSystemHandler<IHttpMessage> obj = (ClrSystemHandler<IHttpMessage>) gc.Target;
+                    obj.OnNext(new HttpMessage(clr2Java));
+                }
+                catch (Exception ex)
+                {
+                  
+                    LOGGER.Log(Level.Info, "Caught exception: " + ex.Message + ex.StackTrace );
+                    Exceptions.CaughtAndThrow(ex, Level.Warning,  LOGGER);
+                }}
         }
 
         public static void Call_ClrSystemClosedContext_OnNext(ulong handle, IClosedContextClr2Java clr2Java)
@@ -235,32 +250,16 @@ namespace Org.Apache.REEF.Driver.Bridge
 
         private static ulong[] GetHandlers(string httpServerPortNumber)
         {
-            IStartHandler startHandler;
-            IInjector injector = null;
-            string errorMessage;
-            string bridgeConfiguration = Path.Combine(Directory.GetCurrentDirectory(), "reef", "global", Constants.DriverBridgeConfiguration);
-            if (!File.Exists(bridgeConfiguration))
-            {
-                errorMessage = "Cannot find CLR Driver bridge configuration file " + bridgeConfiguration;
-                Exceptions.Throw(new InvalidOperationException(errorMessage), LOGGER);
-            }
-            try
-            {
-                IConfiguration driverBridgeConfiguration = new AvroConfigurationSerializer().FromFile(bridgeConfiguration);
-                injector = TangFactory.GetTang().NewInjector(driverBridgeConfiguration);
-            }
-            catch (Exception e)
-            {
-                errorMessage = "Failed to get injector from driver bridge configuration.";
-                Exceptions.CaughtAndThrow(new InvalidOperationException(errorMessage, e), Level.Error, errorMessage, LOGGER);
-            }
+            var injector = BridgeConfigurationProvider.GetBridgeInjector();
 
             try
             {
-                HttpServerPort port = injector.GetInstance<HttpServerPort>();
-                port.PortNumber = httpServerPortNumber == null ? 0 : int.Parse(httpServerPortNumber, CultureInfo.InvariantCulture);
+                var port = injector.GetInstance<HttpServerPort>();
+                port.PortNumber = httpServerPortNumber == null
+                    ? 0
+                    : int.Parse(httpServerPortNumber, CultureInfo.InvariantCulture);
 
-                startHandler = injector.GetInstance<IStartHandler>();
+                var startHandler = injector.GetInstance<IStartHandler>();
                 LOGGER.Log(Level.Info, "Start handler set to be " + startHandler.Identifier);
                 _driverBridge = injector.GetInstance<DriverBridge>();
             }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Driver/Constants.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Constants.cs b/lang/cs/Org.Apache.REEF.Driver/Constants.cs
index 0ec5659..011d4e0 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Constants.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Constants.cs
@@ -17,6 +17,7 @@
  * under the License.
  */
 
+using System;
 using System.Collections.Generic;
 
 namespace Org.Apache.REEF.Driver
@@ -67,6 +68,7 @@ namespace Org.Apache.REEF.Driver
 
         public const string DriverRestartRunningTaskHandler = "DriverRestartRunningTask";
 
+        [Obsolete(message:"Use REEFFileNames instead.")]
         public const string DriverBridgeConfiguration = Common.Constants.ClrBridgeRuntimeConfiguration;
 
         public const string DriverAppDirectory = "ReefDriverAppDlls";

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj b/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj
index 359b956..9e26254 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj
+++ b/lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj
@@ -38,6 +38,7 @@ under the License.
     <Reference Include="System.Runtime.Serialization" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Bridge\BridgeConfigurationProvider.cs" />
     <Compile Include="Bridge\BridgeLogger.cs" />
     <Compile Include="Bridge\Clr2java\IActiveContextClr2Java.cs" />
     <Compile Include="Bridge\Clr2java\IAllocatedEvaluaotrClr2Java.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/561e7a0a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/App.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/App.config b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/App.config
new file mode 100644
index 0000000..e429816
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/App.config
@@ -0,0 +1,24 @@
+<?xml version="1.0" encoding="utf-8" ?>
+<!--
+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.
+-->
+<configuration>
+    <startup> 
+        <supportedRuntime version="v4.0" sku=".NETFramework,Version=v4.5" />
+    </startup>
+</configuration>
\ No newline at end of file