You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by we...@apache.org on 2015/05/01 03:16:20 UTC

incubator-reef git commit: [REEF-279]:Add ConfigurationProvider support for CLR

Repository: incubator-reef
Updated Branches:
  refs/heads/master 637f9fcaa -> 867449090


[REEF-279]:Add ConfigurationProvider support for CLR

This addressed the issue adding necessary plumbing to flow the
configurations from client to evaluators

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

Pull Request:
  This closes #168


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

Branch: refs/heads/master
Commit: 8674490905999c478f9f403b898e5a302a68bf4c
Parents: 637f9fc
Author: Beysim Sezgin <be...@microsoft.com>
Authored: Thu Apr 30 14:08:48 2015 -0700
Committer: Markus Weimer <we...@apache.org>
Committed: Thu Apr 30 18:14:33 2015 -0700

----------------------------------------------------------------------
 .../API/IJobSubmission.cs                       |  65 +++++++
 .../API/IJobSubmissionBuilder.cs                |  80 +++++++++
 .../Org.Apache.REEF.Client/API/IREEFClient.cs   |   2 +-
 .../Org.Apache.REEF.Client/API/JobSubmission.cs | 156 ++++-------------
 .../API/JobSubmissionBuilder.cs                 | 169 +++++++++++++++++++
 .../API/JobSubmissionBuilderFactory.cs          |  49 ++++++
 .../Common/DriverFolderPreparationHelper.cs     |   4 +-
 .../Org.Apache.REEF.Client/Common/FileSets.cs   |   2 +-
 .../Org.Apache.REEF.Client/Local/LocalClient.cs |   2 +-
 .../Local/LocalRuntimeClientConfiguration.cs    |   8 +
 .../Parameters/DriverConfigurationProviders.cs  |  33 ++++
 .../Org.Apache.REEF.Client.csproj               |   5 +
 .../Org.Apache.REEF.Client/YARN/YARNClient.cs   |   2 +-
 .../YARN/YARNClientConfiguration.cs             |  10 ++
 .../EvaluatorConfigurationProviders.cs          |  34 ++++
 .../Org.Apache.REEF.Common.csproj               |   1 +
 .../Bridge/ClrSystemHandlerWrapper.cs           |   2 +-
 .../Bridge/DriverBridge.cs                      |   8 +
 .../Bridge/Events/AllocatedEvaluator.cs         |  22 ++-
 .../HelloREEF.cs                                |  18 +-
 .../Interface/IConfigurationProvider.cs         |  32 ++++
 .../Org.Apache.REEF.Tang.csproj                 |   1 +
 22 files changed, 570 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Client/API/IJobSubmission.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/IJobSubmission.cs b/lang/cs/Org.Apache.REEF.Client/API/IJobSubmission.cs
new file mode 100644
index 0000000..08fd91b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/IJobSubmission.cs
@@ -0,0 +1,65 @@
+/**
+ * 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 Org.Apache.REEF.Tang.Interface;
+
+namespace Org.Apache.REEF.Client.API
+{
+    /// <summary>
+    /// This interfaces provides all the information that is needed for 
+    /// a job submission
+    /// </summary>
+    public interface IJobSubmission
+    {
+        /// <summary>
+        /// The assemblies to be made available to all containers.
+        /// </summary>
+        ISet<string> GlobalAssemblies { get; }
+
+        /// <summary>
+        /// The driver configurations
+        /// </summary>
+        ISet<IConfiguration> DriverConfigurations { get; }
+
+        /// <summary>
+        /// Global files. 
+        /// </summary>
+        ISet<string> GlobalFiles { get; }
+
+        /// <summary>
+        /// Local assemblies.
+        /// </summary>
+        ISet<string> LocalAssemblies { get; }
+
+        /// <summary>
+        /// Local files. 
+        /// </summary>
+        ISet<string> LocalFiles { get; }
+
+        /// <summary>
+        /// Driver memory in MegaBytes. 
+        /// </summary>
+        int DriverMemory { get; }
+
+        /// <summary>
+        /// The Job's identifier
+        /// </summary>
+        string JobIdentifier { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Client/API/IJobSubmissionBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/IJobSubmissionBuilder.cs b/lang/cs/Org.Apache.REEF.Client/API/IJobSubmissionBuilder.cs
new file mode 100644
index 0000000..af8b51f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/IJobSubmissionBuilder.cs
@@ -0,0 +1,80 @@
+/**
+ * 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 Org.Apache.REEF.Tang.Interface;
+
+namespace Org.Apache.REEF.Client.API
+{
+    /// <summary>
+    /// Facilitates building of job submissions
+    /// </summary>
+    public interface IJobSubmissionBuilder
+    {
+        /// <summary>
+        /// Bake the information provided so far and return a IJobSubmission 
+        /// </summary>
+        IJobSubmission Build();
+
+        /// <summary>
+        /// Make this file available to all containers
+        /// </summary>
+        IJobSubmissionBuilder AddGlobalFile(string fileName);
+
+        /// <summary>
+        /// Files specific to one container
+        /// </summary>
+        IJobSubmissionBuilder AddLocalFile(string fileName);
+
+        /// <summary>
+        /// Assemblies available to all containers
+        /// </summary>
+        IJobSubmissionBuilder AddGlobalAssembly(string fileName);
+
+        /// <summary>
+        /// Assemblies available to a specific container
+        /// </summary>
+        IJobSubmissionBuilder AddLocalAssembly(string fileName);
+
+        /// <summary>
+        /// Configuration that will be available to the driver
+        /// </summary>
+        IJobSubmissionBuilder AddDriverConfiguration(IConfiguration configuration);
+
+        /// <summary>
+        /// Find the assembly for this type and make it available to a specific container 
+        /// </summary>
+        IJobSubmissionBuilder AddLocalAssemblyForType(Type type);
+
+        /// <summary>
+        /// Find the assembly for this type and make it available to all containers
+        /// </summary>
+        IJobSubmissionBuilder AddGlobalAssemblyForType(Type type);
+
+        /// <summary>
+        /// Specify job identifier.
+        /// </summary>
+        IJobSubmissionBuilder SetJobIdentifier(string id);
+
+        /// <summary>
+        /// Set driver memory in megabytes
+        /// </summary>
+        IJobSubmissionBuilder SetDriverMemory(int driverMemoryInMb);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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
index 56b0042..e4f4c88 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/IREEFClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/IREEFClient.cs
@@ -29,6 +29,6 @@ namespace Org.Apache.REEF.Client.API
         /// Submit the job described in jobSubmission to the cluster.
         /// </summary>
         /// <param name="jobSubmission"></param>
-        void Submit(JobSubmission jobSubmission);
+        void Submit(IJobSubmission jobSubmission);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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
index 440755e..da77ece 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/JobSubmission.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobSubmission.cs
@@ -27,19 +27,38 @@ namespace Org.Apache.REEF.Client.API
     /// <summary>
     /// Captures a submission of a REEF Job to a cluster.
     /// </summary>
-    public sealed class JobSubmission
+    internal sealed class JobSubmission : IJobSubmission
     {
-        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;
+        private readonly ISet<IConfiguration> _driverConfigurations;
+        private readonly ISet<string> _globalAssemblies;
+        private readonly ISet<string> _globalFiles;
+        private readonly ISet<string> _localAssemblies;
+        private readonly ISet<string> _localFiles;
+        private int _driverMemory ;
+        private string _jobIdentifier;
+
+        internal JobSubmission(
+            ISet<IConfiguration> driverConfigurations,
+            ISet<string> globalAssemblies,
+            ISet<string> globalFiles,
+            ISet<string> localAssemblies,
+            ISet<string> localFiles,
+            int driverMemory,
+            string jobIdentifier)
+        {
+            _driverConfigurations = driverConfigurations;
+            _globalAssemblies = globalAssemblies;
+            _globalFiles = globalFiles;
+            _localAssemblies = localAssemblies;
+            _localFiles = localFiles;
+            _driverMemory = driverMemory;
+            _jobIdentifier = jobIdentifier;
+        }
 
         /// <summary>
         /// The assemblies to be made available to all containers.
         /// </summary>
-        internal ISet<string> GlobalAssemblies
+        ISet<string> IJobSubmission.GlobalAssemblies
         {
             get { return _globalAssemblies; }
         }
@@ -47,27 +66,27 @@ namespace Org.Apache.REEF.Client.API
         /// <summary>
         /// The driver configurations
         /// </summary>
-        internal ISet<IConfiguration> DriverConfigurations
+        ISet<IConfiguration> IJobSubmission.DriverConfigurations
         {
             get { return _driverConfigurations; }
         }
 
-        internal ISet<string> GlobalFiles
+        ISet<string> IJobSubmission.GlobalFiles
         {
             get { return _globalFiles; }
         }
 
-        internal ISet<string> LocalAssemblies
+        ISet<string> IJobSubmission.LocalAssemblies
         {
             get { return _localAssemblies; }
         }
 
-        internal ISet<string> LocalFiles
+        ISet<string> IJobSubmission.LocalFiles
         {
             get { return _localFiles; }
         }
 
-        internal int DriverMemory
+        int IJobSubmission.DriverMemory
         {
             get { return _driverMemory; }
         }
@@ -75,116 +94,9 @@ namespace Org.Apache.REEF.Client.API
         /// <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;
+        string IJobSubmission.JobIdentifier {
+            get { return _jobIdentifier; }
         }
 
-        /// <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/86744909/lang/cs/Org.Apache.REEF.Client/API/JobSubmissionBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/JobSubmissionBuilder.cs b/lang/cs/Org.Apache.REEF.Client/API/JobSubmissionBuilder.cs
new file mode 100644
index 0000000..f56bffd
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobSubmissionBuilder.cs
@@ -0,0 +1,169 @@
+/**
+ * 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
+{
+    internal class JobSubmissionBuilder : IJobSubmissionBuilder
+    {
+        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;
+        private string _jobIdentifier;
+        private readonly ISet<IConfigurationProvider> _configurationProviders;
+
+        internal JobSubmissionBuilder(ISet<IConfigurationProvider> configurationProviders)
+        {
+            _configurationProviders = configurationProviders;
+        }
+
+
+        /// <summary>
+        /// Add a file to be made available in all containers.
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        public IJobSubmissionBuilder 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 IJobSubmissionBuilder 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 IJobSubmissionBuilder AddGlobalAssembly(string fileName)
+        {
+            _globalAssemblies.Add(fileName);
+            return this;
+        }
+
+        /// <summary>
+        /// Add an assembly to the driver only.
+        /// </summary>
+        /// <param name="fileName"></param>
+        /// <returns></returns>
+        public IJobSubmissionBuilder AddLocalAssembly(string fileName)
+        {
+            _localAssemblies.Add(fileName);
+            return this;
+        }
+
+        /// <summary>
+        /// Add a Configuration to the Driver.
+        /// </summary>
+        /// <param name="configuration"></param>
+        /// <returns></returns>
+        public IJobSubmissionBuilder 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 IJobSubmissionBuilder 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 IJobSubmissionBuilder AddGlobalAssemblyForType(Type type)
+        {
+            AddGlobalAssembly(GetAssemblyPathForType(type));
+            return this;
+        }
+
+        /// <summary>
+        /// Gives the job an identifier.
+        /// </summary>
+        /// <param name="id"></param>
+        /// <returns></returns>
+        public IJobSubmissionBuilder 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 IJobSubmissionBuilder 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);
+        }
+
+        /// <summary>
+        /// Builds the submission
+        /// </summary>
+        /// <returns>IJobSubmission</returns>
+        public IJobSubmission Build()
+        {
+            foreach (var cfg in _configurationProviders)
+            {
+                _driverConfigurations.Add(cfg.GetConfiguration());
+            }
+
+            return new JobSubmission(_driverConfigurations, _globalAssemblies, _globalFiles, _localAssemblies,
+                _localFiles, _driverMemory, _jobIdentifier);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Client/API/JobSubmissionBuilderFactory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/JobSubmissionBuilderFactory.cs b/lang/cs/Org.Apache.REEF.Client/API/JobSubmissionBuilderFactory.cs
new file mode 100644
index 0000000..b6b414a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobSubmissionBuilderFactory.cs
@@ -0,0 +1,49 @@
+/**
+ * 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 Org.Apache.REEF.Client.Local.Parameters;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
+
+namespace Org.Apache.REEF.Client.API
+{
+    /// <summary>
+    /// Instantiates IJobSubmissionBuilder based on configurationProviders.
+    /// </summary>
+    public sealed class JobSubmissionBuilderFactory
+    {
+        private readonly ISet<IConfigurationProvider> _configurationProviders;
+
+        [Inject]
+        internal JobSubmissionBuilderFactory(
+            [Parameter(typeof(DriverConfigurationProviders))] ISet<IConfigurationProvider> configurationProviders)
+        {
+            _configurationProviders = configurationProviders;
+        }
+
+        /// <summary>
+        /// Instantiates IJobSubmissionBuilder
+        /// </summary>
+        public IJobSubmissionBuilder GetJobSubmissionBuilder()
+        {
+            return new JobSubmissionBuilder(_configurationProviders);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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
index 7e155b8..2bdb9f8 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
@@ -59,7 +59,7 @@ namespace Org.Apache.REEF.Client.Common
         /// </summary>
         /// <param name="jobSubmission"></param>
         /// <param name="driverFolderPath"></param>
-        internal void PrepareDriverFolder(JobSubmission jobSubmission, string driverFolderPath)
+        internal void PrepareDriverFolder(IJobSubmission jobSubmission, string driverFolderPath)
         {
             Logger.Log(Level.Info, "Preparing Driver filesystem layout in " + driverFolderPath);
 
@@ -88,7 +88,7 @@ namespace Org.Apache.REEF.Client.Common
         /// </summary>
         /// <param name="jobSubmission"></param>
         /// <param name="driverFolderPath"></param>
-        internal void CreateDriverConfiguration(JobSubmission jobSubmission, string driverFolderPath)
+        internal void CreateDriverConfiguration(IJobSubmission jobSubmission, string driverFolderPath)
         {
             var driverConfiguration = Configurations.Merge(jobSubmission.DriverConfigurations.ToArray());
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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
index e337112..cb154b3 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/FileSets.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/FileSets.cs
@@ -121,7 +121,7 @@ namespace Org.Apache.REEF.Client.Common
         /// Adds all the files referenced in the given JobSubmission
         /// </summary>
         /// <param name="submission"></param>
-        internal void AddJobFiles(JobSubmission submission)
+        internal void AddJobFiles(IJobSubmission submission)
         {
             AddToGlobalFiles(submission.GlobalFiles);
             AddToGlobalFiles(submission.GlobalAssemblies);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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
index f5aa561..3c4316c 100644
--- a/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
@@ -74,7 +74,7 @@ namespace Org.Apache.REEF.Client.Local
             // Intentionally left blank.
         }
 
-        public void Submit(JobSubmission jobSubmission)
+        public void Submit(IJobSubmission jobSubmission)
         {
             // Prepare the job submission folder
             var jobFolder = CreateJobFolder(jobSubmission.JobIdentifier);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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
index 20c3c98..e24d2ae 100644
--- a/lang/cs/Org.Apache.REEF.Client/Local/LocalRuntimeClientConfiguration.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalRuntimeClientConfiguration.cs
@@ -20,6 +20,7 @@
 using Org.Apache.REEF.Client.API;
 using Org.Apache.REEF.Client.Local.Parameters;
 using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Tang.Util;
 
 namespace Org.Apache.REEF.Client.Local
@@ -45,10 +46,17 @@ namespace Org.Apache.REEF.Client.Local
         /// </remarks>
         public static readonly OptionalParameter<string> RuntimeFolder = new OptionalParameter<string>();
 
+        /// <summary>
+        /// Configuration provides whose Configuration will be merged into all Driver Configuration.
+        /// </summary>
+        public static readonly OptionalImpl<IConfigurationProvider> DriverConfigurationProvider =
+            new OptionalImpl<IConfigurationProvider>();
+
         public static ConfigurationModule ConfigurationModule = new LocalRuntimeClientConfiguration()
             .BindImplementation(GenericType<IREEFClient>.Class, GenericType<LocalClient>.Class)
             .BindNamedParameter(GenericType<LocalRuntimeDirectory>.Class, RuntimeFolder)
             .BindNamedParameter(GenericType<NumberOfEvaluators>.Class, NumberOfEvaluators)
+            .BindSetEntry(GenericType<DriverConfigurationProviders>.Class, DriverConfigurationProvider)
             .Build();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Client/Local/Parameters/DriverConfigurationProviders.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/Parameters/DriverConfigurationProviders.cs b/lang/cs/Org.Apache.REEF.Client/Local/Parameters/DriverConfigurationProviders.cs
new file mode 100644
index 0000000..1002337
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Local/Parameters/DriverConfigurationProviders.cs
@@ -0,0 +1,33 @@
+/**
+ * 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 Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
+
+namespace Org.Apache.REEF.Client.Local.Parameters
+{
+    /// <summary>
+    // This name parameter is used to target receviers Configuration providers at driver level
+    /// </summary>
+    [NamedParameter]
+    public sealed class DriverConfigurationProviders : Name<ISet<IConfigurationProvider>>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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 b0be389..fde24ed 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
@@ -42,8 +42,12 @@ under the License.
     <Compile Include="API\ClientFactory.cs" />
     <Compile Include="API\Exceptions\ClasspathException.cs" />
     <Compile Include="API\Exceptions\JavaNotFoundException.cs" />
+    <Compile Include="API\IJobSubmission.cs" />
+    <Compile Include="API\IJobSubmissionBuilder.cs" />
     <Compile Include="API\IREEFClient.cs" />
     <Compile Include="API\JobSubmission.cs" />
+    <Compile Include="API\JobSubmissionBuilder.cs" />
+    <Compile Include="API\JobSubmissionBuilderFactory.cs" />
     <Compile Include="CLRBridgeClient.cs" />
     <Compile Include="Common\ClientConstants.cs" />
     <Compile Include="Common\DriverFolderPreparationHelper.cs" />
@@ -51,6 +55,7 @@ under the License.
     <Compile Include="Common\JavaClientLauncher.cs" />
     <Compile Include="Local\LocalClient.cs" />
     <Compile Include="Local\LocalRuntimeClientConfiguration.cs" />
+    <Compile Include="Local\Parameters\DriverConfigurationProviders.cs" />
     <Compile Include="Local\Parameters\LocalRuntimeDirectory.cs" />
     <Compile Include="Local\Parameters\NumberOfEvaluators.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Client/YARN/YARNClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YARNClient.cs
index bcace19..93a53a0 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNClient.cs
@@ -47,7 +47,7 @@ namespace Org.Apache.REEF.Client.YARN
             _driverFolderPreparationHelper = driverFolderPreparationHelper;
         }
 
-        public void Submit(JobSubmission jobSubmission)
+        public void Submit(IJobSubmission jobSubmission)
         {
             // Prepare the job submission folder
             var driverFolderPath = CreateDriverFolder(jobSubmission.JobIdentifier);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
index ad53d0c..94105a0 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNClientConfiguration.cs
@@ -17,8 +17,12 @@
  * under the License.
  */
 using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.Local.Parameters;
+using Org.Apache.REEF.Common.Io;
 using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Wake.Remote;
 
 namespace Org.Apache.REEF.Client.YARN
 {
@@ -27,8 +31,14 @@ namespace Org.Apache.REEF.Client.YARN
     /// </summary>
     public sealed class YARNClientConfiguration : ConfigurationModuleBuilder
     {
+        /// <summary>
+        /// Configuration provides whose Configuration will be merged into all Driver Configuration.
+        /// </summary>
+        public static readonly OptionalImpl<IConfigurationProvider> DriverConfigurationProvider = new OptionalImpl<IConfigurationProvider>();
+
         public static ConfigurationModule ConfigurationModule = new YARNClientConfiguration()
             .BindImplementation(GenericType<IREEFClient>.Class, GenericType<YARNClient>.Class)
+            .BindSetEntry(GenericType<DriverConfigurationProviders>.Class, DriverConfigurationProvider)
             .Build();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Common/Evaluator/Parameters/EvaluatorConfigurationProviders.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Evaluator/Parameters/EvaluatorConfigurationProviders.cs b/lang/cs/Org.Apache.REEF.Common/Evaluator/Parameters/EvaluatorConfigurationProviders.cs
new file mode 100644
index 0000000..34bbe86
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Common/Evaluator/Parameters/EvaluatorConfigurationProviders.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.
+ */
+
+using System.Collections.Generic;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
+
+namespace Org.Apache.REEF.Common.Evaluator.Parameters
+{
+    /// <summary>
+    // This name parameter is used to target receviers Configuration providers at evaluator level
+    /// </summary>
+    [NamedParameter(
+        Documentation = "Configuration providers whose Configurations will be mixed into the Evaluator Configuration.")]
+    public sealed class EvaluatorConfigurationProviders : Name<ISet<IConfigurationProvider>>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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 556b182..77da0f9 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
@@ -87,6 +87,7 @@ under the License.
     <Compile Include="Evaluator\EvaluatorRuntimeState.cs" />
     <Compile Include="Evaluator\EvaluatorType.cs" />
     <Compile Include="Evaluator\IDriverConnection.cs" />
+    <Compile Include="Evaluator\Parameters\EvaluatorConfigurationProviders.cs" />
     <Compile Include="Events\IContextStart.cs" />
     <Compile Include="Events\IContextStop.cs" />
     <Compile Include="Exceptions\EvaluatorException.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/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 3a65103..6c92ec7 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrSystemHandlerWrapper.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/ClrSystemHandlerWrapper.cs
@@ -52,7 +52,7 @@ namespace Org.Apache.REEF.Driver.Bridge
             {
                 GCHandle gc = GCHandle.FromIntPtr((IntPtr)handle);
                 ClrSystemHandler<IAllocatedEvaluator> obj = (ClrSystemHandler<IAllocatedEvaluator>)gc.Target;
-                obj.OnNext(new AllocatedEvaluator(clr2Java)); 
+                obj.OnNext(new AllocatedEvaluator(clr2Java, _driverBridge.ConfigurationProviders));
             }       
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs
index 281af0d..e989199 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/DriverBridge.cs
@@ -27,8 +27,10 @@ using Org.Apache.REEF.Driver.Context;
 using Org.Apache.REEF.Driver.Evaluator;
 using Org.Apache.REEF.Driver.Task;
 using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Utilities.Logging;
 using Org.Apache.REEF.Wake.Time.Event;
+using Org.Apache.REEF.Common.Evaluator.Parameters;
 
 namespace Org.Apache.REEF.Driver.Bridge
 {
@@ -104,6 +106,8 @@ namespace Org.Apache.REEF.Driver.Bridge
 
         private readonly HttpServerHandler _httpServerHandler;
 
+        private readonly ISet<IConfigurationProvider> _configurationProviders;
+
         [Inject]
         public DriverBridge(
             [Parameter(Value = typeof(DriverBridgeConfigurationOptions.DriverRestartHandler))] IObserver<StartTime> driverRestartHandler,
@@ -123,6 +127,7 @@ namespace Org.Apache.REEF.Driver.Bridge
             [Parameter(Value = typeof(DriverBridgeConfigurationOptions.DriverRestartActiveContextHandlers))] ISet<IObserver<IActiveContext>> driverRestartActiveContextHandlers,
             [Parameter(Value = typeof(DriverBridgeConfigurationOptions.DriverRestartRunningTaskHandlers))] ISet<IObserver<IRunningTask>> driverRestartRunningTaskHandlers,
             [Parameter(Value = typeof(DriverBridgeConfigurationOptions.TraceListenersSet))] ISet<TraceListener> traceListeners,
+            [Parameter(Value = typeof(EvaluatorConfigurationProviders))] ISet<IConfigurationProvider> configurationProviders,
             [Parameter(Value = typeof(DriverBridgeConfigurationOptions.TraceLevel))] string traceLevel,
             HttpServerHandler httpServerHandler)
         {
@@ -160,6 +165,7 @@ namespace Org.Apache.REEF.Driver.Bridge
             _driverRestartActiveContextHandlers = driverRestartActiveContextHandlers;
             _driverRestartRunningTaskHandlers = driverRestartRunningTaskHandlers;
             _httpServerHandler = httpServerHandler;
+            _configurationProviders = configurationProviders;
 
             _evaluatorRequestorSubscriber = new ClrSystemHandler<IEvaluatorRequestor>();
             _allocatedEvaluatorSubscriber = new ClrSystemHandler<IAllocatedEvaluator>();
@@ -316,5 +322,7 @@ namespace Org.Apache.REEF.Driver.Bridge
 
             return handlers;
         }
+
+        internal ISet<IConfigurationProvider> ConfigurationProviders { get { return _configurationProviders; } }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/AllocatedEvaluator.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/AllocatedEvaluator.cs b/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/AllocatedEvaluator.cs
index 7db4be8..05f22ba 100644
--- a/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/AllocatedEvaluator.cs
+++ b/lang/cs/Org.Apache.REEF.Driver/Bridge/Events/AllocatedEvaluator.cs
@@ -26,6 +26,7 @@ using Org.Apache.REEF.Common.Evaluator;
 using Org.Apache.REEF.Driver.Bridge.Clr2java;
 using Org.Apache.REEF.Driver.Evaluator;
 using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Implementations.Configuration;
 using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Utilities.Logging;
 
@@ -40,8 +41,11 @@ namespace Org.Apache.REEF.Driver.Bridge.Events
 
         private IEvaluatorDescriptor _evaluatorDescriptor;
 
-        public AllocatedEvaluator(IAllocatedEvaluaotrClr2Java clr2Java)
+        private readonly ISet<IConfigurationProvider> _configurationProviders;
+
+        public AllocatedEvaluator(IAllocatedEvaluaotrClr2Java clr2Java, ISet<IConfigurationProvider> configurationProviders)
         {
+            _configurationProviders = configurationProviders;
             InstanceId = Guid.NewGuid().ToString("N");
             _serializer = new AvroConfigurationSerializer();
             Clr2Java = clr2Java;
@@ -77,6 +81,7 @@ namespace Org.Apache.REEF.Driver.Bridge.Events
         {
             LOGGER.Log(Level.Info, "AllocatedEvaluator::SubmitContextAndTask");
 
+            contextConfiguration = MergeContextConfiguration(contextConfiguration);
             string context = _serializer.ToString(contextConfiguration);
             string task = _serializer.ToString(taskConfiguration);
 
@@ -90,6 +95,7 @@ namespace Org.Apache.REEF.Driver.Bridge.Events
         {
             LOGGER.Log(Level.Info, "AllocatedEvaluator::SubmitContextAndService");
 
+            contextConfiguration = MergeContextConfiguration(contextConfiguration);
             string context = _serializer.ToString(contextConfiguration);
             string service = _serializer.ToString(serviceConfiguration);
 
@@ -103,6 +109,7 @@ namespace Org.Apache.REEF.Driver.Bridge.Events
         {
             LOGGER.Log(Level.Info, "AllocatedEvaluator::SubmitContextAndServiceAndTask");
 
+            contextConfiguration = MergeContextConfiguration(contextConfiguration);
             string context = _serializer.ToString(contextConfiguration);
             string service = _serializer.ToString(serviceConfiguration);
             string task = _serializer.ToString(taskConfiguration);
@@ -171,5 +178,18 @@ namespace Org.Apache.REEF.Driver.Bridge.Events
                 }
             }
         }
+
+        private IConfiguration MergeContextConfiguration(IConfiguration contextConfiguration)
+        {
+            IConfiguration contextConfig = contextConfiguration;
+            if (_configurationProviders != null)
+            {
+                foreach (var configurationProvider in _configurationProviders)
+                {
+                    contextConfig = Configurations.Merge(contextConfig, configurationProvider.GetConfiguration());
+                }
+            }
+            return contextConfig;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
index c0c81ff..1e74c9f 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
@@ -18,14 +18,19 @@
  */
 
 using System;
+using System.Linq;
+using System.Net;
+using System.Net.Sockets;
 using Org.Apache.REEF.Client.API;
 using Org.Apache.REEF.Client.Local;
 using Org.Apache.REEF.Client.YARN;
+using Org.Apache.REEF.Common.Io;
 using Org.Apache.REEF.Driver.Bridge;
 using Org.Apache.REEF.Tang.Annotations;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.Wake.Remote;
 
 namespace Org.Apache.REEF.Examples.HelloREEF
 {
@@ -37,11 +42,13 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         private const string Local = "local";
         private const string YARN = "yarn";
         private readonly IREEFClient _reefClient;
+        private readonly JobSubmissionBuilderFactory _jobSubmissionBuilderFactory;
 
         [Inject]
-        private HelloREEF(IREEFClient reefClient)
+        private HelloREEF(IREEFClient reefClient, JobSubmissionBuilderFactory jobSubmissionBuilderFactory)
         {
             _reefClient = reefClient;
+            _jobSubmissionBuilderFactory = jobSubmissionBuilderFactory;
         }
 
         /// <summary>
@@ -56,10 +63,11 @@ namespace Org.Apache.REEF.Examples.HelloREEF
                 .Set(DriverBridgeConfiguration.OnDriverStarted, GenericType<HelloDriver>.Class)
                 .Build();
             // The JobSubmission contains the Driver configuration as well as the files needed on the Driver.
-            var helloJobSubmission = new JobSubmission()
+            var helloJobSubmission = _jobSubmissionBuilderFactory.GetJobSubmissionBuilder()
                 .AddDriverConfiguration(helloDriverConfiguration)
-                .AddGlobalAssemblyForType(typeof(HelloDriver))
-                .SetJobIdentifier("HelloREEF");
+                .AddGlobalAssemblyForType(typeof (HelloDriver))
+                .SetJobIdentifier("HelloREEF")
+                .Build();
 
             _reefClient.Submit(helloJobSubmission);
         }
@@ -85,7 +93,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
 
         public static void Main(string[] args)
         {
-            TangFactory.GetTang().NewInjector(GetRuntimeConfiguration(args[0])).GetInstance<HelloREEF>().Run();
+            TangFactory.GetTang().NewInjector(GetRuntimeConfiguration(args.Length > 0 ? args[0] : Local)).GetInstance<HelloREEF>().Run();
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Tang/Interface/IConfigurationProvider.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Interface/IConfigurationProvider.cs b/lang/cs/Org.Apache.REEF.Tang/Interface/IConfigurationProvider.cs
new file mode 100644
index 0000000..fe1f588
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tang/Interface/IConfigurationProvider.cs
@@ -0,0 +1,32 @@
+/*
+ * 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.Tang.Interface
+{
+    /// <summary>
+    /// Provides configurations
+    /// </summary>
+    public interface IConfigurationProvider
+    {
+        /// <summary>
+        /// <returns>The IConfiguration used for instantiation</returns>
+        /// </summary>
+        IConfiguration GetConfiguration();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/86744909/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj b/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj
index 8a837e2..8f84149 100644
--- a/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj
+++ b/lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj
@@ -105,6 +105,7 @@ under the License.
     <Compile Include="Interface\ICsInternalConfigurationBuilder.cs" />
     <Compile Include="Interface\IExternalConstructor.cs" />
     <Compile Include="Interface\IInjector.cs" />
+    <Compile Include="Interface\IConfigurationProvider.cs" />
     <Compile Include="Interface\ITang.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
     <Compile Include="Protobuf\class_hierarchy.cs" />