You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by do...@apache.org on 2018/03/10 01:33:10 UTC

[01/28] reef git commit: [REEF-1910] Fix TestResubmitEvaluator unit test failure

Repository: reef
Updated Branches:
  refs/heads/REEF-335 72ecec743 -> b759764c7


[REEF-1910] Fix TestResubmitEvaluator unit test failure

Fix concurrency issue in the test driver.

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

Pull Request:
  This closes #1400


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

Branch: refs/heads/REEF-335
Commit: e147293f2f6635171851aaabbd220e11d7465642
Parents: 72ecec7
Author: Julia Wang <jw...@yahoo.com>
Authored: Wed Oct 25 11:57:29 2017 -0700
Committer: Markus Weimer <we...@apache.org>
Committed: Wed Oct 25 15:07:31 2017 -0700

----------------------------------------------------------------------
 .../Functional/FaultTolerant/TestResubmitEvaluator.cs     | 10 ++++------
 1 file changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/e147293f/lang/cs/Org.Apache.REEF.Tests/Functional/FaultTolerant/TestResubmitEvaluator.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/FaultTolerant/TestResubmitEvaluator.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/FaultTolerant/TestResubmitEvaluator.cs
index d0bddde..bb96848 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Functional/FaultTolerant/TestResubmitEvaluator.cs
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/FaultTolerant/TestResubmitEvaluator.cs
@@ -103,8 +103,8 @@ namespace Org.Apache.REEF.Tests.Functional.FaultTolerant
             private static readonly Logger Logger = Logger.GetLogger(typeof(ResubmitEvaluatorDriver));
 
             private readonly IEvaluatorRequestor _requestor;
-            private int _taskNumber = 1;
-            private int _contextNumber = 1;
+            private int _taskNumber;
+            private int _contextNumber;
             private string _failedContextId;
             private string _failedTaskId = TaskId + "1";
             private readonly ISet<ICompletedTask> _completedTasks = new HashSet<ICompletedTask>();
@@ -126,21 +126,19 @@ namespace Org.Apache.REEF.Tests.Functional.FaultTolerant
                 Logger.Log(Level.Info, "AllocatedEvaluator: " + value.Id);
                 value.SubmitContext(
                     ContextConfiguration.ConfigurationModule
-                        .Set(ContextConfiguration.Identifier, ContextId + _contextNumber)
+                        .Set(ContextConfiguration.Identifier, ContextId + Interlocked.Increment(ref _contextNumber))
                         .Build());
-                Interlocked.Increment(ref _contextNumber);
             }
 
             public void OnNext(IActiveContext value)
             {
                 Logger.Log(Level.Info, "ActiveContext: " + value.Id);
                 value.SubmitTask(TaskConfiguration.ConfigurationModule
-                    .Set(TaskConfiguration.Identifier, TaskId + _taskNumber)
+                    .Set(TaskConfiguration.Identifier, TaskId + Interlocked.Increment(ref _taskNumber))
                     .Set(TaskConfiguration.Task, GenericType<FailEvaluatorTask>.Class)
                     .Set(TaskConfiguration.OnMessage, GenericType<FailEvaluatorTask>.Class)
                     .Set(TaskConfiguration.OnClose, GenericType<FailEvaluatorTask>.Class)
                     .Build());
-                Interlocked.Increment(ref _taskNumber);
             }
 
             public void OnNext(IRunningTask value)


[22/28] reef git commit: [REEF-1968] Allow the use of C#7.2

Posted by do...@apache.org.
[REEF-1968] Allow the use of C#7.2

This adds a directiive to both `build.props` and `build.DotNet.props` to
allow for C# 7.2 code.

This also adds the `.vscode` folder to `.gitignore`.

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

Pull Request:
  This closes #1422


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

Branch: refs/heads/REEF-335
Commit: b0b9cb1dd5048da7429cf2fd3143de7a6ebdea75
Parents: 8c1fbf1
Author: Markus Weimer <we...@apache.org>
Authored: Thu Dec 21 11:53:58 2017 -0800
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Thu Jan 4 11:48:39 2018 -0800

----------------------------------------------------------------------
 .gitignore                 | 1 +
 lang/cs/build.DotNet.props | 5 +++++
 lang/cs/build.props        | 5 +++++
 3 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/b0b9cb1d/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index dcb1de0..b4032c4 100644
--- a/.gitignore
+++ b/.gitignore
@@ -29,6 +29,7 @@ atlassian-ide-plugin.xml
 *.VC.db
 *.vcxproj.user
 .vs
+.vscode
 #
 # ----------------------------------------------------------------------
 # OS Files

http://git-wip-us.apache.org/repos/asf/reef/blob/b0b9cb1d/lang/cs/build.DotNet.props
----------------------------------------------------------------------
diff --git a/lang/cs/build.DotNet.props b/lang/cs/build.DotNet.props
index a202355..1635d52 100644
--- a/lang/cs/build.DotNet.props
+++ b/lang/cs/build.DotNet.props
@@ -33,6 +33,11 @@ under the License.
     <DefaultItemExcludes>$(DefaultItemExcludes);**/AssemblyInfo.cs;packages.config;*.nuspec</DefaultItemExcludes>
   </PropertyGroup>
 
+  <!-- Allow for C# 7.2 language constructs -->
+  <PropertyGroup>
+    <LangVersion>7.2</LangVersion>
+  </PropertyGroup>
+  
   <PropertyGroup>
     <Platforms>x64</Platforms>
     <PlatformTarget>x64</PlatformTarget>

http://git-wip-us.apache.org/repos/asf/reef/blob/b0b9cb1d/lang/cs/build.props
----------------------------------------------------------------------
diff --git a/lang/cs/build.props b/lang/cs/build.props
index f2f6020..63f62eb 100644
--- a/lang/cs/build.props
+++ b/lang/cs/build.props
@@ -56,6 +56,11 @@ under the License.
     <WarningsNotAsErrors>612,618</WarningsNotAsErrors>
   </PropertyGroup>
 
+  <!-- Allow for C# 7.2 language constructs -->
+  <PropertyGroup>
+    <LangVersion>7.2</LangVersion>
+  </PropertyGroup>
+
   <!-- REEF NuGet properties -->
   <PropertyGroup>
     <IsSnapshot>true</IsSnapshot>


[19/28] reef git commit: [REEF-1826] Clarify the API to add assemblies to a Job submission

Posted by do...@apache.org.
[REEF-1826] Clarify the API to add assemblies to a Job submission

This addressed the issue by

  * Moved all functionality for assigning global assemblies to the
    `JobRequestBuilder`; removed from `DriverFolderPreparationHelper`
  * Job request builder adds the assemblies of the local directory if no
    assemblies were added (local or global) and logs a warning.
  * Added a helper function to add assemblies from the Executing
    Assembly's directory
  * Added a helper function to use the directory of the currently-running
    executable
  * Updated HelloREEF to use the executable directory instead of the
    current working directory

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

Pull request:
  This closes #1826


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

Branch: refs/heads/REEF-335
Commit: 4296e20f24baded5143f1f53c29b2deb2f5e90a0
Parents: cc1073f
Author: roganc <ro...@microsoft.com>
Authored: Sat Nov 4 16:35:45 2017 -0700
Committer: Markus Weimer <we...@apache.org>
Committed: Thu Nov 16 17:35:29 2017 -0800

----------------------------------------------------------------------
 .../API/JobRequestBuilder.cs                    | 82 +++++++++++++++++++-
 .../Common/DriverFolderPreparationHelper.cs     | 31 --------
 .../HelloREEF.cs                                | 11 ++-
 .../ClientUtilities.cs                          | 38 +++++++++
 .../Org.Apache.Reef.Utilities.csproj            |  1 +
 5 files changed, 129 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/4296e20f/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs b/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs
index 7760b7f..29d7a66 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs
@@ -18,23 +18,32 @@
 using System;
 using System.Collections.Generic;
 using Org.Apache.REEF.Common.Client.Parameters;
+using Org.Apache.REEF.Client.Common;
 using Org.Apache.REEF.Tang.Annotations;
 using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Utilities.Logging;
+using System.IO;
+using System.Linq;
+using Org.Apache.REEF.Utilities;
 
 namespace Org.Apache.REEF.Client.API
 {
     public sealed class JobRequestBuilder
     {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(JobRequestBuilder));
         private readonly AppParametersBuilder _appParametersBuilder = AppParametersBuilder.NewBuilder();
         private readonly JobParametersBuilder _jobParametersBuilder = JobParametersBuilder.NewBuilder();
+        private const string DLLFileNameExtension = ".dll";
+        private const string EXEFileNameExtension = ".exe";
+        private bool _assembliesAdded = false;
 
         private JobRequestBuilder()
         {
         }
 
         [Inject]
-        internal JobRequestBuilder([Parameter(typeof(DriverConfigurationProviders))] ISet<IConfigurationProvider> configurationProviders)
+        internal JobRequestBuilder(
+            [Parameter(typeof(DriverConfigurationProviders))] ISet<IConfigurationProvider> configurationProviders)
         {
             AddDriverConfigurationProviders(configurationProviders);
         }
@@ -44,6 +53,13 @@ namespace Org.Apache.REEF.Client.API
         /// </summary>
         public JobRequest Build()
         {
+            // If no assemblies have been added, default to those in the working directory
+            if (!_assembliesAdded)
+            {
+                Logger.Log(Level.Warning, "No assemlies added to the job; Adding assemblies from the current working directory.");
+                AddGlobalAssembliesInDirectory(Directory.GetCurrentDirectory());
+            }
+
             return new JobRequest(_jobParametersBuilder.Build(), _appParametersBuilder.Build());
         }
 
@@ -52,6 +68,7 @@ namespace Org.Apache.REEF.Client.API
         /// </summary>
         public JobRequestBuilder AddGlobalFile(string fileName)
         {
+            ThrowIfFileDoesnotExist(fileName, "Global File");
             _appParametersBuilder.AddGlobalFile(fileName);
             return this;
         }
@@ -61,6 +78,7 @@ namespace Org.Apache.REEF.Client.API
         /// </summary>
         public JobRequestBuilder AddLocalFile(string fileName)
         {
+            ThrowIfFileDoesnotExist(fileName, "Local File");
             _appParametersBuilder.AddLocalFile(fileName);
             return this;
         }
@@ -72,18 +90,54 @@ namespace Org.Apache.REEF.Client.API
         /// <returns></returns>
         public JobRequestBuilder AddGlobalAssembly(string fileName)
         {
+            ThrowIfFileDoesnotExist(fileName, "Global Assembly");
             _appParametersBuilder.AddGlobalAssembly(fileName);
+            _assembliesAdded = true;
             return this;
         }
 
         /// <summary>
+        /// Add all the assemblies in a directory of type EXE, DLL, or with the Client-JarFile Prefix to the global path
+        /// </summary>
+        /// <param name="globalAssemblyDirectory">The directory to search for assemblies</param>
+        public JobRequestBuilder AddGlobalAssembliesInDirectory(string globalAssemblyDirectory)
+        {
+            if (Directory.Exists(globalAssemblyDirectory))
+            {
+                // For input paths that are directories, extract only files of a predetermined type
+                foreach (var assembly in Directory.GetFiles(globalAssemblyDirectory).Where(IsAssemblyToCopy))
+                {
+                    AddGlobalAssembly(assembly);
+                }
+            }
+            else
+            {
+                // Throw if a path input was not a file or a directory
+                throw new FileNotFoundException(string.Format("Global Assembly Directory not Found: {0}", globalAssemblyDirectory));
+            }
+            return this;
+        }
+
+        /// <summary>
+        /// Add any assemblies of type EXE, DLL, or with the Client-JarFile Prefix to the global path
+        /// found in the same directory as the executing assembly's directory
+        /// </summary>
+        public JobRequestBuilder AddGlobalAssembliesInDirectoryOfExecutingAssembly()
+        {
+            var directory = ClientUtilities.GetPathToExecutingAssembly();
+            return AddGlobalAssembliesInDirectory(directory);
+        }
+
+        /// <summary>
         /// Add an assembly to the driver only.
         /// </summary>
         /// <param name="fileName"></param>
         /// <returns></returns>
         public JobRequestBuilder AddLocalAssembly(string fileName)
         {
+            ThrowIfFileDoesnotExist(fileName, "Local Assembly");
             _appParametersBuilder.AddLocalAssembly(fileName);
+            _assembliesAdded = true;
             return this;
         }
 
@@ -204,5 +258,31 @@ namespace Org.Apache.REEF.Client.API
             _jobParametersBuilder.SetJavaLogLevel(javaLogLevel);
             return this;
         }
+
+        /// <summary>
+        /// Returns true, if the given file path references a DLL or EXE or JAR.
+        /// </summary>
+        /// <param name="filePath"></param>
+        /// <returns></returns>
+        private static bool 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.ClientJarFilePrefix);
+        }
+
+        private static void ThrowIfFileDoesnotExist(string path, string fileDescription)
+        {
+            if (!File.Exists(path))
+            {
+                throw new FileNotFoundException(string.Format("{0} not Found: {1}", fileDescription, path));
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/4296e20f/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 7401235..dfab211 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
@@ -87,9 +87,6 @@ namespace Org.Apache.REEF.Client.Common
             // Create the driver configuration
             CreateDriverConfiguration(appParameters, driverFolderPath);
 
-            // Add the REEF assemblies
-            AddAssemblies();
-
             // Initiate the final copy
             _fileSets.CopyToDriverFolder(driverFolderPath);
 
@@ -149,33 +146,5 @@ namespace Org.Apache.REEF.Client.Common
             File.WriteAllText(Path.Combine(driverFolderPath, _fileNames.GetGlobalFolderPath(), EvaluatorExecutable), 
                 DefaultDriverConfigurationFileContents);
         }
-
-        /// <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 bool 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.ClientJarFilePrefix);
-        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/4296e20f/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 8f9053d..33f5aa7 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
@@ -17,6 +17,7 @@
 
 using System;
 using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.Common;
 using Org.Apache.REEF.Client.Local;
 using Org.Apache.REEF.Client.Yarn;
 using Org.Apache.REEF.Client.YARN.HDI;
@@ -27,7 +28,6 @@ using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Tang.Util;
 using Org.Apache.REEF.Utilities.Logging;
-using Org.Apache.REEF.Client.Common;
 
 namespace Org.Apache.REEF.Examples.HelloREEF
 {
@@ -64,6 +64,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
             var helloJobRequest = _reefClient.NewJobRequestBuilder()
                 .AddDriverConfiguration(helloDriverConfiguration)
                 .AddGlobalAssemblyForType(typeof(HelloDriver))
+                .AddGlobalAssembliesInDirectoryOfExecutingAssembly()
                 .SetJobIdentifier("HelloREEF")
                 .SetJavaLogLevel(JavaLoggingSetting.Verbose)
                 .Build();
@@ -110,7 +111,13 @@ namespace Org.Apache.REEF.Examples.HelloREEF
 
         public static void MainSimple(string[] args)
         {
-            TangFactory.GetTang().NewInjector(GetRuntimeConfiguration(args.Length > 0 ? args[0] : Local)).GetInstance<HelloREEF>().Run();
+            var runtime = args.Length > 0 ? args[0] : Local;
+
+            // Execute the HelloREEF, with these parameters injected
+            TangFactory.GetTang()
+                .NewInjector(GetRuntimeConfiguration(runtime))
+                .GetInstance<HelloREEF>()
+                .Run();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/4296e20f/lang/cs/Org.Apache.REEF.Utilities/ClientUtilities.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Utilities/ClientUtilities.cs b/lang/cs/Org.Apache.REEF.Utilities/ClientUtilities.cs
new file mode 100644
index 0000000..c70ab39
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Utilities/ClientUtilities.cs
@@ -0,0 +1,38 @@
+// 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.Reflection;
+
+namespace Org.Apache.REEF.Utilities
+{
+    /// <summary>
+    /// A library of helper functions for REEF clients
+    /// </summary>
+    public static class ClientUtilities
+    {
+        /// <summary>
+        /// Gets the path of the executing assembly as a string.
+        /// </summary>
+        /// <returns>A string path to the executing assembly</returns>
+        public static string GetPathToExecutingAssembly()
+        {
+            return Path.GetDirectoryName(new Uri(Assembly.GetExecutingAssembly().CodeBase).AbsolutePath);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/4296e20f/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj b/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj
index 0e5ce4e..660a280 100644
--- a/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj
+++ b/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj
@@ -54,6 +54,7 @@ under the License.
     <Compile Include="Attributes\ThreadSafeAttribute.cs" />
     <Compile Include="Attributes\UnstableAttribute.cs" />
     <Compile Include="ByteUtilities.cs" />
+    <Compile Include="ClientUtilities.cs" />
     <Compile Include="Collections\PriorityQueue.cs" />
     <Compile Include="Collections\ReadOnlySet.cs" />
     <Compile Include="Diagnostics\DiagnosticsMessages.cs" />


[08/28] reef git commit: [REEF-1936] Use dependency injection in C# ProtocolSerializer

Posted by do...@apache.org.
[REEF-1936] Use dependency injection in C# ProtocolSerializer

  Summary of changes:
    * Make `ProtocolSerializer` constructor injectible
    * Create named parameters for the constructor's input
    * Fix the `ProtocolSerializerTest` unit tests to use injection
    * Bugfix: register the `Header` protocol object regardless of the namespace
    * bugfix for the future PR: relax the type constraints in `ProtocolSerializer.Read()` to make injection easier for the observers.
    * Minor fixes to make code compliant with REEF condig standards and naming conventions
    * Minor refactoring

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

Pull Request:
  Closes #1402


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

Branch: refs/heads/REEF-335
Commit: 94af38d363b5c8ea4fef0d920082113523442b8a
Parents: eb522e9
Author: Sergiy Matusevych <mo...@apache.com>
Authored: Thu Oct 26 15:21:14 2017 -0700
Committer: Doug Service <do...@apache.org>
Committed: Thu Nov 2 23:16:44 2017 +0000

----------------------------------------------------------------------
 .../ProtocolSerializerTest.cs                   | 34 ++++++---
 .../Avro/ProtocolSerializer.cs                  | 78 +++++++++++++-------
 2 files changed, 72 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/94af38d3/lang/cs/Org.Apache.REEF.Wake.Tests/ProtocolSerializerTest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake.Tests/ProtocolSerializerTest.cs b/lang/cs/Org.Apache.REEF.Wake.Tests/ProtocolSerializerTest.cs
index cf3ac02..574f522 100644
--- a/lang/cs/Org.Apache.REEF.Wake.Tests/ProtocolSerializerTest.cs
+++ b/lang/cs/Org.Apache.REEF.Wake.Tests/ProtocolSerializerTest.cs
@@ -25,6 +25,7 @@ using Org.Apache.REEF.Wake.Remote.Impl;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 using org.apache.reef.wake.tests.message;
 using Xunit;
+using Org.Apache.REEF.Tang.Interface;
 
 namespace Org.Apache.REEF.Wake.Tests
 {
@@ -33,32 +34,36 @@ namespace Org.Apache.REEF.Wake.Tests
     /// </summary>
     internal sealed class TestMessageObserver : IObserver<IMessageInstance<AvroTestMessage>>
     {
-        private readonly IMessageInstance<AvroTestMessage> messageInstance;
+        private readonly IMessageInstance<AvroTestMessage> _messageInstance;
 
         public TestMessageObserver(long seq, AvroTestMessage msg)
         {
-            messageInstance = new MessageInstance<AvroTestMessage>(seq, msg);
+            _messageInstance = new MessageInstance<AvroTestMessage>(seq, msg);
         }
 
         public void OnNext(IMessageInstance<AvroTestMessage> otherMessageInstance)
         {
-            Assert.Equal(messageInstance.Message.number, otherMessageInstance.Message.number);
-            Assert.Equal(messageInstance.Message.data, otherMessageInstance.Message.data);
+            Assert.Equal(_messageInstance.Message.number, otherMessageInstance.Message.number);
+            Assert.Equal(_messageInstance.Message.data, otherMessageInstance.Message.data);
         }
 
         public void OnError(Exception error)
         {
-            throw new NotImplementedException();
+            throw new NotImplementedException("This method should never be called");
         }
 
         public void OnCompleted()
         {
-            throw new NotImplementedException();
+            throw new NotImplementedException("This method should never be called");
         }
     }
 
     public sealed class TestProtocolSerializer
     {
+        private static readonly ITang Tang = TangFactory.GetTang();
+        private static readonly IPAddress ListeningAddress = IPAddress.Parse("127.0.0.1");
+        private static readonly ByteCodec Codec = new ByteCodec();
+
         /// <summary>
         /// Setup two way communication between two remote managers through the loopback
         /// network and verify that Avro messages are properly serialized and deserialzied
@@ -72,18 +77,23 @@ namespace Org.Apache.REEF.Wake.Tests
             int[] numbers = { 12, 25 };
             string[] strings = { "The first string", "The second string" };
 
-            IPAddress listeningAddress = IPAddress.Parse("127.0.0.1");
             BlockingCollection<byte[]> queue1 = new BlockingCollection<byte[]>();
             BlockingCollection<byte[]> queue2 = new BlockingCollection<byte[]>();
 
-            ProtocolSerializer serializer = new ProtocolSerializer(this.GetType().Assembly, "org.apache.reef.wake.tests.message");
-            IRemoteManagerFactory _remoteManagerFactory = TangFactory.GetTang().NewInjector().GetInstance<IRemoteManagerFactory>();
+            IConfiguration config = Tang.NewConfigurationBuilder()
+                .BindStringNamedParam<ProtocolSerializer.AssemblyName>(this.GetType().Assembly.FullName)
+                .BindStringNamedParam<ProtocolSerializer.MessageNamespace>("org.apache.reef.wake.tests.message")
+                .Build();
+
+            var injector = Tang.NewInjector(config);
+            var remoteManagerFactory = injector.GetInstance<IRemoteManagerFactory>();
+            var serializer = injector.GetInstance<ProtocolSerializer>();
 
-            using (var remoteManager1 = _remoteManagerFactory.GetInstance(listeningAddress, new ByteCodec()))
-            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, new ByteCodec()))
+            using (var remoteManager1 = remoteManagerFactory.GetInstance(ListeningAddress, Codec))
+            using (var remoteManager2 = remoteManagerFactory.GetInstance(ListeningAddress, Codec))
             {
                 // Register observers for remote manager 1 and remote manager 2
-                var remoteEndpoint = new IPEndPoint(listeningAddress, 0);
+                var remoteEndpoint = new IPEndPoint(ListeningAddress, 0);
                 remoteManager1.RegisterObserver(remoteEndpoint, Observer.Create<byte[]>(queue1.Add));
                 remoteManager2.RegisterObserver(remoteEndpoint, Observer.Create<byte[]>(queue2.Add));
 

http://git-wip-us.apache.org/repos/asf/reef/blob/94af38d3/lang/cs/Org.Apache.REEF.Wake/Avro/ProtocolSerializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake/Avro/ProtocolSerializer.cs b/lang/cs/Org.Apache.REEF.Wake/Avro/ProtocolSerializer.cs
index 4adcc88..df93138 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Avro/ProtocolSerializer.cs
+++ b/lang/cs/Org.Apache.REEF.Wake/Avro/ProtocolSerializer.cs
@@ -22,6 +22,7 @@ using System.Reflection;
 using Microsoft.Hadoop.Avro;
 using Org.Apache.REEF.Utilities.Logging;
 using org.apache.reef.wake.avro.message;
+using Org.Apache.REEF.Tang.Annotations;
 
 namespace Org.Apache.REEF.Wake.Avro
 {
@@ -34,7 +35,17 @@ namespace Org.Apache.REEF.Wake.Avro
     /// </summary>
     public sealed class ProtocolSerializer
     {
-        private static readonly Logger Logr = Logger.GetLogger(typeof(ProtocolSerializer));
+        [NamedParameter("Name of the assembly that contains serializable classes.")]
+        public class AssemblyName : Name<string>
+        {
+        }
+
+        [NamedParameter("Package name to search for serializabe classes.")]
+        public class MessageNamespace : Name<string>
+        {
+        }
+
+        private static readonly Logger Log = Logger.GetLogger(typeof(ProtocolSerializer));
 
         /// <summary>
         /// Delegate for message serializer.
@@ -50,16 +61,14 @@ namespace Org.Apache.REEF.Wake.Avro
         /// <summary>
         /// Map from message type (a string with the message class name) to serializer.
         /// </summary>
-        private readonly SortedDictionary<string, Serialize>
-            serializeMap = new SortedDictionary<string, Serialize>();
+        private readonly IDictionary<string, Serialize> _serializeMap = new Dictionary<string, Serialize>();
 
         /// <summary>
         /// Map from message type (a string with the message class name) to deserializer.
         /// </summary>
-        private readonly SortedDictionary<string, Deserialize>
-            deserializeMap = new SortedDictionary<string, Deserialize>();
+        private readonly IDictionary<string, Deserialize> _deserializeMap = new Dictionary<string, Deserialize>();
 
-        private readonly IAvroSerializer<Header> headerSerializer = AvroSerializer.Create<Header>();
+        private readonly IAvroSerializer<Header> _headerSerializer = AvroSerializer.Create<Header>();
 
         /// <summary>
         /// Non-generic reflection record for the Register() method of this class. A constant.
@@ -68,40 +77,50 @@ namespace Org.Apache.REEF.Wake.Avro
             typeof(ProtocolSerializer).GetMethod("Register", BindingFlags.Instance | BindingFlags.NonPublic);
 
         /// <summary>
-        /// Register all of the protocol messages using reflection.
+        /// Construct an initialized protocol serializer.
         /// </summary>
-        /// <param name="assembly">The Assembly object which contains the namespace of the message classes.</param>
+        /// <param name="assemblyName">The full name of the assembly
+        /// which contains the namespace of the message classes.</param>
         /// <param name="messageNamespace">A string which contains the namespace the protocol messages.</param>
-        public ProtocolSerializer(Assembly assembly, string messageNamespace)
+        [Inject]
+        public ProtocolSerializer(
+            [Parameter(typeof(AssemblyName))] string assemblyName,
+            [Parameter(typeof(MessageNamespace))] string messageNamespace)
         {
-            Logr.Log(Level.Verbose, "Retrieving types for assembly: {0}", assembly.FullName);
+            Log.Log(Level.Info, "Retrieving types for assembly: {0}", assemblyName);
+            Assembly assembly = Assembly.Load(assemblyName);
 
-            var types = new List<Type>(assembly.GetTypes()) { typeof(Header) };
-            foreach (Type type in types)
+            CreateProtocolObject(typeof(Header));
+            foreach (Type type in assembly.GetTypes())
             {
                 if (type.FullName.StartsWith(messageNamespace))
                 {
-                    MethodInfo genericInfo = RegisterMethodInfo.MakeGenericMethod(new[] { type });
-                    genericInfo.Invoke(this, null);
+                    CreateProtocolObject(type);
                 }
             }
         }
 
+        private void CreateProtocolObject(Type type)
+        {
+            MethodInfo genericInfo = RegisterMethodInfo.MakeGenericMethod(new[] { type });
+            genericInfo.Invoke(this, null);
+        }
+
         /// <summary>
         /// Generate and store the metadata necessary to serialze and deserialize a specific message type.
         /// </summary>
         /// <typeparam name="TMessage">The class type of the message being registered.</typeparam>
         internal void Register<TMessage>()
         {
-            Logr.Log(Level.Info, "Registering message type: {0} {1}",
-                typeof(TMessage).FullName, typeof(TMessage).Name);
+            string name = typeof(TMessage).FullName;
+            Log.Log(Level.Info, "Registering message type: {0}", name);
 
             IAvroSerializer<TMessage> messageSerializer = AvroSerializer.Create<TMessage>();
             Serialize serialize = (MemoryStream stream, object message) =>
             {
                 messageSerializer.Serialize(stream, (TMessage)message);
             };
-            serializeMap.Add(typeof(TMessage).Name, serialize);
+            _serializeMap.Add(name, serialize);
 
             Deserialize deserialize = (MemoryStream stream, object observer, long sequence) =>
             {
@@ -109,14 +128,16 @@ namespace Org.Apache.REEF.Wake.Avro
                 var msgObserver = observer as IObserver<IMessageInstance<TMessage>>;
                 if (msgObserver != null)
                 {
+                    Log.Log(Level.Verbose, "Invoking message observer {0} with message {1}", msgObserver, message);
                     msgObserver.OnNext(new MessageInstance<TMessage>(sequence, message));
                 }
                 else
                 {
-                    Logr.Log(Level.Warning, "Unhandled message received: {0}", message);
+                    Log.Log(Level.Warning, "Unhandled message received: {0}", message);
                 }
             };
-            deserializeMap.Add(typeof(TMessage).Name, deserialize);
+
+            _deserializeMap.Add(name, deserialize);
         }
 
         /// <summary>
@@ -128,17 +149,17 @@ namespace Org.Apache.REEF.Wake.Avro
         /// <returns>A byte array containing the serialized header and message.</returns>
         public byte[] Write(object message, long sequence) 
         {
-            string name = message.GetType().Name;
-            Logr.Log(Level.Info, "Serializing message: {0}", name);
+            string name = message.GetType().FullName;
+            Log.Log(Level.Verbose, "Serializing message: {0}", name);
             try
             { 
                 using (MemoryStream stream = new MemoryStream())
                 {
                     Header header = new Header(sequence, name);
-                    headerSerializer.Serialize(stream, header);
+                    _headerSerializer.Serialize(stream, header);
 
                     Serialize serialize;
-                    if (serializeMap.TryGetValue(name, out serialize))
+                    if (_serializeMap.TryGetValue(name, out serialize))
                     {
                         serialize(stream, message);
                     }
@@ -152,7 +173,7 @@ namespace Org.Apache.REEF.Wake.Avro
             }
             catch (Exception e)
             {
-                Logr.Log(Level.Error, "Failure writing message.", e);
+                Log.Log(Level.Error, "Failure writing message.", e);
                 throw e;
             }
         }
@@ -163,15 +184,16 @@ namespace Org.Apache.REEF.Wake.Avro
         /// <param name="data">Byte array containing a header message and message to be deserialized.</param>
         /// <param name="observer">An object which implements the IObserver<>
         /// interface for the message being deserialized.</param>
-        public void Read<T>(byte[] data, IObserver<IMessageInstance<T>> observer)
+        public void Read(byte[] data, object observer)
         {
             try
             {
                 using (MemoryStream stream = new MemoryStream(data))
                 {
-                    Header head = headerSerializer.Deserialize(stream);
+                    Header head = _headerSerializer.Deserialize(stream);
+                    Log.Log(Level.Verbose, "Message header {0}", head);
                     Deserialize deserialize;
-                    if (deserializeMap.TryGetValue(head.className, out deserialize))
+                    if (_deserializeMap.TryGetValue(head.className, out deserialize))
                     {
                         deserialize(stream, observer, head.sequence);
                     }
@@ -184,7 +206,7 @@ namespace Org.Apache.REEF.Wake.Avro
             }
             catch (Exception e)
             {
-                Logr.Log(Level.Error, "Failure reading message.", e);
+                Log.Log(Level.Error, "Failure reading message.", e);
                 throw e;
             }
         }


[02/28] reef git commit: [REEF-1930] Update environmentVariablesMap name in .NET code

Posted by do...@apache.org.
[REEF-1930] Update environmentVariablesMap name in .NET code

  * Rename envMap to environmentVariablesMap
  * Remove the real env variable in HelloREEFYarn as it impacts the testing.

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

Pull Request:
  This closes #1399


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

Branch: refs/heads/REEF-335
Commit: daf33d56ca5937f49392bb1e950ab6456a81983a
Parents: e147293
Author: Julia Wang <jw...@yahoo.com>
Authored: Tue Oct 24 21:59:22 2017 -0700
Committer: Markus Weimer <we...@apache.org>
Committed: Wed Oct 25 15:29:33 2017 -0700

----------------------------------------------------------------------
 .../YarnREEFParamSerializerTests.cs                            | 2 +-
 .../Avro/YARN/AvroYarnClusterJobSubmissionParameters.cs        | 6 +++---
 lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs | 2 +-
 lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs    | 3 +--
 4 files changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/daf33d56/lang/cs/Org.Apache.REEF.Client.Tests/YarnREEFParamSerializerTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/YarnREEFParamSerializerTests.cs b/lang/cs/Org.Apache.REEF.Client.Tests/YarnREEFParamSerializerTests.cs
index 040a9f8..76c591d 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/YarnREEFParamSerializerTests.cs
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/YarnREEFParamSerializerTests.cs
@@ -159,7 +159,7 @@ namespace Org.Apache.REEF.Client.Tests
                 "\"securityTokenService\":\"{0}\"," +
                 "\"maxApplicationSubmissions\":{1}," +
                 "\"driverMemory\":{1}," +
-                "\"envMap\":{{\"key1\":\"{0}\",\"key2\":\"{0}\"}}," +
+                "\"environmentVariablesMap\":{{\"key1\":\"{0}\",\"key2\":\"{0}\"}}," +
                 "\"driverStdoutFilePath\":\"{0}\"," +
                 "\"driverStderrFilePath\":\"{0}\"" +
                 "}}";

http://git-wip-us.apache.org/repos/asf/reef/blob/daf33d56/lang/cs/Org.Apache.REEF.Client/Avro/YARN/AvroYarnClusterJobSubmissionParameters.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Avro/YARN/AvroYarnClusterJobSubmissionParameters.cs b/lang/cs/Org.Apache.REEF.Client/Avro/YARN/AvroYarnClusterJobSubmissionParameters.cs
index 9bf2a64..d344c4b 100644
--- a/lang/cs/Org.Apache.REEF.Client/Avro/YARN/AvroYarnClusterJobSubmissionParameters.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Avro/YARN/AvroYarnClusterJobSubmissionParameters.cs
@@ -29,7 +29,7 @@ namespace Org.Apache.REEF.Client.Avro.YARN
     [DataContract(Namespace = "org.apache.reef.reef.bridge.client.avro")]
     public sealed class AvroYarnClusterJobSubmissionParameters
     {
-        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroYarnClusterJobSubmissionParameters"",""doc"":""Cross-language submission parameters to the YARN runtime using Hadoop's submission client"",""fields"":[{""name"":""yarnJobSubmissionParameters"",""type"":{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroYarnJobSubmissionParameters"",""doc"":""General cross-language submission parameters to the YARN runtime"",""fields"":[{""name"":""sharedJobSubmissionParameters"",""type"":{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroJobSubmissionParameters"",""doc"":""General cross-language job submission parameters shared by all runtimes"",""fields"":[{""name"":""jobId"",""type"":""string""},{""name"":""jobSubmissionFolder"",""type"":""string""}]}},{""name"":""dfsJobSubmissionFolder"",""type"":""string""},{""name"":""fileSystemUrl"",""type"":""string""},{""name"":""jobSubmissi
 onDirectoryPrefix"",""type"":""string""}]}},{""name"":""securityTokenKind"",""type"":""string""},{""name"":""securityTokenService"",""type"":""string""},{""name"":""driverMemory"",""type"":""int""},{""name"":""envMap"",""type"":{""type"":""map"",""values"":""string""}},{""name"":""maxApplicationSubmissions"",""type"":""int""},{""name"":""driverStdoutFilePath"",""type"":""string""},{""name"":""driverStderrFilePath"",""type"":""string""}]}";
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroYarnClusterJobSubmissionParameters"",""doc"":""Cross-language submission parameters to the YARN runtime using Hadoop's submission client"",""fields"":[{""name"":""yarnJobSubmissionParameters"",""type"":{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroYarnJobSubmissionParameters"",""doc"":""General cross-language submission parameters to the YARN runtime"",""fields"":[{""name"":""sharedJobSubmissionParameters"",""type"":{""type"":""record"",""name"":""org.apache.reef.reef.bridge.client.avro.AvroJobSubmissionParameters"",""doc"":""General cross-language job submission parameters shared by all runtimes"",""fields"":[{""name"":""jobId"",""type"":""string""},{""name"":""jobSubmissionFolder"",""type"":""string""}]}},{""name"":""dfsJobSubmissionFolder"",""type"":""string""},{""name"":""fileSystemUrl"",""type"":""string""},{""name"":""jobSubmissi
 onDirectoryPrefix"",""type"":""string""}]}},{""name"":""securityTokenKind"",""type"":""string""},{""name"":""securityTokenService"",""type"":""string""},{""name"":""driverMemory"",""type"":""int""},{""name"":""environmentVariablesMap"",""type"":{""type"":""map"",""values"":""string""}},{""name"":""maxApplicationSubmissions"",""type"":""int""},{""name"":""driverStdoutFilePath"",""type"":""string""},{""name"":""driverStderrFilePath"",""type"":""string""}]}";
 
         /// <summary>
         /// Gets the schema.
@@ -70,7 +70,7 @@ namespace Org.Apache.REEF.Client.Avro.YARN
         /// Gets or sets the envMap field.
         /// </summary>
         [DataMember]
-        public IDictionary<string, string> envMap { get; set; }
+        public IDictionary<string, string> environmentVariablesMap { get; set; }
 
         /// <summary>
         /// Gets or sets the maxApplicationSubmissions field.
@@ -116,7 +116,7 @@ namespace Org.Apache.REEF.Client.Avro.YARN
             this.securityTokenKind = securityTokenKind;
             this.securityTokenService = securityTokenService;
             this.driverMemory = driverMemory;
-            this.envMap = envMap;
+            this.environmentVariablesMap = envMap;
             this.maxApplicationSubmissions = maxApplicationSubmissions;
             this.driverStdoutFilePath = driverStdoutFilePath;
             this.driverStderrFilePath = driverStderrFilePath;

http://git-wip-us.apache.org/repos/asf/reef/blob/daf33d56/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs
index 8fd6a4f..49285bc 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFParamSerializer.cs
@@ -168,7 +168,7 @@ namespace Org.Apache.REEF.Client.YARN
 
                 yarnJobSubmissionParameters = avroYarnJobSubmissionParameters,
                 driverMemory = jobParameters.DriverMemoryInMB,
-                envMap = jobParameters.JobSubmissionEnvMap,
+                environmentVariablesMap = jobParameters.JobSubmissionEnvMap,
                 maxApplicationSubmissions = jobParameters.MaxApplicationSubmissions,
                 driverStdoutFilePath = string.IsNullOrWhiteSpace(jobParameters.StdoutFilePath.Value) ?
                     _fileNames.GetDefaultYarnDriverStdoutFilePath() : jobParameters.StdoutFilePath.Value,

http://git-wip-us.apache.org/repos/asf/reef/blob/daf33d56/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
index 2285a50..e7dfcf0 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
@@ -94,8 +94,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
                 .AddDriverConfiguration(driverConfig.Build())
                 .AddGlobalAssemblyForType(typeof(HelloDriverYarn))
                 .SetJobIdentifier("HelloREEF")
-                .SetJobSubmissionEnvironmentVariable(Environment.PATH.ToString(), "value1")
-                .SetJobSubmissionEnvironmentVariable("UserDefineKey", "value2")
+                .SetJobSubmissionEnvironmentVariable("UserDefineKey", "value1")
                 .SetJavaLogLevel(JavaLoggingSetting.Verbose)
                 .Build();
 


[24/28] reef git commit: [REEF-1973] Update .NET Target to 4.5.2

Posted by do...@apache.org.
[REEF-1973] Update .NET Target to 4.5.2

This change updates the target framework in all `.csproj` files to .NET
4.5.2.

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

Pull Request:
  This closes #1425


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

Branch: refs/heads/REEF-335
Commit: e43019f40d31341818e921c6180e6fd53e5f3df1
Parents: 227ce1d
Author: Mandy Shieh <me...@microsoft.com>
Authored: Tue Jan 16 11:17:06 2018 -0800
Committer: Markus Weimer <we...@apache.org>
Committed: Sat Jan 27 17:03:09 2018 -0800

----------------------------------------------------------------------
 lang/cs/App.config                                             | 2 +-
 lang/cs/Org.Apache.REEF.All/Org.Apache.REEF.All.csproj         | 4 ++--
 .../Org.Apache.REEF.Bridge.CLR.csproj                          | 6 +++---
 .../Org.Apache.REEF.Bridge.JAR.csproj                          | 4 ++--
 .../Org.Apache.REEF.Client.Tests.csproj                        | 4 ++--
 lang/cs/Org.Apache.REEF.Client/Org.Apache.REEF.Client.csproj   | 4 ++--
 .../Org.Apache.REEF.Common.Tests.csproj                        | 4 ++--
 lang/cs/Org.Apache.REEF.Common/Org.Apache.REEF.Common.csproj   | 4 ++--
 lang/cs/Org.Apache.REEF.Driver/Org.Apache.REEF.Driver.csproj   | 4 ++--
 .../Org.Apache.REEF.Evaluator.Tests.csproj                     | 4 ++--
 .../Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj | 4 ++--
 .../Org.Apache.REEF.Examples.AllHandlers.csproj                | 4 ++--
 .../Org.Apache.REEF.Examples.DriverRestart.csproj              | 4 ++--
 .../Org.Apache.REEF.Examples.HelloREEF.csproj                  | 4 ++--
 .../Org.Apache.REEF.Examples/Org.Apache.REEF.Examples.csproj   | 4 ++--
 .../Org.Apache.REEF.FatNuGet/Org.Apache.REEF.FatNuGet.csproj   | 4 ++--
 .../Org.Apache.REEF.IMRU.Examples.csproj                       | 4 ++--
 .../Org.Apache.REEF.IMRU.Tests.csproj                          | 4 ++--
 lang/cs/Org.Apache.REEF.IMRU/Org.Apache.REEF.IMRU.csproj       | 4 ++--
 .../Org.Apache.REEF.IO.TestClient.csproj                       | 4 ++--
 .../Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj   | 4 ++--
 lang/cs/Org.Apache.REEF.IO.Tests/packages.config               | 2 +-
 lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj           | 4 ++--
 .../Org.Apache.REEF.Network.Examples.Client.csproj             | 4 ++--
 .../Org.Apache.REEF.Network.Examples.csproj                    | 4 ++--
 .../Org.Apache.REEF.Network.Tests.csproj                       | 2 +-
 lang/cs/Org.Apache.REEF.Network/Org.Apache.REEF.Network.csproj | 2 +-
 .../Org.Apache.REEF.Tang.Examples.csproj                       | 4 ++--
 .../Org.Apache.REEF.Tang.Tests.csproj                          | 4 ++--
 .../Org.Apache.REEF.Tang.Tools.csproj                          | 4 ++--
 lang/cs/Org.Apache.REEF.Tang/Org.Apache.REEF.Tang.csproj       | 4 ++--
 lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj     | 4 ++--
 .../Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj | 4 ++--
 .../Org.Apache.REEF.Wake.Tests.csproj                          | 2 +-
 lang/cs/Org.Apache.REEF.Wake/Org.Apache.REEF.Wake.csproj       | 4 ++--
 35 files changed, 66 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/App.config
----------------------------------------------------------------------
diff --git a/lang/cs/App.config b/lang/cs/App.config
index 8054969..735f2b3 100644
--- a/lang/cs/App.config
+++ b/lang/cs/App.config
@@ -19,7 +19,7 @@ under the License.
 -->
 <configuration>
   <startup>
-    <supportedRuntime version="v4.0" sku=".NETFramework,Version=v4.5.1"/>
+    <supportedRuntime version="v4.0" sku=".NETFramework,Version=v4.5.2" />
   </startup>
   <runtime>
     <assemblyBinding xmlns="urn:schemas-microsoft-com:asm.v1">

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.All/Org.Apache.REEF.All.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.All/Org.Apache.REEF.All.csproj b/lang/cs/Org.Apache.REEF.All/Org.Apache.REEF.All.csproj
index 1a3ea14..51b4ed5 100644
--- a/lang/cs/Org.Apache.REEF.All/Org.Apache.REEF.All.csproj
+++ b/lang/cs/Org.Apache.REEF.All/Org.Apache.REEF.All.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.All</RootNamespace>
     <AssemblyName>Org.Apache.REEF.All</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj b/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
index 20500c3..6cd2bb8 100644
--- a/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Bridge.CLR</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Bridge.CLR</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>
@@ -92,4 +92,4 @@ under the License.
   <Import Project="$(SolutionDir)\AvroCodeGeneration.Targets" Condition="Exists('$(SolutionDir)\AvroCodeGeneration.Targets')" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
   <Import Project="$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets" Condition="Exists('$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets')" />
-</Project>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/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 dd99b14..918955a 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
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -25,7 +25,7 @@ under the License.
     <AssemblyName>Org.Apache.REEF.Bridge.JAR</AssemblyName>
     <RestorePackages>true</RestorePackages>
     <BuildPackage>false</BuildPackage>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
   </PropertyGroup>
   <Import Project="$(SolutionDir)\build.props" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj b/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
index f1bf43e..384e178 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -22,7 +22,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Client.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Client.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <ProjectTypeGuids>{3AC096D0-A1C2-E12C-1390-A8335801FDAB};{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}</ProjectTypeGuids>
     <TestProjectType>UnitTest</TestProjectType>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/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 451597f..6245647 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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Client</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Client</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
@@ -371,4 +371,4 @@ under the License.
   </Target>
   <Target Name="BeforeBuild" DependsOnTargets="$(BeforeBuildDependsOn);RewriteClientResources">
   </Target>
-</Project>
\ No newline at end of file
+</Project>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Common.Tests/Org.Apache.REEF.Common.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common.Tests/Org.Apache.REEF.Common.Tests.csproj b/lang/cs/Org.Apache.REEF.Common.Tests/Org.Apache.REEF.Common.Tests.csproj
index 9233556..a59282b 100644
--- a/lang/cs/Org.Apache.REEF.Common.Tests/Org.Apache.REEF.Common.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Common.Tests/Org.Apache.REEF.Common.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Common.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Common.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/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 8a82948..633bb33 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
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Common</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Common</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/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 1857ada..a5b79f2 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
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Driver</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Driver</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Evaluator.Tests/Org.Apache.REEF.Evaluator.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Evaluator.Tests/Org.Apache.REEF.Evaluator.Tests.csproj b/lang/cs/Org.Apache.REEF.Evaluator.Tests/Org.Apache.REEF.Evaluator.Tests.csproj
index 4fc54ad..5c11de6 100644
--- a/lang/cs/Org.Apache.REEF.Evaluator.Tests/Org.Apache.REEF.Evaluator.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Evaluator.Tests/Org.Apache.REEF.Evaluator.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -22,7 +22,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Evaluator.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Evaluator.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <ProjectTypeGuids>{3AC096D0-A1C2-E12C-1390-A8335801FDAB};{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}</ProjectTypeGuids>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj b/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj
index 6833548..7b567c8 100644
--- a/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj
+++ b/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Evaluator</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Evaluator</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Examples.AllHandlers/Org.Apache.REEF.Examples.AllHandlers.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.AllHandlers/Org.Apache.REEF.Examples.AllHandlers.csproj b/lang/cs/Org.Apache.REEF.Examples.AllHandlers/Org.Apache.REEF.Examples.AllHandlers.csproj
index 1cf74df..9e9d5e0 100644
--- a/lang/cs/Org.Apache.REEF.Examples.AllHandlers/Org.Apache.REEF.Examples.AllHandlers.csproj
+++ b/lang/cs/Org.Apache.REEF.Examples.AllHandlers/Org.Apache.REEF.Examples.AllHandlers.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Examples.AllHandlers</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Examples.AllHandlers</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
   </PropertyGroup>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Examples.DriverRestart/Org.Apache.REEF.Examples.DriverRestart.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.DriverRestart/Org.Apache.REEF.Examples.DriverRestart.csproj b/lang/cs/Org.Apache.REEF.Examples.DriverRestart/Org.Apache.REEF.Examples.DriverRestart.csproj
index d606b04..e73806c 100644
--- a/lang/cs/Org.Apache.REEF.Examples.DriverRestart/Org.Apache.REEF.Examples.DriverRestart.csproj
+++ b/lang/cs/Org.Apache.REEF.Examples.DriverRestart/Org.Apache.REEF.Examples.DriverRestart.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?xml version="1.0" encoding="utf-8"?>
 <Project ToolsVersion="12.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
   <PropertyGroup>
@@ -7,7 +7,7 @@
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Examples.DriverRestart</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Examples.DriverRestart</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
   </PropertyGroup>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj
index 2cce0ee..989cb65 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/Org.Apache.REEF.Examples.HelloREEF.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?xml version="1.0" encoding="utf-8"?>
 <Project ToolsVersion="12.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
   <PropertyGroup>
@@ -7,7 +7,7 @@
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Examples.HelloREEF</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Examples.HelloREEF</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
   </PropertyGroup>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Examples/Org.Apache.REEF.Examples.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples/Org.Apache.REEF.Examples.csproj b/lang/cs/Org.Apache.REEF.Examples/Org.Apache.REEF.Examples.csproj
index 32b6313..a3ee295 100644
--- a/lang/cs/Org.Apache.REEF.Examples/Org.Apache.REEF.Examples.csproj
+++ b/lang/cs/Org.Apache.REEF.Examples/Org.Apache.REEF.Examples.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Examples</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Examples</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.FatNuGet/Org.Apache.REEF.FatNuGet.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.FatNuGet/Org.Apache.REEF.FatNuGet.csproj b/lang/cs/Org.Apache.REEF.FatNuGet/Org.Apache.REEF.FatNuGet.csproj
index 71119ed..8827946 100644
--- a/lang/cs/Org.Apache.REEF.FatNuGet/Org.Apache.REEF.FatNuGet.csproj
+++ b/lang/cs/Org.Apache.REEF.FatNuGet/Org.Apache.REEF.FatNuGet.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.FatNuGet</RootNamespace>
     <AssemblyName>Org.Apache.REEF.FatNuGet</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..\</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.IMRU.Examples/Org.Apache.REEF.IMRU.Examples.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU.Examples/Org.Apache.REEF.IMRU.Examples.csproj b/lang/cs/Org.Apache.REEF.IMRU.Examples/Org.Apache.REEF.IMRU.Examples.csproj
index 782f40d..43d3b24 100644
--- a/lang/cs/Org.Apache.REEF.IMRU.Examples/Org.Apache.REEF.IMRU.Examples.csproj
+++ b/lang/cs/Org.Apache.REEF.IMRU.Examples/Org.Apache.REEF.IMRU.Examples.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.IMRU.Examples</RootNamespace>
     <AssemblyName>Org.Apache.REEF.IMRU.Examples</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <UseVSHostingProcess>false</UseVSHostingProcess>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.IMRU.Tests/Org.Apache.REEF.IMRU.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU.Tests/Org.Apache.REEF.IMRU.Tests.csproj b/lang/cs/Org.Apache.REEF.IMRU.Tests/Org.Apache.REEF.IMRU.Tests.csproj
index d8596d1..bf71ef6 100644
--- a/lang/cs/Org.Apache.REEF.IMRU.Tests/Org.Apache.REEF.IMRU.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.IMRU.Tests/Org.Apache.REEF.IMRU.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.IMRU.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.IMRU.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.IMRU/Org.Apache.REEF.IMRU.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU/Org.Apache.REEF.IMRU.csproj b/lang/cs/Org.Apache.REEF.IMRU/Org.Apache.REEF.IMRU.csproj
index be74b86..99d2665 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/Org.Apache.REEF.IMRU.csproj
+++ b/lang/cs/Org.Apache.REEF.IMRU/Org.Apache.REEF.IMRU.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.IMRU</RootNamespace>
     <AssemblyName>Org.Apache.REEF.IMRU</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
   </PropertyGroup>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.IO.TestClient/Org.Apache.REEF.IO.TestClient.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.TestClient/Org.Apache.REEF.IO.TestClient.csproj b/lang/cs/Org.Apache.REEF.IO.TestClient/Org.Apache.REEF.IO.TestClient.csproj
index d8e4f92..00bccaf 100644
--- a/lang/cs/Org.Apache.REEF.IO.TestClient/Org.Apache.REEF.IO.TestClient.csproj
+++ b/lang/cs/Org.Apache.REEF.IO.TestClient/Org.Apache.REEF.IO.TestClient.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.IO.TestClient</RootNamespace>
     <AssemblyName>Org.Apache.REEF.IO.TestClient</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
   </PropertyGroup>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj b/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
index 58e9bc6..5a7e323 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.IO.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.IO.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
   </PropertyGroup>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/packages.config b/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
index 9b7f977..6cae99e 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj b/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
index 237670d..97a6565 100644
--- a/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
+++ b/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.IO</RootNamespace>
     <AssemblyName>Org.Apache.REEF.IO</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Network.Examples.Client/Org.Apache.REEF.Network.Examples.Client.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network.Examples.Client/Org.Apache.REEF.Network.Examples.Client.csproj b/lang/cs/Org.Apache.REEF.Network.Examples.Client/Org.Apache.REEF.Network.Examples.Client.csproj
index bdae939..700e419 100644
--- a/lang/cs/Org.Apache.REEF.Network.Examples.Client/Org.Apache.REEF.Network.Examples.Client.csproj
+++ b/lang/cs/Org.Apache.REEF.Network.Examples.Client/Org.Apache.REEF.Network.Examples.Client.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Network.Examples.Client</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Network.Examples.Client</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Network.Examples/Org.Apache.REEF.Network.Examples.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network.Examples/Org.Apache.REEF.Network.Examples.csproj b/lang/cs/Org.Apache.REEF.Network.Examples/Org.Apache.REEF.Network.Examples.csproj
index 9a4adb3..d37efed 100644
--- a/lang/cs/Org.Apache.REEF.Network.Examples/Org.Apache.REEF.Network.Examples.csproj
+++ b/lang/cs/Org.Apache.REEF.Network.Examples/Org.Apache.REEF.Network.Examples.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Network.Examples</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Network.Examples</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Network.Tests/Org.Apache.REEF.Network.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network.Tests/Org.Apache.REEF.Network.Tests.csproj b/lang/cs/Org.Apache.REEF.Network.Tests/Org.Apache.REEF.Network.Tests.csproj
index 07f7c6a..84147a0 100644
--- a/lang/cs/Org.Apache.REEF.Network.Tests/Org.Apache.REEF.Network.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Network.Tests/Org.Apache.REEF.Network.Tests.csproj
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Network.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Network.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Network/Org.Apache.REEF.Network.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network/Org.Apache.REEF.Network.csproj b/lang/cs/Org.Apache.REEF.Network/Org.Apache.REEF.Network.csproj
index 30bc685..1e46689 100644
--- a/lang/cs/Org.Apache.REEF.Network/Org.Apache.REEF.Network.csproj
+++ b/lang/cs/Org.Apache.REEF.Network/Org.Apache.REEF.Network.csproj
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Network</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Network</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Tang.Examples/Org.Apache.REEF.Tang.Examples.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang.Examples/Org.Apache.REEF.Tang.Examples.csproj b/lang/cs/Org.Apache.REEF.Tang.Examples/Org.Apache.REEF.Tang.Examples.csproj
index 63f053f..7f425d4 100644
--- a/lang/cs/Org.Apache.REEF.Tang.Examples/Org.Apache.REEF.Tang.Examples.csproj
+++ b/lang/cs/Org.Apache.REEF.Tang.Examples/Org.Apache.REEF.Tang.Examples.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Tang.Examples</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Tang.Examples</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj b/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj
index 4313e69..28b5a1c 100644
--- a/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Tang.Tests/Org.Apache.REEF.Tang.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Tang.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Tang.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Tang.Tools/Org.Apache.REEF.Tang.Tools.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang.Tools/Org.Apache.REEF.Tang.Tools.csproj b/lang/cs/Org.Apache.REEF.Tang.Tools/Org.Apache.REEF.Tang.Tools.csproj
index ed3e2f8..d562047 100644
--- a/lang/cs/Org.Apache.REEF.Tang.Tools/Org.Apache.REEF.Tang.Tools.csproj
+++ b/lang/cs/Org.Apache.REEF.Tang.Tools/Org.Apache.REEF.Tang.Tools.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Tang.Tools</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Tang.Tools</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/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 616cb57..d184ad7 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
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Tang</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Tang</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
index ff71bd6..7436b09 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj b/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj
index 660a280..815030b 100644
--- a/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj
+++ b/lang/cs/Org.Apache.REEF.Utilities/Org.Apache.Reef.Utilities.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Utilities</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Utilities</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Wake.Tests/Org.Apache.REEF.Wake.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake.Tests/Org.Apache.REEF.Wake.Tests.csproj b/lang/cs/Org.Apache.REEF.Wake.Tests/Org.Apache.REEF.Wake.Tests.csproj
index daf065d..628ae91 100644
--- a/lang/cs/Org.Apache.REEF.Wake.Tests/Org.Apache.REEF.Wake.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Wake.Tests/Org.Apache.REEF.Wake.Tests.csproj
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Wake.Tests</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Wake.Tests</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>
     <RestorePackages>true</RestorePackages>

http://git-wip-us.apache.org/repos/asf/reef/blob/e43019f4/lang/cs/Org.Apache.REEF.Wake/Org.Apache.REEF.Wake.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake/Org.Apache.REEF.Wake.csproj b/lang/cs/Org.Apache.REEF.Wake/Org.Apache.REEF.Wake.csproj
index 6e75ff3..ce171bc 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Org.Apache.REEF.Wake.csproj
+++ b/lang/cs/Org.Apache.REEF.Wake/Org.Apache.REEF.Wake.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -23,7 +23,7 @@ under the License.
     <AppDesignerFolder>Properties</AppDesignerFolder>
     <RootNamespace>Org.Apache.REEF.Wake</RootNamespace>
     <AssemblyName>Org.Apache.REEF.Wake</AssemblyName>
-    <TargetFrameworkVersion>v4.5.1</TargetFrameworkVersion>
+    <TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
     <RestorePackages>true</RestorePackages>
     <SolutionDir Condition="$(SolutionDir) == '' Or $(SolutionDir) == '*Undefined*'">..</SolutionDir>


[20/28] reef git commit: [REEF-1963] Jar resource reef-bridge-client missing in reef/global directory

Posted by do...@apache.org.
[REEF-1963] Jar resource reef-bridge-client missing in reef/global directory

This addressed the issue by
  * Manually Adding the file to the driver's reef/global directory in the DriverFolderPreparationHelper

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

Pull request:
  This closes #1419


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

Branch: refs/heads/REEF-335
Commit: 8d5f861ec4071252181f5707f007d71e4e132724
Parents: 4296e20
Author: RoganC <ro...@microsoft.com>
Authored: Fri Nov 17 16:33:38 2017 -0800
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Thu Nov 30 14:03:05 2017 -0800

----------------------------------------------------------------------
 .../Common/DriverFolderPreparationHelper.cs        | 17 ++++++++++++++---
 1 file changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/8d5f861e/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 dfab211..229d9e5 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
@@ -56,6 +56,12 @@ namespace Org.Apache.REEF.Client.Common
         private readonly FileSets _fileSets;
         private readonly ISet<IConfigurationProvider> _driverConfigurationProviders;
 
+        /// <summary>
+        /// The folder in which we search for the client jar.
+        /// In the manner of JavaClientLauncher.cs.
+        /// </summary>
+        private const string JarFolder = "./";
+
         [Inject]
         internal DriverFolderPreparationHelper(
             REEFFileNames fileNames,
@@ -76,7 +82,7 @@ namespace Org.Apache.REEF.Client.Common
         /// <param name="driverFolderPath"></param>
         internal void PrepareDriverFolder(AppParameters appParameters, string driverFolderPath)
         {
-            Logger.Log(Level.Verbose, "Preparing Driver filesystem layout in " + driverFolderPath);
+            Logger.Log(Level.Verbose, "Preparing Driver filesystem layout in {0}", driverFolderPath);
 
             // Setup the folder structure
             CreateDefaultFolderStructure(appParameters, driverFolderPath);
@@ -84,13 +90,18 @@ namespace Org.Apache.REEF.Client.Common
             // Add the appParameters into that folder structure
             _fileSets.AddJobFiles(appParameters);
 
+            // Add the reef-bridge-client jar to the global files in the manner of JavaClientLauncher.cs.
+            _fileSets.AddToLocalFiles(Directory.GetFiles(JarFolder)
+                .Where(file => !string.IsNullOrWhiteSpace(file))
+                .Where(jarFile => Path.GetFileName(jarFile).ToLower().StartsWith(ClientConstants.ClientJarFilePrefix)));
+
             // Create the driver configuration
             CreateDriverConfiguration(appParameters, driverFolderPath);
 
             // Initiate the final copy
             _fileSets.CopyToDriverFolder(driverFolderPath);
 
-            Logger.Log(Level.Info, "Done preparing Driver filesystem layout in " + driverFolderPath);
+            Logger.Log(Level.Info, "Done preparing Driver filesystem layout in {0}", driverFolderPath);
         }
 
         /// <summary>
@@ -147,4 +158,4 @@ namespace Org.Apache.REEF.Client.Common
                 DefaultDriverConfigurationFileContents);
         }
     }
-}
\ No newline at end of file
+}


[09/28] reef git commit: [REEF-1938] Implement network transport for the Java-CLR bridge

Posted by do...@apache.org.
[REEF-1938] Implement network transport for the Java-CLR bridge

This PR is for the Java side of the bridge.

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

Pull Request
  Closes #1405


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

Branch: refs/heads/REEF-335
Commit: bee789a75daf5db55205e75a466de140e770749d
Parents: 94af38d
Author: Sergiy Matusevych <mo...@apache.org>
Authored: Fri Oct 27 16:37:34 2017 -0700
Committer: Doug Service <do...@apache.org>
Committed: Fri Nov 3 00:25:54 2017 +0000

----------------------------------------------------------------------
 .../apache/reef/bridge/NetworkTransport.java    | 139 +++++++++++++++++++
 .../org/apache/reef/bridge/package-info.java    |  22 +++
 2 files changed, 161 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/bee789a7/lang/java/reef-bridge-java/src/main/java/org/apache/reef/bridge/NetworkTransport.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-java/src/main/java/org/apache/reef/bridge/NetworkTransport.java b/lang/java/reef-bridge-java/src/main/java/org/apache/reef/bridge/NetworkTransport.java
new file mode 100644
index 0000000..ab0436c
--- /dev/null
+++ b/lang/java/reef-bridge-java/src/main/java/org/apache/reef/bridge/NetworkTransport.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.bridge;
+
+import org.apache.avro.specific.SpecificRecord;
+import org.apache.reef.tang.InjectionFuture;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.MultiObserver;
+import org.apache.reef.wake.avro.ProtocolSerializer;
+import org.apache.reef.wake.remote.RemoteMessage;
+import org.apache.reef.wake.remote.address.LocalAddressProvider;
+import org.apache.reef.wake.remote.impl.ByteCodec;
+import org.apache.reef.wake.remote.impl.SocketRemoteIdentifier;
+import org.apache.reef.wake.remote.RemoteIdentifier;
+import org.apache.reef.wake.remote.RemoteManager;
+import org.apache.reef.wake.remote.RemoteManagerFactory;
+
+import javax.inject.Inject;
+import java.net.InetSocketAddress;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * The CLR Bridge Network class aggregates a RemoteManager and
+ * Protocol Serializer to provide a simple send/receive interface
+ * between the Java and CLR sides of the bridge.
+ */
+public final class NetworkTransport {
+
+  private static final Logger LOG = Logger.getLogger(NetworkTransport.class.getName());
+
+  /** Remote manager to handle java-C# bridge communication. */
+  private final RemoteManager remoteManager;
+
+  private final ProtocolSerializer serializer;
+  private final InetSocketAddress inetSocketAddress;
+  private final InjectionFuture<MultiObserver> messageObserver;
+
+  private EventHandler<byte[]> sender;
+
+  /**
+   * Sends and receives messages between the java bridge and C# bridge.
+   * @param observer A multiobserver instance that will receive all incoming messages.
+   */
+  @Inject
+  private NetworkTransport(
+      final RemoteManagerFactory remoteManagerFactory,
+      final LocalAddressProvider localAddressProvider,
+      final ProtocolSerializer serializer,
+      final InjectionFuture<MultiObserver> observer) {
+
+    LOG.log(Level.FINE, "Java bridge network initializing");
+
+    this.serializer = serializer;
+    this.messageObserver = observer;
+
+    this.remoteManager = remoteManagerFactory.getInstance(
+        "JavaBridgeNetwork", localAddressProvider.getLocalAddress(), 0, new ByteCodec());
+
+    // Get our address and port number.
+    final RemoteIdentifier remoteIdentifier = this.remoteManager.getMyIdentifier();
+    if (remoteIdentifier instanceof SocketRemoteIdentifier) {
+      final SocketRemoteIdentifier socketIdentifier = (SocketRemoteIdentifier)remoteIdentifier;
+      this.inetSocketAddress = socketIdentifier.getSocketAddress();
+    } else {
+      throw new RuntimeException("Identifier is not a SocketRemoteIdentifier: " + remoteIdentifier);
+    }
+
+    // Register as the message handler for any incoming messages.
+    this.remoteManager.registerHandler(byte[].class, new LocalObserver());
+  }
+
+  /**
+   * Sends a message to the C# side of the bridge.
+   * @param message An Avro message class derived from SpecificRecord.
+   * @throws RuntimeException if invoked before initialization is complete.
+   */
+  public void send(final long identifier, final SpecificRecord message) {
+    if (sender != null) {
+      sender.onNext(serializer.write(message, identifier));
+    } else {
+      final String msgClassName = message.getClass().getCanonicalName();
+      LOG.log(Level.SEVERE, "Attempt to send message [{0}] before network is initialized", msgClassName);
+      throw new RuntimeException("NetworkTransport not initialized: failed to send " + msgClassName);
+    }
+  }
+
+  /**
+   * Provides the IP address and port of the java bridge network.
+   * @return A InetSockerAddress that contains the ip and port of the bridge network.
+   */
+  public InetSocketAddress getAddress() {
+    return inetSocketAddress;
+  }
+
+  /**
+   * Processes messages from the network remote manager.
+   */
+  private final class LocalObserver implements EventHandler<RemoteMessage<byte[]>> {
+
+    /**
+     * Deserialize and direct incoming messages to the registered MuiltiObserver event handler.
+     * @param message A RemoteMessage<byte[]> object which will be deserialized.
+     */
+    @Override
+    public void onNext(final RemoteMessage<byte[]> message) {
+      LOG.log(Level.FINEST, "Received remote message: {0}", message);
+
+      if (sender == null) {
+        // Instantiate a network connection to the C# side of the bridge.
+        // THERE COULD BE  A SECURITY ISSUE HERE WHERE SOMEONE SPOOFS THE
+        // C# BRIDGE, WE RECEIVE IT FIRST, AND CONNECT TO THE SPOOFER,
+        // THOUGH THE TIME WINDOW IS VERY SMALL.
+        final RemoteIdentifier remoteIdentifier = message.getIdentifier();
+        LOG.log(Level.FINE, "Connecting to: {0}", remoteIdentifier);
+        sender = remoteManager.getHandler(remoteIdentifier, byte[].class);
+      }
+
+      // Deserialize the message and invoke the appropriate processing method.
+      serializer.read(message.getMessage(), messageObserver.get());
+    }
+  }
+}

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


[11/28] reef git commit: [REEF-1948] Add IREEFClient.NewJobRequestBuilder

Posted by do...@apache.org.
[REEF-1948] Add IREEFClient.NewJobRequestBuilder

This adds the new API to `IREEFClient`.

To ease implementation and foster re-use, this also adds the new class
`JobRequestBuilderFactory`. It is used on all implementations of
`IREEFClient` to implement the new method.

Lastly, this changes `HelloREEF` to use the new API to demonstrate it to
work.

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

Pull Request:
  This closes #1411


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

Branch: refs/heads/REEF-335
Commit: 7a2aa5cc6e2f6133976c60f00b6146c383c3d1ab
Parents: fece629
Author: Markus Weimer <we...@apache.org>
Authored: Fri Nov 3 09:46:18 2017 -0700
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Fri Nov 3 11:07:24 2017 -0700

----------------------------------------------------------------------
 .../Org.Apache.REEF.Client/API/IREEFClient.cs   |  6 +++
 .../API/JobRequestBuilder.cs                    |  7 +--
 .../API/JobRequestBuilderFactory.cs             | 48 ++++++++++++++++++++
 .../Org.Apache.REEF.Client/Local/LocalClient.cs | 24 +++++++---
 .../Org.Apache.REEF.Client.csproj               |  1 +
 .../YARN/YARNREEFClient.cs                      | 10 +++-
 .../YARN/YarnREEFDotNetClient.cs                | 10 +++-
 .../HelloREEF.cs                                |  6 +--
 8 files changed, 94 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/7a2aa5cc/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 a4734f5..bc04186 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/IREEFClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/IREEFClient.cs
@@ -35,6 +35,12 @@ namespace Org.Apache.REEF.Client.API
         void Submit(JobRequest jobRequest);
 
         /// <summary>
+        /// Instantiate a new JobRequestBuilder.
+        /// </summary>
+        /// <returns>A new JobRequestBuilder</returns>
+        JobRequestBuilder NewJobRequestBuilder();
+
+        /// <summary>
         /// Submit the job described in jobRequest to the cluster.
         /// Expect IJobSubmissionResult returned after the call.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/reef/blob/7a2aa5cc/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs b/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs
index 234cd7a..7760b7f 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilder.cs
@@ -34,16 +34,11 @@ namespace Org.Apache.REEF.Client.API
         }
 
         [Inject]
-        private JobRequestBuilder([Parameter(typeof(DriverConfigurationProviders))] ISet<IConfigurationProvider> configurationProviders)
+        internal JobRequestBuilder([Parameter(typeof(DriverConfigurationProviders))] ISet<IConfigurationProvider> configurationProviders)
         {
             AddDriverConfigurationProviders(configurationProviders);
         }
 
-        public static JobRequestBuilder NewBuilder()
-        {
-            return new JobRequestBuilder();
-        }
-
         /// <summary>
         /// Bake the information provided so far and return a IJobSubmission 
         /// </summary>

http://git-wip-us.apache.org/repos/asf/reef/blob/7a2aa5cc/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilderFactory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilderFactory.cs b/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilderFactory.cs
new file mode 100644
index 0000000..6986c34
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/JobRequestBuilderFactory.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 System.Collections.Generic;
+using Org.Apache.REEF.Common.Client.Parameters;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
+
+namespace Org.Apache.REEF.Client.API
+{
+    /// <summary>
+    /// Helper class to create instances of JobRequestBuilder.
+    /// </summary>
+    internal sealed class JobRequestBuilderFactory
+    {
+        private readonly ISet<IConfigurationProvider> _configurationProviders;
+
+        [Inject]
+        private JobRequestBuilderFactory(
+            [Parameter(typeof(DriverConfigurationProviders))] ISet<IConfigurationProvider> configurationProviders)
+        {
+            _configurationProviders = configurationProviders;
+        }
+
+        /// <summary>
+        /// Instantiates a new JobRequestBuilder.
+        /// </summary>
+        /// <returns>JobRequestBuilder</returns>
+        public JobRequestBuilder NewInstance()
+        {
+            return new JobRequestBuilder(_configurationProviders);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/7a2aa5cc/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 0f1ef1d..722bf9b 100644
--- a/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
@@ -31,7 +31,6 @@ using Org.Apache.REEF.Common.Files;
 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.Utilities.AsyncUtils;
 using Org.Apache.REEF.Utilities.Attributes;
 using Org.Apache.REEF.Utilities.Logging;
@@ -63,20 +62,25 @@ namespace Org.Apache.REEF.Client.Local
         private readonly string _runtimeFolder;
         private readonly REEFFileNames _fileNames;
         private readonly IConfiguration _localConfigurationOnDriver;
+        private readonly JobRequestBuilderFactory _jobRequestBuilderFactory;
 
         [Inject]
         private LocalClient(DriverFolderPreparationHelper driverFolderPreparationHelper,
             [Parameter(typeof(LocalRuntimeDirectory))] string runtimeFolder,
             [Parameter(typeof(NumberOfEvaluators))] int maxNumberOfConcurrentEvaluators,
             IJavaClientLauncher javaClientLauncher,
-            REEFFileNames fileNames)
+            REEFFileNames fileNames,
+            JobRequestBuilderFactory jobRequestBuilderFactory)
         {
             _driverFolderPreparationHelper = driverFolderPreparationHelper;
             _runtimeFolder = runtimeFolder;
             _maxNumberOfConcurrentEvaluators = maxNumberOfConcurrentEvaluators;
             _javaClientLauncher = javaClientLauncher;
             _fileNames = fileNames;
-            _localConfigurationOnDriver = TangFactory.GetTang().NewConfigurationBuilder().BindImplementation(GenericType<ILocalAddressProvider>.Class, GenericType<LoopbackLocalAddressProvider>.Class).Build();
+            _jobRequestBuilderFactory = jobRequestBuilderFactory;
+            _localConfigurationOnDriver = TangFactory.GetTang().NewConfigurationBuilder()
+                .BindImplementation<ILocalAddressProvider, LoopbackLocalAddressProvider>()
+                .Build();
         }
 
         /// <summary>
@@ -86,13 +90,16 @@ namespace Org.Apache.REEF.Client.Local
         /// <param name="numberOfEvaluators"></param>
         /// <param name="javaClientLauncher"></param>
         /// <param name="fileNames"></param>
+        /// <param name="jobRequestBuilderFactory">The helper used to instantiate JobRequestBuilder instances.</param>
         [Inject]
         private LocalClient(
             DriverFolderPreparationHelper driverFolderPreparationHelper,
             [Parameter(typeof(NumberOfEvaluators))] int numberOfEvaluators,
             IJavaClientLauncher javaClientLauncher,
-            REEFFileNames fileNames)
-            : this(driverFolderPreparationHelper, Path.GetTempPath(), numberOfEvaluators, javaClientLauncher, fileNames)
+            REEFFileNames fileNames,
+            JobRequestBuilderFactory jobRequestBuilderFactory)
+            : this(driverFolderPreparationHelper, Path.GetTempPath(),
+                numberOfEvaluators, javaClientLauncher, fileNames, jobRequestBuilderFactory)
         {
             // Intentionally left blank.
         }
@@ -171,6 +178,11 @@ namespace Org.Apache.REEF.Client.Local
             Logger.Log(Level.Info, "Submitted the Driver for execution.");
         }
 
+        public JobRequestBuilder NewJobRequestBuilder()
+        {
+            return _jobRequestBuilderFactory.NewInstance();
+        }
+
         public IJobSubmissionResult SubmitAndGetJobStatus(JobRequest jobRequest)
         {
             var driverFolder = PrepareDriverFolder(jobRequest);
@@ -211,4 +223,4 @@ namespace Org.Apache.REEF.Client.Local
             return Path.Combine(_runtimeFolder, string.Join("-", "reef", jobId, timestamp));
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/7a2aa5cc/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 00c6d87..80b8d90 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
@@ -73,6 +73,7 @@ under the License.
     <Compile Include="API\JobParametersBuilder.cs" />
     <Compile Include="API\JobRequest.cs" />
     <Compile Include="API\JobRequestBuilder.cs" />
+    <Compile Include="API\JobRequestBuilderFactory.cs" />
     <Compile Include="API\TcpPortConfigurationModule.cs" />
     <Compile Include="Avro\AvroAppSubmissionParameters.cs" />
     <Compile Include="Avro\AvroJobSubmissionParameters.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/7a2aa5cc/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
index b4e5ad7..09434f0 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
@@ -47,6 +47,7 @@ namespace Org.Apache.REEF.Client.Yarn
         private readonly REEFFileNames _fileNames;
         private readonly IYarnRMClient _yarnClient;
         private readonly YarnREEFParamSerializer _paramSerializer;
+        private readonly JobRequestBuilderFactory _jobRequestBuilderFactory;
 
         [Inject]
         internal YarnREEFClient(IJavaClientLauncher javaClientLauncher,
@@ -54,7 +55,8 @@ namespace Org.Apache.REEF.Client.Yarn
             REEFFileNames fileNames,
             YarnCommandLineEnvironment yarn,
             IYarnRMClient yarnClient,
-            YarnREEFParamSerializer paramSerializer)
+            YarnREEFParamSerializer paramSerializer,
+            JobRequestBuilderFactory jobRequestBuilderFactory)
         {
             _javaClientLauncher = javaClientLauncher;
             _javaClientLauncher.AddToClassPath(yarn.GetYarnClasspathList());
@@ -62,6 +64,7 @@ namespace Org.Apache.REEF.Client.Yarn
             _fileNames = fileNames;
             _yarnClient = yarnClient;
             _paramSerializer = paramSerializer;
+            _jobRequestBuilderFactory = jobRequestBuilderFactory;
         }
 
         public void Submit(JobRequest jobRequest)
@@ -73,6 +76,11 @@ namespace Org.Apache.REEF.Client.Yarn
             Launch(jobRequest, driverFolderPath);
         }
 
+        public JobRequestBuilder NewJobRequestBuilder()
+        {
+            return _jobRequestBuilderFactory.NewInstance();
+        }
+
         public IJobSubmissionResult SubmitAndGetJobStatus(JobRequest jobRequest)
         {
             // Prepare the job submission folder

http://git-wip-us.apache.org/repos/asf/reef/blob/7a2aa5cc/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs
index babb613..65c6c87 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YarnREEFDotNetClient.cs
@@ -51,6 +51,7 @@ namespace Org.Apache.REEF.Client.YARN
         private readonly REEFFileNames _fileNames;
         private readonly IJobSubmissionDirectoryProvider _jobSubmissionDirectoryProvider;
         private readonly YarnREEFDotNetParamSerializer _paramSerializer;
+        private readonly JobRequestBuilderFactory _jobRequestBuilderFactory;
 
         [Inject]
         private YarnREEFDotNetClient(
@@ -60,7 +61,8 @@ namespace Org.Apache.REEF.Client.YARN
             IJobResourceUploader jobResourceUploader,
             REEFFileNames fileNames,
             IJobSubmissionDirectoryProvider jobSubmissionDirectoryProvider,
-            YarnREEFDotNetParamSerializer paramSerializer)
+            YarnREEFDotNetParamSerializer paramSerializer,
+            JobRequestBuilderFactory jobRequestBuilderFactory)
         {
             _injector = injector;
             _jobSubmissionDirectoryProvider = jobSubmissionDirectoryProvider;
@@ -69,6 +71,7 @@ namespace Org.Apache.REEF.Client.YARN
             _driverFolderPreparationHelper = driverFolderPreparationHelper;
             _yarnRMClient = yarnRMClient;
             _paramSerializer = paramSerializer;
+            _jobRequestBuilderFactory = jobRequestBuilderFactory;
         }
 
         public void Submit(JobRequest jobRequest)
@@ -135,6 +138,11 @@ namespace Org.Apache.REEF.Client.YARN
             }
         }
 
+        public JobRequestBuilder NewJobRequestBuilder()
+        {
+            return _jobRequestBuilderFactory.NewInstance();
+        }
+
         public IJobSubmissionResult SubmitAndGetJobStatus(JobRequest jobRequest)
         {
             throw new NotSupportedException();

http://git-wip-us.apache.org/repos/asf/reef/blob/7a2aa5cc/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 b1d0cd3..e29ea0c 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
@@ -41,13 +41,11 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         private const string YARNRest = "yarnrest";
         private const string HDInsight = "hdi";
         private readonly IREEFClient _reefClient;
-        private readonly JobRequestBuilder _jobRequestBuilder;
 
         [Inject]
-        private HelloREEF(IREEFClient reefClient, JobRequestBuilder jobRequestBuilder)
+        private HelloREEF(IREEFClient reefClient)
         {
             _reefClient = reefClient;
-            _jobRequestBuilder = jobRequestBuilder;
         }
 
         /// <summary>
@@ -62,7 +60,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
                 .Build();
 
             // The JobSubmission contains the Driver configuration as well as the files needed on the Driver.
-            var helloJobRequest = _jobRequestBuilder
+            var helloJobRequest = _reefClient.NewJobRequestBuilder()
                 .AddDriverConfiguration(helloDriverConfiguration)
                 .AddGlobalAssemblyForType(typeof(HelloDriver))
                 .SetJobIdentifier("HelloREEF")


[23/28] reef git commit: [REEF-1969] Switch to StyleCop 5.0

Posted by do...@apache.org.
[REEF-1969] Switch to StyleCop 5.0

This updates all package references to use version `5.0.0` of StyleCop.
Also, it fixes newly introduced StyleCop warnings. All of those are just
inverting comparisons of the form `constant == variable` to `variable ==
constant`.

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

Pull Request:
  This closes #1423


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

Branch: refs/heads/REEF-335
Commit: 227ce1d4b92b70d5f142d05f37c1ce34ac8c3985
Parents: b0b9cb1
Author: Markus Weimer <we...@apache.org>
Authored: Wed Jan 3 16:17:58 2018 -0800
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Thu Jan 4 18:16:36 2018 -0800

----------------------------------------------------------------------
 lang/cs/Org.Apache.REEF.Bridge.CLR/packages.config      |  2 +-
 lang/cs/Org.Apache.REEF.Client.Tests/packages.config    |  2 +-
 .../Common/JobSubmissionResult.cs                       |  2 +-
 .../Local/TestRunner/LocalTestRunner.cs                 |  2 +-
 lang/cs/Org.Apache.REEF.Client/packages.config          |  2 +-
 lang/cs/Org.Apache.REEF.Common.Tests/packages.config    |  2 +-
 lang/cs/Org.Apache.REEF.Common/Files/PathUtilities.cs   |  6 +-----
 lang/cs/Org.Apache.REEF.Common/Jar/ResourceHelper.cs    |  6 +++---
 lang/cs/Org.Apache.REEF.Common/packages.config          |  2 +-
 lang/cs/Org.Apache.REEF.Driver/packages.config          |  2 +-
 lang/cs/Org.Apache.REEF.Evaluator.Tests/packages.config |  2 +-
 lang/cs/Org.Apache.REEF.Evaluator/packages.config       |  2 +-
 .../packages.config                                     |  2 +-
 .../packages.config                                     |  2 +-
 .../Org.Apache.REEF.Examples.HelloREEF/packages.config  |  2 +-
 lang/cs/Org.Apache.REEF.Examples/packages.config        |  2 +-
 lang/cs/Org.Apache.REEF.IMRU.Examples/packages.config   |  2 +-
 lang/cs/Org.Apache.REEF.IMRU.Tests/packages.config      |  2 +-
 .../API/IMRUJobDefinitionBuilder.cs                     | 12 ++++++------
 lang/cs/Org.Apache.REEF.IMRU/packages.config            |  2 +-
 lang/cs/Org.Apache.REEF.IO.Tests/packages.config        |  2 +-
 .../FileSystem/Hadoop/HDFSCommandRunner.cs              |  4 ++--
 lang/cs/Org.Apache.REEF.IO/packages.config              |  2 +-
 .../packages.config                                     |  2 +-
 .../cs/Org.Apache.REEF.Network.Examples/packages.config |  2 +-
 lang/cs/Org.Apache.REEF.Network.Tests/packages.config   |  2 +-
 lang/cs/Org.Apache.REEF.Network/packages.config         |  2 +-
 lang/cs/Org.Apache.REEF.Tang.Examples/packages.config   |  2 +-
 lang/cs/Org.Apache.REEF.Tang.Tests/packages.config      |  2 +-
 lang/cs/Org.Apache.REEF.Tang.Tools/packages.config      |  2 +-
 .../ClassHierarchy/ClassHierarchyImpl.cs                |  8 ++++----
 .../Implementations/ClassHierarchy/NodeFactory.cs       |  6 +++---
 .../Implementations/InjectionPlan/InjectorImpl.cs       |  4 ++--
 .../cs/Org.Apache.REEF.Tang/Util/ReflectionUtilities.cs |  2 +-
 lang/cs/Org.Apache.REEF.Tang/packages.config            |  2 +-
 .../Functional/Bridge/HelloSimpleEventHandlers.cs       |  2 +-
 .../Functional/ReefFunctionalTest.cs                    |  2 +-
 lang/cs/Org.Apache.REEF.Tests/packages.config           |  2 +-
 lang/cs/Org.Apache.REEF.Utilities/packages.config       |  2 +-
 lang/cs/Org.Apache.REEF.Wake.Tests/packages.config      |  2 +-
 lang/cs/Org.Apache.REEF.Wake/Examples/P2p/Pull2Push.cs  |  4 ++--
 lang/cs/Org.Apache.REEF.Wake/packages.config            |  2 +-
 lang/cs/build.DotNet.props                              |  2 +-
 lang/cs/build.props                                     |  2 +-
 44 files changed, 59 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Bridge.CLR/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/packages.config b/lang/cs/Org.Apache.REEF.Bridge.CLR/packages.config
index 9846a87..7e886bd 100644
--- a/lang/cs/Org.Apache.REEF.Bridge.CLR/packages.config
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/packages.config
@@ -20,5 +20,5 @@ under the License.
 <packages>
   <package id="Microsoft.Hadoop.Avro" version="1.5.6" targetFramework="net45" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Client.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/packages.config b/lang/cs/Org.Apache.REEF.Client.Tests/packages.config
index 123da0d..347c9f0 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/packages.config
@@ -20,7 +20,7 @@ under the License.
 <packages>
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
   <package id="NSubstitute" version="1.8.2.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />
   <package id="xunit.abstractions" version="2.0.0" targetFramework="net45" />
   <package id="xunit.assert" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
index 0649422..e1967e2 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
@@ -364,7 +364,7 @@ namespace Org.Apache.REEF.Client.Common
                     _driverUrl = values[TrackingUrlKey].ToString();
                     LOGGER.Log(Level.Info, "trackingUrl[" + _driverUrl + "]");
 
-                    if (0 == string.Compare(_driverUrl, UnAssigned))
+                    if (string.Compare(_driverUrl, UnAssigned) == 0)
                     {
                         resultKind = UrlResultKind.UrlNotAssignedYet;
                     }

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs
index 653917d..a282364 100644
--- a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs
@@ -61,7 +61,7 @@ namespace Org.Apache.REEF.Client.Local.TestRunner
                 assertFileName);
             IJobSubmissionResult jobStatus = _client.SubmitAndGetJobStatus(jobRequest);
 
-            if (null == jobStatus)
+            if (jobStatus == null)
             {
                 return TestResult.Fail(
                     "JobStatus returned by the Client was null. This points to an environment setup problem.");

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/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
index b15b7fd..d6832d2 100644
--- a/lang/cs/Org.Apache.REEF.Client/packages.config
+++ b/lang/cs/Org.Apache.REEF.Client/packages.config
@@ -19,6 +19,6 @@ under the License.
 -->
 <packages>
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="TransientFaultHandling.Core" version="5.1.1209.1" targetFramework="net45" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Common.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common.Tests/packages.config b/lang/cs/Org.Apache.REEF.Common.Tests/packages.config
index df4e5ea..3b963f3 100644
--- a/lang/cs/Org.Apache.REEF.Common.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.Common.Tests/packages.config
@@ -18,7 +18,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />
   <package id="xunit.abstractions" version="2.0.0" targetFramework="net45" />
   <package id="xunit.assert" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Common/Files/PathUtilities.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Files/PathUtilities.cs b/lang/cs/Org.Apache.REEF.Common/Files/PathUtilities.cs
index ecd1124..1c73d66 100644
--- a/lang/cs/Org.Apache.REEF.Common/Files/PathUtilities.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Files/PathUtilities.cs
@@ -45,11 +45,7 @@ namespace Org.Apache.REEF.Common.Files
         /// <returns></returns>
         public static bool AreNormalizedEquals(string path1, string path2)
         {
-            if (null == path1 || null == path2)
-            {
-                return false;
-            }
-            return NormalizePath(path1).Equals(NormalizePath(path2));
+            return path1 != null && path2 != null && NormalizePath(path1).Equals(NormalizePath(path2));
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Common/Jar/ResourceHelper.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Jar/ResourceHelper.cs b/lang/cs/Org.Apache.REEF.Common/Jar/ResourceHelper.cs
index 555c91d..61da7d9 100644
--- a/lang/cs/Org.Apache.REEF.Common/Jar/ResourceHelper.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Jar/ResourceHelper.cs
@@ -60,12 +60,12 @@ namespace Org.Apache.REEF.Common.Jar
         public ResourceHelper(Assembly assembly)
         {
             var names = assembly.GetManifestResourceNames();
-            if (names == null || names.Length == 0 || null == names[0])
+            if (names == null || names.Length == 0 || names[0] == null)
             {
                 throw new ApplicationException("Could not retrieve Assembly Manifest Resource names");
             }
             var manifestResources = assembly.GetManifestResourceStream(names[0]);
-            if (null == manifestResources)
+            if (manifestResources == null)
             {
                 throw new ApplicationException("Could not retrieve Assembly Manifest Resource stream");
             }
@@ -81,7 +81,7 @@ namespace Org.Apache.REEF.Common.Jar
         internal T GetResource<T>(string resourceName)
         {
             var resource = _resourceSet.GetObject(resourceName);
-            if (null == resource)
+            if (resource == null)
             {
                 throw new ApplicationException(string.Format(CouldNotRetrieveResource, resourceName));
             }

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Common/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/packages.config b/lang/cs/Org.Apache.REEF.Common/packages.config
index a69e8bd..c0664c3 100644
--- a/lang/cs/Org.Apache.REEF.Common/packages.config
+++ b/lang/cs/Org.Apache.REEF.Common/packages.config
@@ -21,7 +21,7 @@ under the License.
   <package id="Microsoft.Hadoop.Avro" version="1.5.6" targetFramework="net45" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="System.Reactive.Core" version="3.1.1" targetFramework="net451" />
   <package id="System.Reactive.Interfaces" version="3.1.1" targetFramework="net451" />
 </packages>

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Driver/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Driver/packages.config b/lang/cs/Org.Apache.REEF.Driver/packages.config
index 57e83c0..9d07038 100644
--- a/lang/cs/Org.Apache.REEF.Driver/packages.config
+++ b/lang/cs/Org.Apache.REEF.Driver/packages.config
@@ -21,5 +21,5 @@ under the License.
   <package id="Microsoft.Hadoop.Avro" version="1.5.6" targetFramework="net45" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Evaluator.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Evaluator.Tests/packages.config b/lang/cs/Org.Apache.REEF.Evaluator.Tests/packages.config
index 8e15bbc..091af92 100644
--- a/lang/cs/Org.Apache.REEF.Evaluator.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.Evaluator.Tests/packages.config
@@ -20,7 +20,7 @@ under the License.
 <packages>
   <package id="NSubstitute" version="1.8.2.0" targetFramework="net45" />
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />
   <package id="xunit.abstractions" version="2.0.0" targetFramework="net45" />
   <package id="xunit.assert" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Evaluator/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Evaluator/packages.config b/lang/cs/Org.Apache.REEF.Evaluator/packages.config
index a0f4669..f64c5a2 100644
--- a/lang/cs/Org.Apache.REEF.Evaluator/packages.config
+++ b/lang/cs/Org.Apache.REEF.Evaluator/packages.config
@@ -19,5 +19,5 @@ under the License.
 -->
 <packages>
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Examples.AllHandlers/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.AllHandlers/packages.config b/lang/cs/Org.Apache.REEF.Examples.AllHandlers/packages.config
index d952982..f03af4e 100644
--- a/lang/cs/Org.Apache.REEF.Examples.AllHandlers/packages.config
+++ b/lang/cs/Org.Apache.REEF.Examples.AllHandlers/packages.config
@@ -18,5 +18,5 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Examples.DriverRestart/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.DriverRestart/packages.config b/lang/cs/Org.Apache.REEF.Examples.DriverRestart/packages.config
index d952982..f03af4e 100644
--- a/lang/cs/Org.Apache.REEF.Examples.DriverRestart/packages.config
+++ b/lang/cs/Org.Apache.REEF.Examples.DriverRestart/packages.config
@@ -18,5 +18,5 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config
index 13ca20f..17b9f77 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/packages.config
@@ -18,6 +18,6 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Examples/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples/packages.config b/lang/cs/Org.Apache.REEF.Examples/packages.config
index 4c586dd..f627d0b 100644
--- a/lang/cs/Org.Apache.REEF.Examples/packages.config
+++ b/lang/cs/Org.Apache.REEF.Examples/packages.config
@@ -20,5 +20,5 @@ under the License.
 <packages>
   <package id="Microsoft.Hadoop.Avro" version="1.5.6" targetFramework="net45" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.IMRU.Examples/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU.Examples/packages.config b/lang/cs/Org.Apache.REEF.IMRU.Examples/packages.config
index d952982..f03af4e 100644
--- a/lang/cs/Org.Apache.REEF.IMRU.Examples/packages.config
+++ b/lang/cs/Org.Apache.REEF.IMRU.Examples/packages.config
@@ -18,5 +18,5 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.IMRU.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU.Tests/packages.config b/lang/cs/Org.Apache.REEF.IMRU.Tests/packages.config
index a57dabf..6cae99e 100644
--- a/lang/cs/Org.Apache.REEF.IMRU.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.IMRU.Tests/packages.config
@@ -19,7 +19,7 @@ under the License.
 -->
 <packages>
   <package id="NSubstitute" version="1.8.2.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />
   <package id="xunit.abstractions" version="2.0.0" targetFramework="net45" />
   <package id="xunit.assert" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.IMRU/API/IMRUJobDefinitionBuilder.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU/API/IMRUJobDefinitionBuilder.cs b/lang/cs/Org.Apache.REEF.IMRU/API/IMRUJobDefinitionBuilder.cs
index e63bb69..f3fe89c 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/API/IMRUJobDefinitionBuilder.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU/API/IMRUJobDefinitionBuilder.cs
@@ -324,34 +324,34 @@ namespace Org.Apache.REEF.IMRU.API
         /// <exception cref="NullReferenceException">If any of the required parameters is not set.</exception>
         public IMRUJobDefinition Build()
         {
-            if (null == _jobName)
+            if (_jobName == null)
             {
                 Exceptions.Throw(new NullReferenceException("Job name cannot be null"),
                     Logger);
             }
 
-            if (null == _mapFunctionConfiguration)
+            if (_mapFunctionConfiguration == null)
             {
                 Exceptions.Throw(new NullReferenceException("Map function configuration cannot be null"), Logger);
             }
 
-            if (null == _mapInputCodecConfiguration)
+            if (_mapInputCodecConfiguration == null)
             {
                 Exceptions.Throw(new NullReferenceException("Map input codec configuration cannot be null"), Logger);
             }
 
-            if (null == _updateFunctionCodecsConfiguration)
+            if (_updateFunctionCodecsConfiguration == null)
             {
                 Exceptions.Throw(new NullReferenceException("Update function codecs configuration cannot be null"),
                     Logger);
             }
 
-            if (null == _reduceFunctionConfiguration)
+            if (_reduceFunctionConfiguration == null)
             {
                 Exceptions.Throw(new NullReferenceException("Reduce function configuration cannot be null"), Logger);
             }
 
-            if (null == _updateFunctionConfiguration)
+            if (_updateFunctionConfiguration == null)
             {
                 Exceptions.Throw(new NullReferenceException("Update function configuration cannot be null"), Logger);
             }

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.IMRU/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU/packages.config b/lang/cs/Org.Apache.REEF.IMRU/packages.config
index d952982..f03af4e 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/packages.config
+++ b/lang/cs/Org.Apache.REEF.IMRU/packages.config
@@ -18,5 +18,5 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/packages.config b/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
index 4a71ef7..9b7f977 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
@@ -19,7 +19,7 @@ under the License.
 -->
 <packages>
   <package id="NSubstitute" version="1.8.2.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />
   <package id="xunit.abstractions" version="2.0.0" targetFramework="net45" />
   <package id="xunit.assert" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HDFSCommandRunner.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HDFSCommandRunner.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HDFSCommandRunner.cs
index 3939218..04eff7c 100644
--- a/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HDFSCommandRunner.cs
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/HDFSCommandRunner.cs
@@ -122,7 +122,7 @@ namespace Org.Apache.REEF.IO.FileSystem.Hadoop
             {
                 var processName = string.Format("HDFS_Attempt_{0}_of_{1}", attemptNumber, _numberOfRetries);
                 var result = RunAttempt(processStartInfo, _timeOutInMilliSeconds, processName);
-                if (null != result)
+                if (result != null)
                 {
                     LogCommandOutput(result);
                     return result;
@@ -153,7 +153,7 @@ namespace Org.Apache.REEF.IO.FileSystem.Hadoop
         {
             var hadoopHomeFromEnv = Environment.GetEnvironmentVariable(HadoopHomeEnvironmentVariableName);
             Logger.Log(Level.Verbose, "{0} evaluated to {1}.", HadoopHomeEnvironmentVariableName, hadoopHomeFromEnv);
-            if (null == hadoopHomeFromEnv)
+            if (hadoopHomeFromEnv == null)
             {
                 throw new Exception(HadoopHomeEnvironmentVariableName +
                                     " not set and no path to the hadoop installation provided.");

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.IO/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/packages.config b/lang/cs/Org.Apache.REEF.IO/packages.config
index e6783e0..4e1a691 100644
--- a/lang/cs/Org.Apache.REEF.IO/packages.config
+++ b/lang/cs/Org.Apache.REEF.IO/packages.config
@@ -23,7 +23,7 @@ under the License.
   <package id="Microsoft.Data.OData" version="5.6.4" targetFramework="net45" />
   <package id="Microsoft.Data.Services.Client" version="5.6.4" targetFramework="net45" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="System.Spatial" version="5.6.4" targetFramework="net45" />
   <package id="WindowsAzure.Storage" version="6.1.0" targetFramework="net45" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Network.Examples.Client/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network.Examples.Client/packages.config b/lang/cs/Org.Apache.REEF.Network.Examples.Client/packages.config
index d952982..f03af4e 100644
--- a/lang/cs/Org.Apache.REEF.Network.Examples.Client/packages.config
+++ b/lang/cs/Org.Apache.REEF.Network.Examples.Client/packages.config
@@ -18,5 +18,5 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Network.Examples/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network.Examples/packages.config b/lang/cs/Org.Apache.REEF.Network.Examples/packages.config
index d952982..f03af4e 100644
--- a/lang/cs/Org.Apache.REEF.Network.Examples/packages.config
+++ b/lang/cs/Org.Apache.REEF.Network.Examples/packages.config
@@ -18,5 +18,5 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Network.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network.Tests/packages.config b/lang/cs/Org.Apache.REEF.Network.Tests/packages.config
index 581efe6..dd784dc 100644
--- a/lang/cs/Org.Apache.REEF.Network.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.Network.Tests/packages.config
@@ -18,7 +18,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="System.Reactive.Core" version="3.1.1" targetFramework="net451" />
   <package id="System.Reactive.Interfaces" version="3.1.1" targetFramework="net451" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Network/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network/packages.config b/lang/cs/Org.Apache.REEF.Network/packages.config
index 6b92382..a6425aa 100644
--- a/lang/cs/Org.Apache.REEF.Network/packages.config
+++ b/lang/cs/Org.Apache.REEF.Network/packages.config
@@ -28,7 +28,7 @@ under the License.
   <package id="NETStandard.Library" version="1.6.1" targetFramework="net451" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="System.Collections" version="4.3.0" targetFramework="net451" />
   <package id="System.Collections.Concurrent" version="4.3.0" targetFramework="net451" />
   <package id="System.ComponentModel" version="4.3.0" targetFramework="net451" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tang.Examples/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang.Examples/packages.config b/lang/cs/Org.Apache.REEF.Tang.Examples/packages.config
index d952982..f03af4e 100644
--- a/lang/cs/Org.Apache.REEF.Tang.Examples/packages.config
+++ b/lang/cs/Org.Apache.REEF.Tang.Examples/packages.config
@@ -18,5 +18,5 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tang.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang.Tests/packages.config b/lang/cs/Org.Apache.REEF.Tang.Tests/packages.config
index 4b168fb..a8bc1be 100644
--- a/lang/cs/Org.Apache.REEF.Tang.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.Tang.Tests/packages.config
@@ -21,7 +21,7 @@ under the License.
   <package id="Microsoft.Hadoop.Avro" version="1.5.6" targetFramework="net45" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />
   <package id="xunit.abstractions" version="2.0.0" targetFramework="net45" />
   <package id="xunit.assert" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tang.Tools/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang.Tools/packages.config b/lang/cs/Org.Apache.REEF.Tang.Tools/packages.config
index d952982..f03af4e 100644
--- a/lang/cs/Org.Apache.REEF.Tang.Tools/packages.config
+++ b/lang/cs/Org.Apache.REEF.Tang.Tools/packages.config
@@ -18,5 +18,5 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/ClassHierarchyImpl.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/ClassHierarchyImpl.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/ClassHierarchyImpl.cs
index 6db832b..5562963 100644
--- a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/ClassHierarchyImpl.cs
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/ClassHierarchyImpl.cs
@@ -285,12 +285,12 @@ namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy
 
             if (!string.IsNullOrEmpty(np.GetAlias()))
             {
-                IDictionary<string, string> mapping = null;
-                _aliasLookupTable.TryGetValue(np.GetAliasLanguage().ToString(), out mapping);
-                if (null == mapping)
+                var aliasLanguage = np.GetAliasLanguage().ToString();
+                IDictionary<string, string> mapping;
+                if (!_aliasLookupTable.TryGetValue(aliasLanguage, out mapping))
                 {
                     mapping = new Dictionary<string, string>();
-                    _aliasLookupTable.Add(np.GetAliasLanguage().ToString(), mapping);
+                    _aliasLookupTable.Add(aliasLanguage, mapping);
                 }
                 try
                 {

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/NodeFactory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/NodeFactory.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/NodeFactory.cs
index 3a96808..9556d82 100644
--- a/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/NodeFactory.cs
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/ClassHierarchy/NodeFactory.cs
@@ -40,7 +40,7 @@ namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy
         public static INode CreateClassNode(INode parent, Type clazz)
         {
             // var namedParameter = clazz.GetCustomAttribute<NamedParameterAttribute>();
-            var unit = null != clazz.GetCustomAttribute<UnitAttribute>();
+            var unit = clazz.GetCustomAttribute<UnitAttribute>() != null;
             string simpleName = ReflectionUtilities.GetName(clazz);
             string fullName = ReflectionUtilities.GetAssemblyQualifiedName(clazz);
 
@@ -64,7 +64,7 @@ namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy
 
             foreach (ConstructorInfo c in clazz.GetConstructors(BindingFlags.Public | BindingFlags.NonPublic | BindingFlags.Instance))  
             {
-                var constructorAnnotatedInjectable = null != c.GetCustomAttribute<InjectAttribute>();
+                var constructorAnnotatedInjectable = c.GetCustomAttribute<InjectAttribute>() != null;
 
                 bool constructorInjectable = constructorAnnotatedInjectable;
 
@@ -91,7 +91,7 @@ namespace Org.Apache.REEF.Tang.Implementations.ClassHierarchy
 
             string defaultImplementation = null;
             DefaultImplementationAttribute defaultImpl = clazz.GetCustomAttribute<DefaultImplementationAttribute>();
-            if (null != defaultImpl)
+            if (defaultImpl != null)
             {
                 Type defaultImplementationClazz = defaultImpl.Value;
 

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tang/Implementations/InjectionPlan/InjectorImpl.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Implementations/InjectionPlan/InjectorImpl.cs b/lang/cs/Org.Apache.REEF.Tang/Implementations/InjectionPlan/InjectorImpl.cs
index 20ca4e8..ac843f8 100644
--- a/lang/cs/Org.Apache.REEF.Tang/Implementations/InjectionPlan/InjectorImpl.cs
+++ b/lang/cs/Org.Apache.REEF.Tang/Implementations/InjectionPlan/InjectorImpl.cs
@@ -125,7 +125,7 @@ namespace Org.Apache.REEF.Tang.Implementations.InjectionPlan
                     Org.Apache.REEF.Utilities.Diagnostics.Exceptions.Throw(new InjectionException("Could not get class for " + key), LOGGER); 
                 }
             }
-            else if (plan.GetNode() is IClassNode && null != GetCachedInstance((IClassNode)plan.GetNode()))
+            else if (plan.GetNode() is IClassNode && GetCachedInstance((IClassNode)plan.GetNode()) != null)
             {
                 return GetCachedInstance((IClassNode)plan.GetNode());
             }
@@ -512,7 +512,7 @@ namespace Org.Apache.REEF.Tang.Implementations.InjectionPlan
             {
                 List<InjectionPlan> constructors = new List<InjectionPlan>();
                 List<IConstructorDef> constructorList = new List<IConstructorDef>();
-                if (null != this.configuration.GetLegacyConstructor(thisCN))
+                if (this.configuration.GetLegacyConstructor(thisCN) != null)
                 {
                     constructorList.Add(this.configuration.GetLegacyConstructor(thisCN));
                 }

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tang/Util/ReflectionUtilities.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/Util/ReflectionUtilities.cs b/lang/cs/Org.Apache.REEF.Tang/Util/ReflectionUtilities.cs
index eec5693..9616bb3 100644
--- a/lang/cs/Org.Apache.REEF.Tang/Util/ReflectionUtilities.cs
+++ b/lang/cs/Org.Apache.REEF.Tang/Util/ReflectionUtilities.cs
@@ -304,7 +304,7 @@ namespace Org.Apache.REEF.Tang.Util
         /// <returns></returns>
         public static Type EnsureInterfaceType(Type interf)
         {
-            if (interf != null && interf.IsGenericType && null == interf.FullName)
+            if (interf != null && interf.IsGenericType && interf.FullName == null)
             {
                 return interf.GetGenericTypeDefinition(); // this is to test if this line is ever reached
             }

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tang/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tang/packages.config b/lang/cs/Org.Apache.REEF.Tang/packages.config
index 6720116..4f96400 100644
--- a/lang/cs/Org.Apache.REEF.Tang/packages.config
+++ b/lang/cs/Org.Apache.REEF.Tang/packages.config
@@ -21,5 +21,5 @@ under the License.
   <package id="Microsoft.Hadoop.Avro" version="1.5.6" targetFramework="net45" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/HelloSimpleEventHandlers.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/HelloSimpleEventHandlers.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/HelloSimpleEventHandlers.cs
index bdca8c2..6e2a6e6 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/HelloSimpleEventHandlers.cs
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/Bridge/HelloSimpleEventHandlers.cs
@@ -310,7 +310,7 @@ namespace Org.Apache.REEF.Tests.Functional.Bridge
         {
             Logger.Log(Level.Info, "SubmitNextTask with evaluator id: " + activeContext.EvaluatorId);
             IConfiguration finalConfiguration = GetNextTaskConfiguration();
-            if (null != finalConfiguration)
+            if (finalConfiguration != null)
             {
                 Logger.Log(Level.Info, "Executing task id " + _taskContext.CurrentTaskId());
                 Logger.Log(Level.Info, string.Format(CultureInfo.InvariantCulture, "Submitting Task {0}", _taskContext.CurrentTaskId()));

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tests/Functional/ReefFunctionalTest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/ReefFunctionalTest.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/ReefFunctionalTest.cs
index 6a1f1d7..0dbb9b8 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Functional/ReefFunctionalTest.cs
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/ReefFunctionalTest.cs
@@ -236,7 +236,7 @@ namespace Org.Apache.REEF.Tests.Functional
                 }
                 else if (numberOfOccurrences == 0)
                 {
-                    Assert.True(0 == successIndicators.Count(),
+                    Assert.True(successIndicators.Count() == 0,
                         "Message \"" + message + "\" not expected to occur but occurs " + successIndicators.Count() + " times");
                 }
                 else

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/packages.config b/lang/cs/Org.Apache.REEF.Tests/packages.config
index 9a239bc..933ef87 100644
--- a/lang/cs/Org.Apache.REEF.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.Tests/packages.config
@@ -24,7 +24,7 @@ under the License.
   <package id="Microsoft.Data.Services.Client" version="5.6.4" targetFramework="net45" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="System.Reactive.Core" version="3.1.1" targetFramework="net451" />
   <package id="System.Reactive.Interfaces" version="3.1.1" targetFramework="net451" />
   <package id="System.Spatial" version="5.6.4" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Utilities/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Utilities/packages.config b/lang/cs/Org.Apache.REEF.Utilities/packages.config
index e725c50..0fce481 100644
--- a/lang/cs/Org.Apache.REEF.Utilities/packages.config
+++ b/lang/cs/Org.Apache.REEF.Utilities/packages.config
@@ -19,5 +19,5 @@ under the License.
 -->
 <packages>
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Wake.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake.Tests/packages.config b/lang/cs/Org.Apache.REEF.Wake.Tests/packages.config
index e0d4dbf..847e6d5 100644
--- a/lang/cs/Org.Apache.REEF.Wake.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.Wake.Tests/packages.config
@@ -19,7 +19,7 @@ under the License.
 -->
 <packages>
   <package id="Microsoft.Hadoop.Avro" version="1.5.6" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="System.Reactive.Core" version="3.1.1" targetFramework="net451" />
   <package id="System.Reactive.Interfaces" version="3.1.1" targetFramework="net451" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Wake/Examples/P2p/Pull2Push.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake/Examples/P2p/Pull2Push.cs b/lang/cs/Org.Apache.REEF.Wake/Examples/P2p/Pull2Push.cs
index 2593b44..857ed07 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Examples/P2p/Pull2Push.cs
+++ b/lang/cs/Org.Apache.REEF.Wake/Examples/P2p/Pull2Push.cs
@@ -71,11 +71,11 @@ namespace Org.Apache.REEF.Wake.Examples.P2p
             {
                 // Grab the next available message source, if any
                 IEventSource<T> nextSource = _sources.Dequeue();
-                if (null != nextSource)
+                if (nextSource != null)
                 {
                     // Grab the next message from that source, if any
                     T message = nextSource.GetNext();
-                    if (null != message)
+                    if (message != null)
                     {
                         // Add the source to the end of the queue again.
                         _sources.Enqueue(nextSource);

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/Org.Apache.REEF.Wake/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake/packages.config b/lang/cs/Org.Apache.REEF.Wake/packages.config
index e6f2691..6375661 100644
--- a/lang/cs/Org.Apache.REEF.Wake/packages.config
+++ b/lang/cs/Org.Apache.REEF.Wake/packages.config
@@ -23,7 +23,7 @@ under the License.
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" developmentDependency="true" />
   <package id="Microsoft.Hadoop.Avro" version="1.5.6" targetFramework="net45" />
   <package id="protobuf-net" version="2.1.0" targetFramework="net45" />
-  <package id="StyleCop.MSBuild" version="4.7.49.1" targetFramework="net45" developmentDependency="true" />
+  <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="System.Reactive.Core" version="3.1.1" targetFramework="net451" />
   <package id="System.Reactive.Interfaces" version="3.1.1" targetFramework="net451" />
   <package id="TransientFaultHandling.Core" version="5.1.1209.1" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/build.DotNet.props
----------------------------------------------------------------------
diff --git a/lang/cs/build.DotNet.props b/lang/cs/build.DotNet.props
index 1635d52..6e37cef 100644
--- a/lang/cs/build.DotNet.props
+++ b/lang/cs/build.DotNet.props
@@ -49,7 +49,7 @@ under the License.
     <NSubstituteVersion>1.8.2.0</NSubstituteVersion>
     <ProtobufVersion>2.1.0</ProtobufVersion>
     <SystemReactiveVersion>3.1.1</SystemReactiveVersion>
-    <StyleCopVersion>4.7.49.1</StyleCopVersion>
+    <StyleCopVersion>5.0.0</StyleCopVersion>
     <TransientFaultHandlingVersion>1.0.0</TransientFaultHandlingVersion>
     <NSubstituteVersion>2.0.3</NSubstituteVersion>
     <WindowsAzureStorageVersion>8.1.3</WindowsAzureStorageVersion>

http://git-wip-us.apache.org/repos/asf/reef/blob/227ce1d4/lang/cs/build.props
----------------------------------------------------------------------
diff --git a/lang/cs/build.props b/lang/cs/build.props
index 63f62eb..9cfe502 100644
--- a/lang/cs/build.props
+++ b/lang/cs/build.props
@@ -76,7 +76,7 @@ under the License.
     <NewtonsoftJsonVersion>10.0.3</NewtonsoftJsonVersion>
     <ProtobufVersion>2.1.0</ProtobufVersion>
     <SystemReactiveVersion>3.1.1</SystemReactiveVersion>
-    <StyleCopVersion>4.7.49.1</StyleCopVersion>
+    <StyleCopVersion>5.0.0</StyleCopVersion>
     <NSubstituteVersion>1.8.2.0</NSubstituteVersion>
   </PropertyGroup>
 


[25/28] reef git commit: [REEF-1966] Allow user to define app.config for Evaluator

Posted by do...@apache.org.
[REEF-1966] Allow user to define app.config for Evaluator

  * Use user define `app.config` for Evaluator if any. Otherwise, use
    default.
  * Remove `app.config` file from Evaluator project

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

Pull Request:
  This closes #1421


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

Branch: refs/heads/REEF-335
Commit: d459549354a144c94152c8b7ffc323ea186f01fb
Parents: e43019f
Author: jwang98052 <ju...@apache.org>
Authored: Tue Dec 12 16:21:27 2017 -0800
Committer: Markus Weimer <we...@apache.org>
Committed: Mon Feb 5 19:55:06 2018 -0800

----------------------------------------------------------------------
 .../Common/DriverFolderPreparationHelper.cs           | 14 +++++++++++---
 .../Org.Apache.REEF.Evaluator.csproj                  |  3 ++-
 2 files changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/d4595493/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 229d9e5..f51a3c4 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/DriverFolderPreparationHelper.cs
@@ -48,7 +48,7 @@ namespace Org.Apache.REEF.Client.Common
         @"    </assemblyBinding>" +
         @"  </runtime>" +
         @"</configuration>";
-        private const string EvaluatorExecutable = "Org.Apache.REEF.Evaluator.exe.config";
+        private const string EvaluatorExecutableConfig = "Org.Apache.REEF.Evaluator.exe.config";
 
         private static readonly Logger Logger = Logger.GetLogger(typeof(DriverFolderPreparationHelper));
         private readonly AvroConfigurationSerializer _configurationSerializer;
@@ -154,8 +154,16 @@ namespace Org.Apache.REEF.Client.Common
             File.WriteAllText(Path.Combine(driverFolderPath, _fileNames.GetBridgeExeConfigPath()), config);
 
             // generate .config file for Evaluator executable
-            File.WriteAllText(Path.Combine(driverFolderPath, _fileNames.GetGlobalFolderPath(), EvaluatorExecutable), 
-                DefaultDriverConfigurationFileContents);
+            var userDefinedEvaluatorConfigFileName = Path.Combine(JarFolder, EvaluatorExecutableConfig);
+            var evaluatorConfigFilName = Path.Combine(driverFolderPath, _fileNames.GetGlobalFolderPath(), EvaluatorExecutableConfig);
+            string evaluatorAppConfigString = DefaultDriverConfigurationFileContents;
+
+            if (File.Exists(userDefinedEvaluatorConfigFileName))
+            {
+                evaluatorAppConfigString = File.ReadAllText(userDefinedEvaluatorConfigFileName);
+            }
+            Logger.Log(Level.Verbose, "Create EvaluatorConfigFile {0} with config {1}.", evaluatorConfigFilName, evaluatorAppConfigString);
+            File.WriteAllText(evaluatorConfigFilName, evaluatorAppConfigString);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/d4595493/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj b/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj
index 7b567c8..8d2a818 100644
--- a/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj
+++ b/lang/cs/Org.Apache.REEF.Evaluator/Org.Apache.REEF.Evaluator.csproj
@@ -67,6 +67,7 @@ under the License.
   <ItemGroup>
     <None Include="$(SolutionDir)\App.config">
       <Link>App.config</Link>
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </None>
     <None Include="Org.Apache.REEF.Evaluator.nuspec" />
     <None Include="packages.config" />
@@ -96,4 +97,4 @@ under the License.
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
   <Import Project="$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets" Condition="Exists('$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets')" />
-</Project>
+</Project>
\ No newline at end of file


[15/28] reef git commit: [REEF-1950] Better test reporting for REEF.NET

Posted by do...@apache.org.
[REEF-1950] Better test reporting for REEF.NET

This adds an API to write functional tests of REEF. The main API is
`ITestRunner` through which a Test can be submitted. In the Driver of a test,
one can use an injected instance of `Assert` to record the passing or faling of
tests.

A `ITestRunner` is obtained via the `TestRunnerFactory` which uses the same
environment variables used by the Java tests to determine which TestRunner to
instantiate.

A good way to understand the new functionality is the new functional test,
`TestTestFramework`.

Known issues:

  * The `IAssert` API is spartan.
  * There is only an implementation for the local `TestRunner`.

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

Pull Request:
  This closes #1412


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

Branch: refs/heads/REEF-335
Commit: 3b89926a2498790b7056555acd098f42ac5d68a9
Parents: 3d571e6
Author: Markus Weimer <we...@apache.org>
Authored: Sun Oct 22 09:55:17 2017 -0700
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Thu Nov 9 18:12:57 2017 -0800

----------------------------------------------------------------------
 .../Org.Apache.REEF.Client.Tests.csproj         |   5 +
 .../TestFileWritingAssert.cs                    |  82 +++++++++
 .../API/Testing/AbstractAssert.cs               |  38 ++++
 .../API/Testing/AssertResult.cs                 |  68 +++++++
 .../API/Testing/IAssert.cs                      |  51 ++++++
 .../API/Testing/ITestResult.cs                  |  48 +++++
 .../API/Testing/ITestRunner.cs                  |  42 +++++
 .../API/Testing/TestRunnerFactory.cs            |  60 +++++++
 .../FileWritingAssert/FileWritingAssert.cs      |  47 +++++
 .../FileWritingAssertConfiguration.cs           |  39 +++++
 .../TestRunner/FileWritingAssert/Parameters.cs  |  30 ++++
 .../TestRunner/FileWritingAssert/TestResult.cs  | 175 +++++++++++++++++++
 .../Local/TestRunner/LocalTestRunner.cs         | 118 +++++++++++++
 .../Org.Apache.REEF.Client.csproj               |  11 ++
 .../Functional/TestFramework/README.md          |   3 +
 .../TestFramework/TestTestFramework.cs          | 164 +++++++++++++++++
 .../Org.Apache.REEF.Tests.csproj                |   5 +-
 17 files changed, 985 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj b/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
index 6674d25..f1bf43e 100644
--- a/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/Org.Apache.REEF.Client.Tests.csproj
@@ -57,6 +57,7 @@ under the License.
     <Compile Include="LegacyJobResourceUploaderTests.cs" />
     <Compile Include="MultipleRMUrlProviderTests.cs" />
     <Compile Include="RestClientTests.cs" />
+    <Compile Include="TestFileWritingAssert.cs" />
     <Compile Include="WindowsHadoopEmulatorYarnClientTests.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
     <Compile Include="YarnClientTests.cs" />
@@ -89,6 +90,10 @@ under the License.
       <Project>{cdfb3464-4041-42b1-9271-83af24cd5008}</Project>
       <Name>Org.Apache.REEF.Wake</Name>
     </ProjectReference>
+    <ProjectReference Include="..\Org.Apache.REEF.Common\Org.Apache.REEF.Common.csproj">
+      <Project>{545A0582-4105-44CE-B99C-B1379514A630}</Project>
+      <Name>Org.Apache.REEF.Common</Name>
+    </ProjectReference>
   </ItemGroup>
   <ItemGroup>
     <None Include="packages.config" />

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client.Tests/TestFileWritingAssert.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client.Tests/TestFileWritingAssert.cs b/lang/cs/Org.Apache.REEF.Client.Tests/TestFileWritingAssert.cs
new file mode 100644
index 0000000..1f5d788
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client.Tests/TestFileWritingAssert.cs
@@ -0,0 +1,82 @@
+// 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.TestRunner.FileWritingAssert;
+using Xunit;
+
+namespace Org.Apache.REEF.Client.Tests
+{
+    /// <summary>
+    /// Tests for the File Writing Assert
+    /// </summary>
+    public class TestFileWritingAssert
+    {
+        [Fact]
+        public void TestTestResult()
+        {
+            TestResult x = new TestResult();
+            Assert.Equal(0, x.NumberOfPassedAsserts);
+            Assert.Equal(0, x.NumberOfFailedAsserts);
+
+            x.Add(true, "Something went right");
+            Assert.Equal(1, x.NumberOfPassedAsserts);
+            Assert.Equal(0, x.NumberOfFailedAsserts);
+            Assert.True(x.AllTestsSucceeded);
+
+            x.IsTrue("Something else went right");
+            Assert.Equal(2, x.NumberOfPassedAsserts);
+            Assert.Equal(0, x.NumberOfFailedAsserts);
+            Assert.True(x.AllTestsSucceeded);
+
+            x.Add(false, "Something went wrong");
+            Assert.Equal(2, x.NumberOfPassedAsserts);
+            Assert.Equal(1, x.NumberOfFailedAsserts);
+            Assert.False(x.AllTestsSucceeded);
+
+            x.IsFalse("Something else went wrong");
+            Assert.Equal(2, x.NumberOfPassedAsserts);
+            Assert.Equal(2, x.NumberOfFailedAsserts);
+            Assert.False(x.AllTestsSucceeded);
+        }
+
+        [Fact]
+        public void TestTestResultFail()
+        {
+            var x = TestResult.Fail("OMG! It failed!");
+            Assert.Equal(0, x.NumberOfPassedAsserts);
+            Assert.Equal(1, x.NumberOfFailedAsserts);
+            Assert.False(x.AllTestsSucceeded);
+        }
+
+        [Fact]
+        public void TestTestResultSerialization()
+        {
+            TestResult before = new TestResult();
+            before.Add(true, "Something went right");
+            before.Add(true, "Something else went right");
+            before.Add(false, "Something went wrong");
+
+            TestResult after = TestResult.FromJson(before.ToJson());
+
+            Assert.NotNull(after);
+            Assert.Equal(1, after.NumberOfFailedAsserts);
+            Assert.Equal(2, after.NumberOfPassedAsserts);
+
+            Assert.Equal(before.ToJson(), after.ToJson());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/API/Testing/AbstractAssert.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Testing/AbstractAssert.cs b/lang/cs/Org.Apache.REEF.Client/API/Testing/AbstractAssert.cs
new file mode 100644
index 0000000..608eb81
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Testing/AbstractAssert.cs
@@ -0,0 +1,38 @@
+// 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.Testing
+{
+    /// <inheritdoc />
+    /// <summary>
+    /// Helper class to ease the implementation of additional Assert classes.
+    /// </summary>
+    internal abstract class AbstractAssert : IAssert
+    {
+        public abstract void True(bool condition, string format, params object[] args);
+
+        public void False(bool condition, string format, params object[] args)
+        {
+            True(!condition, format);
+        }
+
+        public void Fail(string format, params object[] args)
+        {
+            True(false, format, args);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/API/Testing/AssertResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Testing/AssertResult.cs b/lang/cs/Org.Apache.REEF.Client/API/Testing/AssertResult.cs
new file mode 100644
index 0000000..7cb0a23
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Testing/AssertResult.cs
@@ -0,0 +1,68 @@
+// 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.Utilities.Attributes;
+using System;
+using System.Collections.Generic;
+
+namespace Org.Apache.REEF.Client.API.Testing
+{
+    /// <summary>
+    /// Serializable representation of a result of an assert.
+    /// </summary>
+    [Unstable("0.17", "Work in progress towards a new test infrastructure. See REEF-1271.")]
+    internal sealed class AssertResult : IEquatable<AssertResult>
+    {
+        public AssertResult(string message, bool isTrue)
+        {
+            Message = message;
+            IsTrue = isTrue;
+        }
+
+        public string Message { get; }
+
+        public bool IsTrue { get; }
+
+        public bool IsFalse
+        {
+            get
+            {
+                return !IsTrue;
+            }
+        }
+
+        public override bool Equals(object obj)
+        {
+            return Equals(obj as AssertResult);
+        }
+
+        public bool Equals(AssertResult other)
+        {
+            return other != null &&
+                   Message == other.Message &&
+                   IsTrue == other.IsTrue;
+        }
+
+        public override int GetHashCode()
+        {
+            var hashCode = -1707516999;
+            hashCode = (hashCode * -1521134295) + EqualityComparer<string>.Default.GetHashCode(Message);
+            hashCode = (hashCode * -1521134295) + IsTrue.GetHashCode();
+            return hashCode;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/API/Testing/IAssert.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Testing/IAssert.cs b/lang/cs/Org.Apache.REEF.Client/API/Testing/IAssert.cs
new file mode 100644
index 0000000..7f2ecd3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Testing/IAssert.cs
@@ -0,0 +1,51 @@
+// 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.Utilities.Attributes;
+
+namespace Org.Apache.REEF.Client.API.Testing
+{
+    /// <summary>
+    /// Assert methods to be used in tests of REEF and REEF applications.
+    /// </summary>
+    [Unstable("0.17", "Work in progress towards a new test infrastructure. See REEF-1271.")]
+    public interface IAssert
+    {
+        /// <summary>
+        /// Assert that a boolean condition is true.
+        /// </summary>
+        /// <param name="condition">The condition. True indicates a passed test, false otherwise.</param>
+        /// <param name="format">The error message for the test if condition is false.</param>
+        /// <param name="args">Arguments to `format`.</param>
+        void True(bool condition, string format, params object[] args);
+
+        /// <summary>
+        /// Assert that a boolean condition is false.
+        /// </summary>
+        /// <param name="condition">The condition. False indicates a passed test, true otherwise.</param>
+        /// <param name="format">The error message for the test if condition is true.</param>
+        /// <param name="args">Arguments to `format`.</param>
+        void False(bool condition, string format, params object[] args);
+
+        /// <summary>
+        /// Record a failed test.
+        /// </summary>
+        /// <param name="format">The message for the failed test.</param>
+        /// <param name="args">Arguments to `format`.</param>
+        void Fail(string format, params object[] args);
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/API/Testing/ITestResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Testing/ITestResult.cs b/lang/cs/Org.Apache.REEF.Client/API/Testing/ITestResult.cs
new file mode 100644
index 0000000..839c919
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Testing/ITestResult.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.Utilities.Attributes;
+
+namespace Org.Apache.REEF.Client.API.Testing
+{
+    /// <summary>
+    /// Represents a test result.
+    /// </summary>
+    [Unstable("0.17", "Work in progress towards a new test infrastructure. See REEF-1271.")]
+    public interface ITestResult
+    {
+        /// <summary>
+        /// The number of failed asserts in this test.
+        /// </summary>
+        int NumberOfFailedAsserts { get; }
+
+        /// <summary>
+        /// The number of passed asserts in this test.
+        /// </summary>
+        int NumberOfPassedAsserts { get; }
+
+        /// <summary>
+        /// True, if all asserts passed.
+        /// </summary>
+        bool AllTestsSucceeded { get; }
+
+        /// <summary>
+        /// The error message to use if AllTestsSucceeded is false.
+        /// </summary>
+        string FailedTestMessage { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/API/Testing/ITestRunner.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Testing/ITestRunner.cs b/lang/cs/Org.Apache.REEF.Client/API/Testing/ITestRunner.cs
new file mode 100644
index 0000000..4cfcdbf
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Testing/ITestRunner.cs
@@ -0,0 +1,42 @@
+// 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.Utilities.Attributes;
+
+namespace Org.Apache.REEF.Client.API.Testing
+{
+    /// <summary>
+    /// Runs REEF integration tests and reports their results.
+    /// </summary>
+    [Unstable("0.17", "Work in progress towards a new test infrastructure. See REEF-1271.")]
+    public interface ITestRunner
+    {
+        /// <summary>
+        /// Create a new JobRequestBuilder.
+        /// </summary>
+        /// <remarks>This may pre-configure the job request for this test runner.</remarks>
+        /// <returns>A new JobRequestBuilder.</returns>
+        JobRequestBuilder NewJobRequestBuilder();
+
+        /// <summary>
+        /// Runs the given Job as a test.
+        /// </summary>
+        /// <param name="jobRequestBuilder">The job to run.</param>
+        /// <returns>The test results obtained.</returns>
+        ITestResult RunTest(JobRequestBuilder jobRequestBuilder);
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/API/Testing/TestRunnerFactory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Testing/TestRunnerFactory.cs b/lang/cs/Org.Apache.REEF.Client/API/Testing/TestRunnerFactory.cs
new file mode 100644
index 0000000..5382f1d
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Testing/TestRunnerFactory.cs
@@ -0,0 +1,60 @@
+// 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.TestRunner;
+using Org.Apache.REEF.Utilities.Attributes;
+using System;
+
+namespace Org.Apache.REEF.Client.API.Testing
+{
+    /// <summary>
+    /// Factory for TestRunner instances.
+    /// </summary>
+    /// <remarks>
+    /// This class follows the same approach as org.apache.reef.tests.TestEnvironmentFactory in Java. It reads the
+    /// same environment variables to decide which test runner to instantiate.
+    /// </remarks>
+    [Unstable("0.17", "Work in progress towards a new test infrastructure. See REEF-1271.")]
+    public sealed class TestRunnerFactory
+    {
+        // See `org.apache.reef.tests.TestEnvironmentFactory` in Java.
+        private const string TestOnYARNEnvironmentVariable = "REEF_TEST_YARN";
+
+        /// <summary>
+        /// Instantiates a TestRunner based on the environment variables.
+        /// </summary>
+        /// <returns>A TestRunner instance.</returns>
+        public static ITestRunner NewTestRunner()
+        {
+            if (RunOnYarn())
+            {
+                throw new NotImplementedException("Running tests on YARN is not supported yet.");
+            }
+            return LocalTestRunner.GetLocalTestRunner();
+        }
+
+        /// <summary>
+        /// Check whether the tests are supposed to be run on YARN.
+        /// </summary>
+        /// <returns>True, if the tests are supposed to run on YARN.</returns>
+        private static bool RunOnYarn()
+        {
+            return bool.TryParse(Environment.GetEnvironmentVariable(TestOnYARNEnvironmentVariable),
+                       out bool runOnYARN) && runOnYARN;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/FileWritingAssert.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/FileWritingAssert.cs b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/FileWritingAssert.cs
new file mode 100644
index 0000000..c23dc8c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/FileWritingAssert.cs
@@ -0,0 +1,47 @@
+// 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.Tang.Annotations;
+using Org.Apache.REEF.Client.API.Testing;
+
+namespace Org.Apache.REEF.Client.Local.TestRunner.FileWritingAssert
+{
+    internal sealed class FileWritingAssert : AbstractAssert
+    {
+        private readonly TestResult _testResult = new TestResult();
+        private readonly string _filePath;
+
+        /// <param name="filePath">The path to the file where the assert results shall be written.</param>
+        [Inject]
+        internal FileWritingAssert([Parameter(typeof(Parameters.AssertFilePath))] string filePath)
+        {
+            _filePath = filePath;
+        }
+
+        public override void True(bool condition, string format, params object[] args)
+        {
+            _testResult.Add(condition, format, args);
+            WriteAssertsFile();
+        }
+
+        private void WriteAssertsFile()
+        {
+            File.WriteAllText(_filePath, _testResult.ToJson());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/FileWritingAssertConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/FileWritingAssertConfiguration.cs b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/FileWritingAssertConfiguration.cs
new file mode 100644
index 0000000..7de7cbe
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/FileWritingAssertConfiguration.cs
@@ -0,0 +1,39 @@
+// 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.Testing;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.Client.Local.TestRunner.FileWritingAssert
+{
+    /// <summary>
+    /// Configuration Module for the file writing assert validation.
+    /// </summary>
+    internal sealed class FileWritingAssertConfiguration : ConfigurationModuleBuilder
+    {
+        /// <summary>
+        /// Path to the file to be written for the asserts. Uses a temp file if not set.
+        /// </summary>
+        public static readonly OptionalParameter<string> FilePath = new OptionalParameter<string>();
+
+        public static ConfigurationModule ConfigurationModule = new FileWritingAssertConfiguration()
+            .BindImplementation(GenericType<IAssert>.Class, GenericType<FileWritingAssert>.Class)
+            .BindNamedParameter(GenericType<Parameters.AssertFilePath>.Class, FilePath)
+            .Build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/Parameters.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/Parameters.cs b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/Parameters.cs
new file mode 100644
index 0000000..d83b25b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/Parameters.cs
@@ -0,0 +1,30 @@
+// 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;
+
+// ReSharper disable once CheckNamespace
+namespace Org.Apache.REEF.Client.Local.TestRunner.FileWritingAssert.Parameters
+{
+    [NamedParameter(documentation: "Path where the assert log shall be written.")]
+    internal sealed class AssertFilePath : Name<string>
+    {
+        private AssertFilePath()
+        {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/TestResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/TestResult.cs b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/TestResult.cs
new file mode 100644
index 0000000..2a3a16b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/FileWritingAssert/TestResult.cs
@@ -0,0 +1,175 @@
+// 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.Linq;
+using System;
+using Newtonsoft.Json;
+using Org.Apache.REEF.Client.API.Testing;
+
+namespace Org.Apache.REEF.Client.Local.TestRunner.FileWritingAssert
+{
+    internal sealed class TestResult : ITestResult, IEquatable<TestResult>
+    {
+        private readonly IList<AssertResult> _results;
+
+        public TestResult(IEnumerable<AssertResult> results)
+        {
+            _results = results.ToList();
+        }
+
+        public TestResult() : this(new List<AssertResult>())
+        {
+            // Intentionally empty
+        }
+
+        /// <inheritdoc />
+        public int NumberOfFailedAsserts
+        {
+            get
+            {
+                return _results.Count(_ => _.IsFalse);
+            }
+        }
+
+        /// <inheritdoc />
+        public int NumberOfPassedAsserts
+        {
+            get
+            {
+                return _results.Count(_ => _.IsTrue);
+            }
+        }
+
+        /// <inheritdoc />
+        public bool AllTestsSucceeded
+        {
+            get
+            {
+                return NumberOfFailedAsserts == 0;
+            }
+        }
+
+        private IEnumerable<AssertResult> FailedAsserts
+        {
+            get
+            {
+                return _results.Where(_ => _.IsFalse);
+            }
+        }
+
+        private IEnumerable<AssertResult> PassedAsserts
+        {
+            get
+            {
+                return _results.Where(_ => _.IsTrue);
+            }
+        }
+
+        /// <inheritdoc />
+        public string FailedTestMessage
+        {
+            get { return "Failed tests: " + string.Join(";", FailedAsserts.Select(_ => _.Message)); }
+        }
+
+        /// <summary>
+        /// Add a Assert result to the collection.
+        /// </summary>
+        /// <param name="condition">Whether or not the condition was met.</param>
+        /// <param name="format">The message of the assert.</param>
+        /// <param name="args">Parameters to `format`.</param>
+        /// <returns>this, for chain calls</returns>
+        public TestResult Add(bool condition, string format, params object[] args)
+        {
+            _results.Add(new AssertResult(format, condition));
+            return this;
+        }
+
+        /// <summary>
+        /// Record an assert that passed.
+        /// </summary>
+        /// <param name="format">The message to record</param>
+        /// <param name="args">Parameters for the format string.</param>
+        /// <returns>this, for chain calls</returns>
+        public TestResult IsTrue(string format, params object[] args)
+        {
+            return Add(true, format, args);
+        }
+
+        /// <summary>
+        /// Record an assert that failed.
+        /// </summary>
+        /// <param name="format">The message to record</param>
+        /// <param name="args">Parameters for the format string.</param>
+        /// <returns>this, for chain calls</returns>
+        public TestResult IsFalse(string format, params object[] args)
+        {
+            return Add(false, format, args);
+        }
+
+        /// <summary>
+        /// Serializes the data contained in this object to JSON.
+        /// </summary>
+        /// <returns>A string version of this object.</returns>
+        public string ToJson()
+        {
+            return JsonConvert.SerializeObject(_results);
+        }
+
+        /// <summary>
+        /// Deserializes an instance from a string generated by ToJson().
+        /// </summary>
+        /// <param name="serializedObject">The object to deserialize.</param>
+        /// <returns>The deserialized object or null if serializedObject is null or whitespace.</returns>
+        public static TestResult FromJson(string serializedObject)
+        {
+            if (string.IsNullOrWhiteSpace(serializedObject))
+            {
+                return null;
+            }
+
+            return new TestResult(JsonConvert.DeserializeObject<List<AssertResult>>(serializedObject));
+        }
+
+        /// <summary>
+        /// Creates a TestResult with a single failure inside.
+        /// </summary>
+        /// <param name="format">The message for the failure.</param>
+        /// <param name="args">Parameters, if `format` refers to them.</param>
+        /// <returns>A TestResult with a single failure inside.</returns>
+        public static TestResult Fail(string format, params object[] args)
+        {
+            return new TestResult().IsFalse(format, args);
+        }
+
+        public override bool Equals(object obj)
+        {
+            return Equals(obj as TestResult);
+        }
+
+        public bool Equals(TestResult other)
+        {
+            return other != null &&
+                   EqualityComparer<IList<AssertResult>>.Default.Equals(_results, other._results);
+        }
+
+        public override int GetHashCode()
+        {
+            return -3177284 + EqualityComparer<IList<AssertResult>>.Default.GetHashCode(_results);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs
new file mode 100644
index 0000000..653917d
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/Local/TestRunner/LocalTestRunner.cs
@@ -0,0 +1,118 @@
+// 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.TestRunner.FileWritingAssert;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Client.API.Testing;
+
+namespace Org.Apache.REEF.Client.Local.TestRunner
+{
+    /// <summary>
+    /// Runs a test on the local runtime.
+    /// </summary>
+    internal sealed class LocalTestRunner : ITestRunner
+    {
+        private static readonly Logger LOG = Logger.GetLogger(typeof(LocalTestRunner));
+        private readonly IREEFClient _client;
+
+        [Inject]
+        private LocalTestRunner(IREEFClient client)
+        {
+            _client = client;
+        }
+
+        public JobRequestBuilder NewJobRequestBuilder()
+        {
+            return _client.NewJobRequestBuilder();
+        }
+
+        public ITestResult RunTest(JobRequestBuilder jobRequestBuilder)
+        {
+            // Setup the assert file.
+            var assertFileName = Path.GetTempPath() + "/reef-test-" + DateTime.Now.ToString("yyyyMMddHHmmssfff") + ".json";
+            jobRequestBuilder.AddDriverConfiguration(FileWritingAssertConfiguration.ConfigurationModule
+                    .Set(FileWritingAssertConfiguration.FilePath, assertFileName)
+                    .Build());
+            var jobRequest = jobRequestBuilder.Build();
+
+            LOG.Log(Level.Info, "Submitting job `{0}` for execution. Assert log in `{1}`",
+                jobRequest.JobIdentifier,
+                assertFileName);
+            IJobSubmissionResult jobStatus = _client.SubmitAndGetJobStatus(jobRequest);
+
+            if (null == jobStatus)
+            {
+                return TestResult.Fail(
+                    "JobStatus returned by the Client was null. This points to an environment setup problem.");
+            }
+
+            LOG.Log(Level.Verbose, "Waiting for job `{0}` to complete.", jobRequest.JobIdentifier);
+            jobStatus.WaitForDriverToFinish();
+            LOG.Log(Level.Verbose, "Job `{0}` completed.", jobRequest.JobIdentifier);
+
+            return ReadTestResult(assertFileName);
+        }
+
+        private static TestResult ReadTestResult(string assertFilePath)
+        {
+            if (!File.Exists(assertFilePath))
+            {
+                return TestResult.Fail("Test Results file {0} does not exist.", assertFilePath);
+            }
+
+            try
+            {
+                return TestResult.FromJson(File.ReadAllText(assertFilePath))
+                    ?? TestResult.Fail("Results read from `{0}` where null.", assertFilePath);
+            }
+            catch (Exception exception)
+            {
+                return TestResult.Fail("Could not parse test results: {0}", exception);
+            }
+        }
+
+        /// <summary>
+        /// Convenience method to generate a local test runner with the given number of containers.
+        /// </summary>
+        /// <param name="numberOfContainers"></param>
+        /// <returns></returns>
+        public static ITestRunner GetLocalTestRunner(int numberOfContainers = 4)
+        {
+            return GetLocalTestRunner(
+                LocalRuntimeClientConfiguration.ConfigurationModule
+                    .Set(LocalRuntimeClientConfiguration.NumberOfEvaluators, numberOfContainers.ToString())
+                    .Build());
+        }
+
+        /// <summary>
+        /// Convenience method to instantiate a local test runner with the given runtime Configuration.
+        /// </summary>
+        /// <param name="runtimeConfiguration"></param>
+        /// <returns></returns>
+        public static ITestRunner GetLocalTestRunner(IConfiguration runtimeConfiguration)
+        {
+            return TangFactory.GetTang().NewInjector(runtimeConfiguration).GetInstance<LocalTestRunner>();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/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 879b8c3..451597f 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
@@ -69,6 +69,11 @@ under the License.
     <Compile Include="API\Exceptions\ClasspathException.cs" />
     <Compile Include="API\Exceptions\JavaNotFoundException.cs" />
     <Compile Include="API\IREEFClient.cs" />
+    <Compile Include="API\Testing\AbstractAssert.cs" />
+    <Compile Include="API\Testing\IAssert.cs" />
+    <Compile Include="API\Testing\AssertResult.cs" />
+    <Compile Include="API\Testing\ITestResult.cs" />
+    <Compile Include="API\Testing\ITestRunner.cs" />
     <Compile Include="API\JobParameters.cs" />
     <Compile Include="API\JobParametersBuilder.cs" />
     <Compile Include="API\JobRequest.cs" />
@@ -101,7 +106,13 @@ under the License.
     <Compile Include="Local\LocalRuntimeClientConfiguration.cs" />
     <Compile Include="Local\Parameters\LocalRuntimeDirectory.cs" />
     <Compile Include="Local\Parameters\NumberOfEvaluators.cs" />
+    <Compile Include="Local\TestRunner\FileWritingAssert\FileWritingAssert.cs" />
+    <Compile Include="Local\TestRunner\FileWritingAssert\FileWritingAssertConfiguration.cs" />
+    <Compile Include="Local\TestRunner\FileWritingAssert\Parameters.cs" />
+    <Compile Include="Local\TestRunner\FileWritingAssert\TestResult.cs" />
+    <Compile Include="Local\TestRunner\LocalTestRunner.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="API\Testing\TestRunnerFactory.cs" />
     <Compile Include="YARN\ApplicationReport.cs" />
     <Compile Include="YARN\Environment.cs" />
     <Compile Include="YARN\HDI\HDInsightClientConfiguration.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Tests/Functional/TestFramework/README.md
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/TestFramework/README.md b/lang/cs/Org.Apache.REEF.Tests/Functional/TestFramework/README.md
new file mode 100644
index 0000000..38910bb
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/TestFramework/README.md
@@ -0,0 +1,3 @@
+# Test framework tests
+
+Tests in this namespace test the function of the test framework itself, not REEF.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Tests/Functional/TestFramework/TestTestFramework.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/TestFramework/TestTestFramework.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/TestFramework/TestTestFramework.cs
new file mode 100644
index 0000000..fa8e3d9
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/TestFramework/TestTestFramework.cs
@@ -0,0 +1,164 @@
+// 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.Driver;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using Xunit;
+using Org.Apache.REEF.Driver.Evaluator;
+using Org.Apache.REEF.Client.API.Testing;
+
+namespace Org.Apache.REEF.Tests.Functional.TestFramework
+{
+    /// <summary>
+    /// Tests of REEF's test framework
+    /// </summary>
+    public sealed class TestTestFramework
+    {
+        /// <summary>
+        /// Tests whether a Driver with a single failing assert is reported correctly.
+        /// </summary>
+        [Fact]
+        public void TestTestFailure()
+        {
+            ITestRunner testRunner = TestRunnerFactory.NewTestRunner();
+
+            // The TestRunner cannot be null.
+            Xunit.Assert.NotNull(testRunner);
+
+            // Submit the job.
+            ITestResult testResult = testRunner.RunTest(testRunner.NewJobRequestBuilder()
+                .AddDriverConfiguration(TestFailingStartHandler.GetDriverConfiguration())
+                .AddGlobalAssemblyForType(typeof(TestFailingStartHandler))
+                .SetJobIdentifier("TestFailingTest"));
+
+            // The TestResult cannot be null.
+            Xunit.Assert.NotNull(testResult);
+
+            // There should be at least 1 failing assert.
+            Xunit.Assert.False(testResult.AllTestsSucceeded, testResult.FailedTestMessage);
+
+            // Only the expected assert should have failed.
+            Xunit.Assert.Equal(1, testResult.NumberOfFailedAsserts);
+        }
+
+        /// <summary>
+        /// Tests whether a Driver with a single passing test is reported correctly.
+        /// </summary>
+        [Fact]
+        public void TestTestPassing()
+        {
+            ITestRunner testRunner = TestRunnerFactory.NewTestRunner();
+
+            // The TestRunner cannot be null.
+            Xunit.Assert.NotNull(testRunner);
+
+            // Submit the job.
+            ITestResult testResult = testRunner.RunTest(testRunner.NewJobRequestBuilder()
+                .AddDriverConfiguration(TestPassingStartHandler.GetDriverConfiguration())
+                .AddGlobalAssemblyForType(typeof(TestPassingStartHandler))
+                .SetJobIdentifier("TestPassingTest"));
+
+            // The TestResult cannot be null.
+            Xunit.Assert.NotNull(testResult);
+
+            // The TestResult cannot contain a failed assert.
+            Xunit.Assert.True(testResult.AllTestsSucceeded, testResult.FailedTestMessage);
+
+            // The TestResult cannot contain more than one passed assert.
+            Xunit.Assert.Equal(1, testResult.NumberOfPassedAsserts);
+        }
+    }
+
+    /// <inheritdoc />
+    /// <summary>
+    /// A mock test which always fails.
+    /// </summary>
+    internal sealed class TestFailingStartHandler : IObserver<IDriverStarted>
+    {
+        private readonly Client.API.Testing.IAssert _assert;
+
+        private const string FailedAssertMessage = "This test should never pass.";
+
+        [Inject]
+        private TestFailingStartHandler(Client.API.Testing.IAssert assert, IEvaluatorRequestor evaluatorRequestor)
+        {
+            _assert = assert;
+        }
+
+        public void OnNext(IDriverStarted value)
+        {
+            // Fail the test case.
+            _assert.True(false, FailedAssertMessage);
+        }
+
+        public void OnError(Exception error)
+        {
+            _assert.True(false, "Call to OnError() received.");
+        }
+
+        public void OnCompleted()
+        {
+            // empty on purpose.
+        }
+
+        public static IConfiguration GetDriverConfiguration()
+        {
+            return DriverConfiguration.ConfigurationModule
+                .Set(DriverConfiguration.OnDriverStarted, GenericType<TestFailingStartHandler>.Class)
+                .Build();
+        }
+    }
+
+    /// <summary>
+    /// A mock test which always succeeds.
+    /// </summary>
+    internal sealed class TestPassingStartHandler : IObserver<IDriverStarted>
+    {
+        private readonly Client.API.Testing.IAssert _assert;
+
+        [Inject]
+        private TestPassingStartHandler(Client.API.Testing.IAssert assert)
+        {
+            _assert = assert;
+        }
+
+        public void OnNext(IDriverStarted value)
+        {
+            _assert.True(true, "This test should always pass.");
+        }
+
+        public void OnError(Exception error)
+        {
+            _assert.True(false, "Call to OnError() received.");
+        }
+
+        public void OnCompleted()
+        {
+            // empty on purpose.
+        }
+
+        public static IConfiguration GetDriverConfiguration()
+        {
+            return DriverConfiguration.ConfigurationModule
+                .Set(DriverConfiguration.OnDriverStarted, GenericType<TestPassingStartHandler>.Class)
+                .Build();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3b89926a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
index b83b705..ff71bd6 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.Tests/Org.Apache.REEF.Tests.csproj
@@ -165,6 +165,7 @@ under the License.
     <Compile Include="Functional\Telemetry\MetricsDriver.cs" />
     <Compile Include="Functional\Telemetry\MetricsTask.cs" />
     <Compile Include="Functional\Telemetry\TestMetricsMessage.cs" />
+    <Compile Include="Functional\TestFramework\TestTestFramework.cs" />
     <Compile Include="Performance\TestHelloREEF\TestHelloDriver.cs" />
     <Compile Include="Performance\TestHelloREEF\TestHelloREEFClient.cs" />
     <Compile Include="Performance\TestHelloREEF\TestHelloTask.cs" />
@@ -248,7 +249,9 @@ under the License.
   <ItemGroup>
     <Service Include="{82A7F48D-3B50-4B1E-B82E-3ADA8210C358}" />
   </ItemGroup>
-  <ItemGroup />
+  <ItemGroup>
+    <None Include="Functional\TestFramework\README.md" />
+  </ItemGroup>
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
   <Import Project="$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets" Condition="Exists('$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets')" />


[14/28] reef git commit: [REEF-1951] Make Driver Status more robust.

Posted by do...@apache.org.
[REEF-1951] Make Driver Status more robust.

This change makes the communication of the Driver status between the
Driver and the .NET Client more robust. It gueards against the Client
calling the HTTP server to late by changes on the Java side. It guards
against calling the HTTP server to early by changes on the .NET side.

On the *Java side*, this introduces a new alarm on the clock that makes
sure that `DriverStatusHTTPHandler` is called at least once via HTTP in
2s after launching.

On the *.NET Side*, this introduces a retry loop around the first
attempt to connect to the Driver. This makes sure we don't call it too
early.

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

Pull Request:
  This closes #1413


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

Branch: refs/heads/REEF-335
Commit: 3d571e6a4bd85d095cca5bd2f903c2acf67a5f4f
Parents: e36ee92
Author: Markus Weimer <we...@apache.org>
Authored: Mon Nov 6 11:00:07 2017 -0800
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Tue Nov 7 18:16:25 2017 -0800

----------------------------------------------------------------------
 .../Org.Apache.REEF.Client/API/DriverStatus.cs  |  5 ++
 .../cs/Org.Apache.REEF.Client/API/Parameters.cs | 41 ++++++++++++
 .../Common/IJobSubmissionResult.cs              |  2 +-
 .../Common/JobSubmissionResult.cs               | 42 +++++++++++-
 .../Org.Apache.REEF.Client/Local/LocalClient.cs | 21 +++++-
 .../Local/LocalJobSubmissionResult.cs           |  7 +-
 .../Org.Apache.REEF.Client.csproj               |  1 +
 .../YARN/YARNREEFClient.cs                      | 21 +++++-
 .../YARN/YarnJobSubmissionResult.cs             |  7 +-
 .../bridge/client/DriverStatusHTTPHandler.java  | 70 +++++++++++++++++++-
 .../Parameters/HTTPStatusAlarmInterval.java     | 33 +++++++++
 .../Parameters/HTTPStatusNumberOfRetries.java   | 33 +++++++++
 .../bridge/client/Parameters/package-info.java  | 23 +++++++
 .../client/TestDriverStatusHTTPHandler.java     | 14 ++--
 14 files changed, 303 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs b/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
index d9934a9..2978382 100644
--- a/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
+++ b/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
@@ -23,6 +23,11 @@ namespace Org.Apache.REEF.Client.API
     internal enum DriverStatus
     {
         /// <summary>
+        /// Represents the fact that the Driver status hasn't been received yet.
+        /// </summary>
+        UNKNOWN,
+
+        /// <summary>
         /// Driver is initializing.
         /// </summary>
         INIT,

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/API/Parameters.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/Parameters.cs b/lang/cs/Org.Apache.REEF.Client/API/Parameters.cs
new file mode 100644
index 0000000..0b6db60
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/Parameters.cs
@@ -0,0 +1,41 @@
+// 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.API.Parameters
+{
+    /// <summary>
+    /// Interval ins ms between connection attempts to the Driver's HTTP Server to obtain Status information.
+    /// </summary>
+    [NamedParameter(DefaultValue = "500", Documentation = 
+        "Interval ins ms between connection attempts to the Driver's HTTP Server to obtain Status information.")]
+    internal sealed class DriverHTTPConnectionRetryInterval : Name<int>
+    {
+        // Intentionally empty
+    }
+
+    /// <summary>
+    /// Number of Retries when connecting to the Driver's HTTP server.
+    /// </summary>
+    [NamedParameter(DefaultValue = "10",
+        Documentation = "Number of Retries when connecting to the Driver's HTTP server.")]
+    internal sealed class DriverHTTPConnectionAttempts : Name<int>
+    {
+        // Intentionally empty
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
index 328677a..e3510ed 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
@@ -49,7 +49,7 @@ namespace Org.Apache.REEF.Client.Common
         /// <summary>
         /// Waits for the Driver to complete.
         /// </summary>
-        /// <exception cref="System.Net.WebException">If the Driver cannot be reached.</exception>
+        /// <exception cref="System.Net.WebException">If the Driver could be reached at least once.</exception>
         [Unstable("0.17", "Uses the HTTP server in the Java Driver. Might not work if that cannot be reached.")]
         void WaitForDriverToFinish();
     }

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
index 3880433..0649422 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
@@ -23,10 +23,17 @@ using System.Net.Http;
 using System.Net.Http.Headers;
 using System.Threading;
 using System.Threading.Tasks;
+#if DOTNET_BUILD
+using Microsoft.Practices.EnterpriseLibrary.TransientFaultHandling;
+#else
+using Microsoft.Practices.TransientFaultHandling;
+#endif
 using Newtonsoft.Json;
 using Org.Apache.REEF.Client.API;
+using Org.Apache.REEF.Client.YARN.RestClient;
 using Org.Apache.REEF.Client.YARN.RestClient.DataModel;
 using Org.Apache.REEF.Utilities.Logging;
+using HttpClient = System.Net.Http.HttpClient;
 
 namespace Org.Apache.REEF.Client.Common
 {
@@ -48,7 +55,17 @@ namespace Org.Apache.REEF.Client.Common
         private readonly HttpClient _client;
         private readonly IREEFClient _reefClient;
 
-        internal JobSubmissionResult(IREEFClient reefClient, string filePath)
+        /// <summary>
+        /// Number of retries when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _numberOfRetries;
+
+        /// <summary>
+        /// Retry interval in ms when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly TimeSpan _retryInterval;
+
+        internal JobSubmissionResult(IREEFClient reefClient, string filePath, int numberOfRetries, int retryInterval)
         {
             _reefClient = reefClient;
             _client = new HttpClient
@@ -58,6 +75,9 @@ namespace Org.Apache.REEF.Client.Common
             _client.DefaultRequestHeaders.Accept.Add(new MediaTypeWithQualityHeaderValue(AppJson));
 
             _driverUrl = GetDriverUrl(filePath);
+
+            _numberOfRetries = numberOfRetries;
+            _retryInterval = TimeSpan.FromMilliseconds(retryInterval);
         }
 
         /// <summary>
@@ -98,7 +118,13 @@ namespace Org.Apache.REEF.Client.Common
 
         public void WaitForDriverToFinish()
         {
-            DriverStatus status = FetchDriverStatus();
+            DriverStatus status = FetchFirstDriverStatus();
+
+            if (DriverStatus.UNKNOWN == status)
+            {
+                // We were unable to connect to the Driver at least once.
+                throw new WebException("Unable to connect to the Driver.");
+            }
             
             while (status.IsActive())
             {
@@ -106,7 +132,7 @@ namespace Org.Apache.REEF.Client.Common
                 {
                     status = FetchDriverStatus();
                 }
-                catch (System.Net.WebException)
+                catch (WebException)
                 {
                     // If we no longer can reach the Driver, it must have exited.
                     status = DriverStatus.UNKNOWN_EXITED;
@@ -126,6 +152,16 @@ namespace Org.Apache.REEF.Client.Common
             }
         }
 
+        /// <summary>
+        /// Fetches the Driver Status for the 1st time.
+        /// </summary>
+        /// <returns>The obtained Driver Status or DriverStatus.UNKNOWN, if the Driver was never reached.</returns>
+        private DriverStatus FetchFirstDriverStatus()
+        {
+            var policy = new RetryPolicy<AllErrorsTransientStrategy>(_numberOfRetries, _retryInterval);
+            return policy.ExecuteAction<DriverStatus>(FetchDriverStatus);
+        }
+
         protected abstract string GetDriverUrl(string filepath);
 
         enum UrlResultKind

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/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 722bf9b..9f45986 100644
--- a/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalClient.cs
@@ -37,6 +37,7 @@ using Org.Apache.REEF.Utilities.Logging;
 using Org.Apache.REEF.Wake.Remote;
 using Org.Apache.REEF.Wake.Remote.Impl;
 using Org.Apache.REEF.Wake.Remote.Parameters;
+using Org.Apache.REEF.Client.API.Parameters;
 
 namespace Org.Apache.REEF.Client.Local
 {
@@ -64,10 +65,22 @@ namespace Org.Apache.REEF.Client.Local
         private readonly IConfiguration _localConfigurationOnDriver;
         private readonly JobRequestBuilderFactory _jobRequestBuilderFactory;
 
+        /// <summary>
+        /// Number of retries when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _numberOfRetries;
+
+        /// <summary>
+        /// Retry interval in ms when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _retryInterval;
+
         [Inject]
         private LocalClient(DriverFolderPreparationHelper driverFolderPreparationHelper,
             [Parameter(typeof(LocalRuntimeDirectory))] string runtimeFolder,
             [Parameter(typeof(NumberOfEvaluators))] int maxNumberOfConcurrentEvaluators,
+            [Parameter(typeof(DriverHTTPConnectionRetryInterval))]int retryInterval,
+            [Parameter(typeof(DriverHTTPConnectionAttempts))] int numberOfRetries,
             IJavaClientLauncher javaClientLauncher,
             REEFFileNames fileNames,
             JobRequestBuilderFactory jobRequestBuilderFactory)
@@ -75,6 +88,8 @@ namespace Org.Apache.REEF.Client.Local
             _driverFolderPreparationHelper = driverFolderPreparationHelper;
             _runtimeFolder = runtimeFolder;
             _maxNumberOfConcurrentEvaluators = maxNumberOfConcurrentEvaluators;
+            _retryInterval = retryInterval;
+            _numberOfRetries = numberOfRetries;
             _javaClientLauncher = javaClientLauncher;
             _fileNames = fileNames;
             _jobRequestBuilderFactory = jobRequestBuilderFactory;
@@ -95,11 +110,13 @@ namespace Org.Apache.REEF.Client.Local
         private LocalClient(
             DriverFolderPreparationHelper driverFolderPreparationHelper,
             [Parameter(typeof(NumberOfEvaluators))] int numberOfEvaluators,
+            [Parameter(typeof(DriverHTTPConnectionRetryInterval))]int retryInterval,
+            [Parameter(typeof(DriverHTTPConnectionAttempts))] int numberOfRetries,
             IJavaClientLauncher javaClientLauncher,
             REEFFileNames fileNames,
             JobRequestBuilderFactory jobRequestBuilderFactory)
             : this(driverFolderPreparationHelper, Path.GetTempPath(),
-                numberOfEvaluators, javaClientLauncher, fileNames, jobRequestBuilderFactory)
+                numberOfEvaluators, retryInterval, numberOfRetries, javaClientLauncher, fileNames, jobRequestBuilderFactory)
         {
             // Intentionally left blank.
         }
@@ -193,7 +210,7 @@ namespace Org.Apache.REEF.Client.Local
                 .LogAndIgnoreExceptionIfAny(Logger, "Java launcher failed");
 
             var fileName = Path.Combine(driverFolder, _fileNames.DriverHttpEndpoint);
-            JobSubmissionResult result = new LocalJobSubmissionResult(this, fileName);
+            JobSubmissionResult result = new LocalJobSubmissionResult(this, fileName, _numberOfRetries, _retryInterval);
 
             var msg = string.Format(CultureInfo.CurrentCulture,
                 "Submitted the Driver for execution. Returned driverUrl is: {0}.", result.DriverUrl);

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs
index 51d7afd..f5f50c0 100644
--- a/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Local/LocalJobSubmissionResult.cs
@@ -34,8 +34,11 @@ namespace Org.Apache.REEF.Client.Local
 
         private const string UriTemplate = @"http://{0}/";
 
-        internal LocalJobSubmissionResult(IREEFClient reefClient, string filePath) 
-            : base(reefClient, filePath)
+        internal LocalJobSubmissionResult(IREEFClient reefClient, 
+            string filePath, 
+            int numberOfRetries, 
+            int retryInterval) 
+            : base(reefClient, filePath, numberOfRetries, retryInterval)
         {
         }
 

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/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 80b8d90..879b8c3 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
@@ -74,6 +74,7 @@ under the License.
     <Compile Include="API\JobRequest.cs" />
     <Compile Include="API\JobRequestBuilder.cs" />
     <Compile Include="API\JobRequestBuilderFactory.cs" />
+    <Compile Include="API\Parameters.cs" />
     <Compile Include="API\TcpPortConfigurationModule.cs" />
     <Compile Include="Avro\AvroAppSubmissionParameters.cs" />
     <Compile Include="Avro\AvroJobSubmissionParameters.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
index 09434f0..554914a 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YARNREEFClient.cs
@@ -31,6 +31,8 @@ using Org.Apache.REEF.Common.Files;
 using Org.Apache.REEF.Tang.Annotations;
 using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Client.API.Parameters;
+using Org.Apache.REEF.Client.Local.Parameters;
 
 namespace Org.Apache.REEF.Client.Yarn
 {
@@ -49,6 +51,16 @@ namespace Org.Apache.REEF.Client.Yarn
         private readonly YarnREEFParamSerializer _paramSerializer;
         private readonly JobRequestBuilderFactory _jobRequestBuilderFactory;
 
+        /// <summary>
+        /// Number of retries when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _numberOfRetries;
+
+        /// <summary>
+        /// Retry interval in ms when connecting to the Driver's HTTP endpoint.
+        /// </summary>
+        private readonly int _retryInterval;
+
         [Inject]
         internal YarnREEFClient(IJavaClientLauncher javaClientLauncher,
             DriverFolderPreparationHelper driverFolderPreparationHelper,
@@ -56,7 +68,9 @@ namespace Org.Apache.REEF.Client.Yarn
             YarnCommandLineEnvironment yarn,
             IYarnRMClient yarnClient,
             YarnREEFParamSerializer paramSerializer,
-            JobRequestBuilderFactory jobRequestBuilderFactory)
+            JobRequestBuilderFactory jobRequestBuilderFactory,
+            [Parameter(typeof(DriverHTTPConnectionRetryInterval))]int retryInterval,
+            [Parameter(typeof(DriverHTTPConnectionAttempts))] int numberOfRetries)
         {
             _javaClientLauncher = javaClientLauncher;
             _javaClientLauncher.AddToClassPath(yarn.GetYarnClasspathList());
@@ -65,6 +79,8 @@ namespace Org.Apache.REEF.Client.Yarn
             _yarnClient = yarnClient;
             _paramSerializer = paramSerializer;
             _jobRequestBuilderFactory = jobRequestBuilderFactory;
+            _retryInterval = retryInterval;
+            _numberOfRetries = numberOfRetries;
         }
 
         public void Submit(JobRequest jobRequest)
@@ -90,7 +106,8 @@ namespace Org.Apache.REEF.Client.Yarn
             Launch(jobRequest, driverFolderPath);
 
             var pointerFileName = Path.Combine(driverFolderPath, _fileNames.DriverHttpEndpoint);
-            var jobSubmitionResultImpl = new YarnJobSubmissionResult(this, pointerFileName);
+            var jobSubmitionResultImpl = new YarnJobSubmissionResult(this, 
+                pointerFileName, _numberOfRetries, _retryInterval);
 
             var msg = string.Format(CultureInfo.CurrentCulture,
                 "Submitted the Driver for execution. Returned driverUrl is: {0}, appId is {1}.",

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs
index df9fd8d..617874a 100644
--- a/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/YARN/YarnJobSubmissionResult.cs
@@ -24,8 +24,11 @@ namespace Org.Apache.REEF.Client.YARN
 {
     internal class YarnJobSubmissionResult : JobSubmissionResult
     {
-        internal YarnJobSubmissionResult(IREEFClient reefClient, string filePath) 
-            : base(reefClient, filePath)
+        internal YarnJobSubmissionResult(IREEFClient reefClient, 
+            string filePath,
+            int numberOfRetries,
+            int retryInterval) 
+            : base(reefClient, filePath, numberOfRetries, retryInterval)
         {
         }
 

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
index 058f565..c2c2b87 100644
--- a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
@@ -18,8 +18,14 @@
  */
 package org.apache.reef.bridge.client;
 
+import org.apache.reef.bridge.client.Parameters.HTTPStatusAlarmInterval;
+import org.apache.reef.bridge.client.Parameters.HTTPStatusNumberOfRetries;
 import org.apache.reef.proto.ReefServiceProtos;
 import org.apache.reef.runtime.common.driver.client.JobStatusHandler;
+import org.apache.reef.tang.annotations.Parameter;
+import org.apache.reef.wake.EventHandler;
+import org.apache.reef.wake.time.Clock;
+import org.apache.reef.wake.time.event.Alarm;
 import org.apache.reef.webserver.HttpHandler;
 import org.apache.reef.webserver.ParsedHttpRequest;
 
@@ -52,8 +58,49 @@ final class DriverStatusHTTPHandler implements HttpHandler, JobStatusHandler {
    */
   private ReefServiceProtos.JobStatusProto lastStatus = null;
 
+  /**
+   * The clock is used to schedule a check whether the handler has been called.
+   */
+  private final Clock clock;
+
+  /**
+   * The maximum number of times the AlarmHandler will be scheduled.
+   */
+  private final int maxNumberOfRetries;
+
+  /**
+   * The interval between alarms.
+   */
+  private final int alarmInterval;
+
+  /**
+   * The current retry.
+   */
+  private int retry = 0;
+
+  /**
+   * The alarm handler to keep the Clock alive until the status has been requested once.
+   */
+  private final EventHandler<Alarm> alarmHandler = new EventHandler<Alarm>() {
+    @Override
+    public void onNext(final Alarm value) {
+      scheduleAlarm();
+    }
+  };
+
+  /**
+   * Whether or not this handler was called at least once via HTTP.
+   */
+  private boolean wasCalledViaHTTP = false;
+
   @Inject
-  DriverStatusHTTPHandler(){
+  DriverStatusHTTPHandler(final Clock clock,
+                          @Parameter(HTTPStatusNumberOfRetries.class) final int maxNumberOfRetries,
+                          @Parameter(HTTPStatusAlarmInterval.class) final int alarmInterval) {
+    this.clock = clock;
+    this.maxNumberOfRetries = maxNumberOfRetries;
+    this.alarmInterval = alarmInterval;
+    scheduleAlarm();
   }
 
   @Override
@@ -71,6 +118,7 @@ final class DriverStatusHTTPHandler implements HttpHandler, JobStatusHandler {
       throws IOException, ServletException {
     try (final PrintWriter writer = response.getWriter()) {
       writer.write(waitAndGetMessage());
+      this.wasCalledViaHTTP = true;
     }
   }
 
@@ -126,4 +174,24 @@ final class DriverStatusHTTPHandler implements HttpHandler, JobStatusHandler {
   static String getMessageForStatus(final ReefServiceProtos.JobStatusProto status) {
     return status.getState().name();
   }
+
+  /**
+   * Schedules an alarm, if needed.
+   * <p>
+   * The alarm will prevent the Clock from going idle. This gives the .NET Client time to make a call to this HTTP
+   * handler.
+   */
+  private void scheduleAlarm() {
+    if (wasCalledViaHTTP || retry >= maxNumberOfRetries) {
+      // No alarm necessary anymore.
+      LOG.log(Level.INFO,
+          "Not scheduling additional alarms after {0} out of max {1} retries. The HTTP handles was called: ",
+          new Object[] {retry, maxNumberOfRetries, wasCalledViaHTTP});
+      return;
+    }
+
+    // Scheduling an alarm will prevent the clock from going idle.
+    ++retry;
+    clock.scheduleAlarm(alarmInterval, alarmHandler);
+  }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusAlarmInterval.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusAlarmInterval.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusAlarmInterval.java
new file mode 100644
index 0000000..9f600b6
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusAlarmInterval.java
@@ -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.
+ */
+package org.apache.reef.bridge.client.Parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * The interval between alarms in DriverStatusHTTPHandler.
+ */
+@NamedParameter(default_value = "200", doc = "The interval between alarms in DriverStatusHTTPHandler.")
+public final class HTTPStatusAlarmInterval implements Name<Integer> {
+
+  private HTTPStatusAlarmInterval() {
+    //intentionally empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusNumberOfRetries.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusNumberOfRetries.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusNumberOfRetries.java
new file mode 100644
index 0000000..6f43cb6
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/HTTPStatusNumberOfRetries.java
@@ -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.
+ */
+package org.apache.reef.bridge.client.Parameters;
+
+import org.apache.reef.tang.annotations.Name;
+import org.apache.reef.tang.annotations.NamedParameter;
+
+/**
+ * Number of times the HTTPStatusHandler will advance its alarm.
+ */
+@NamedParameter(default_value = "10", doc = "Number of times the HTTPStatusHandler will advance its alarm.")
+public final class HTTPStatusNumberOfRetries implements Name<Integer> {
+
+  private HTTPStatusNumberOfRetries() {
+    // Intentionally empty.
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/package-info.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/package-info.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/package-info.java
new file mode 100644
index 0000000..48e9af2
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Parameters/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * Named Parameters of the Java side of the .NET Client.
+ */
+package org.apache.reef.bridge.client.Parameters;

http://git-wip-us.apache.org/repos/asf/reef/blob/3d571e6a/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java b/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
index 222832a..8b1b126 100644
--- a/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
+++ b/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
@@ -19,6 +19,8 @@
 package org.apache.reef.bridge.client;
 
 import org.apache.reef.proto.ReefServiceProtos;
+import org.apache.reef.tang.Tang;
+import org.apache.reef.tang.exceptions.InjectionException;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -71,8 +73,8 @@ public final class TestDriverStatusHTTPHandler {
    * {@link org.apache.reef.runtime.common.driver.client.JobStatusHandler}.
    */
   @Test
-  public void testLastStatus() {
-    final DriverStatusHTTPHandler tester = new DriverStatusHTTPHandler();
+  public void testLastStatus() throws InjectionException {
+    final DriverStatusHTTPHandler tester = getInstance();
 
     for (final ReefServiceProtos.JobStatusProto status : allStatuses) {
       tester.onNext(status);
@@ -84,8 +86,8 @@ public final class TestDriverStatusHTTPHandler {
    * Test the wait and notify for correctness.
    */
   @Test
-  public void testAsyncCalls() throws InterruptedException {
-    final DriverStatusHTTPHandler tester = new DriverStatusHTTPHandler();
+  public void testAsyncCalls() throws InterruptedException, InjectionException {
+    final DriverStatusHTTPHandler tester = getInstance();
 
     final WaitingRunnable waiter = new WaitingRunnable(tester);
 
@@ -100,6 +102,10 @@ public final class TestDriverStatusHTTPHandler {
     }
   }
 
+  private static DriverStatusHTTPHandler getInstance() throws InjectionException {
+    return Tang.Factory.getTang().newInjector().getInstance(DriverStatusHTTPHandler.class);
+  }
+
   private final class WaitingRunnable implements Runnable {
     private final DriverStatusHTTPHandler handler;
     private String result = null;


[17/28] reef git commit: [REEF-1958] Set logging level to Verbose in all processes of HelloREEF example

Posted by do...@apache.org.
[REEF-1958] Set logging level to Verbose in all processes of HelloREEF example

Set logging level to `Verbose` in HelloREEF (local and YARN versions)
for the Driver and the Evaluators

+ better logging and some cleanup in the HelloREEF C# driver to make
sure that setting logging levels works properly

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

Pull Request:
  This closes #1417


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

Branch: refs/heads/REEF-335
Commit: 821ef2a2206b4b615d0289b655d57c259196b653
Parents: aeb730a
Author: Sergiy Matusevych <mo...@apache.org>
Authored: Fri Nov 10 16:31:32 2017 -0800
Committer: Markus Weimer <we...@apache.org>
Committed: Tue Nov 14 10:12:00 2017 -0800

----------------------------------------------------------------------
 .../Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs  | 12 ++++++++----
 .../cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs |  5 +++--
 .../HelloREEFYarn.cs                                   | 13 +++++++------
 3 files changed, 18 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/821ef2a2/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs
index fbac3d0..2a0a8da 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloDriver.cs
@@ -42,13 +42,17 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         /// <summary>
         /// Submits the HelloTask to the Evaluator.
         /// </summary>
-        /// <param name="allocatedEvaluator"></param>
+        /// <param name="allocatedEvaluator">Newly allocated evaluator's proxy object.</param>
         public void OnNext(IAllocatedEvaluator allocatedEvaluator)
         {
+            _Logger.Log(Level.Info, "Evaluator allocated: {0}", allocatedEvaluator);
+
             var taskConfiguration = TaskConfiguration.ConfigurationModule
                 .Set(TaskConfiguration.Identifier, "HelloTask")
                 .Set(TaskConfiguration.Task, GenericType<HelloTask>.Class)
                 .Build();
+
+            _Logger.Log(Level.Verbose, "Submit task: {0}", taskConfiguration);
             allocatedEvaluator.SubmitTask(taskConfiguration);
         }
 
@@ -62,12 +66,12 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         }
 
         /// <summary>
-        /// Called to start the user mode driver
+        /// Called to start the user mode driver.
         /// </summary>
-        /// <param name="driverStarted"></param>
+        /// <param name="driverStarted">Notification that the Driver is up and running.</param>
         public void OnNext(IDriverStarted driverStarted)
         {
-            _Logger.Log(Level.Info, string.Format("HelloDriver started at {0}", driverStarted.StartTime));
+            _Logger.Log(Level.Info, "HelloDriver started at {0}", driverStarted.StartTime);
             _evaluatorRequestor.Submit(_evaluatorRequestor.NewBuilder().SetMegabytes(64).Build());
         }
     }

http://git-wip-us.apache.org/repos/asf/reef/blob/821ef2a2/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 e29ea0c..8f9053d 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
@@ -57,6 +57,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
             var helloDriverConfiguration = DriverConfiguration.ConfigurationModule
                 .Set(DriverConfiguration.OnEvaluatorAllocated, GenericType<HelloDriver>.Class)
                 .Set(DriverConfiguration.OnDriverStarted, GenericType<HelloDriver>.Class)
+                .Set(DriverConfiguration.CustomTraceLevel, Level.Verbose.ToString())
                 .Build();
 
             // The JobSubmission contains the Driver configuration as well as the files needed on the Driver.
@@ -91,7 +92,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
                     return YARNClientConfiguration.ConfigurationModuleYARNRest.Build();
                 case HDInsight:
                     // To run against HDInsight please replace placeholders below, with actual values for
-                    // connection string, container name (available at Azure portal) and HDInsight 
+                    // connection string, container name (available at Azure portal) and HDInsight
                     // credentials (username and password)
                     const string connectionString = "ConnString";
                     const string continerName = "foo";
@@ -112,4 +113,4 @@ namespace Org.Apache.REEF.Examples.HelloREEF
             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/reef/blob/821ef2a2/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
index e7dfcf0..754de62 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEFYarn.cs
@@ -61,7 +61,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
         private readonly IList<string> _nodeNames;
 
         [Inject]
-        private HelloREEFYarn(IYarnREEFClient reefClient, 
+        private HelloREEFYarn(IYarnREEFClient reefClient,
             JobRequestBuilder jobRequestBuilder,
             [Parameter(typeof(NodeNames))] ISet<string> nodeNames)
         {
@@ -78,7 +78,8 @@ namespace Org.Apache.REEF.Examples.HelloREEF
             // The driver configuration contains all the needed handler bindings
             var helloDriverConfiguration = DriverConfiguration.ConfigurationModule
                 .Set(DriverConfiguration.OnEvaluatorAllocated, GenericType<HelloDriverYarn>.Class)
-                .Set(DriverConfiguration.OnDriverStarted, GenericType<HelloDriverYarn>.Class)              
+                .Set(DriverConfiguration.OnDriverStarted, GenericType<HelloDriverYarn>.Class)
+                .Set(DriverConfiguration.CustomTraceLevel, Level.Verbose.ToString())
                 .Build();
 
             var driverConfig = TangFactory.GetTang()
@@ -88,7 +89,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
             {
                 driverConfig.BindSetEntry<NodeNames, string>(GenericType<NodeNames>.Class, n);
             }
-            
+
             // The JobSubmission contains the Driver configuration as well as the files needed on the Driver.
             var helloJobRequest = _jobRequestBuilder
                 .AddDriverConfiguration(driverConfig.Build())
@@ -133,8 +134,8 @@ namespace Org.Apache.REEF.Examples.HelloREEF
             }
             else
             {
-                Logger.Log(Level.Info, 
-                    "Failed to kill application {0}, possible reasons are application id is invalid or application has completed.", 
+                Logger.Log(Level.Info,
+                    "Failed to kill application {0}, possible reasons are application id is invalid or application has completed.",
                     appId);
             }
         }
@@ -200,4 +201,4 @@ namespace Org.Apache.REEF.Examples.HelloREEF
             TangFactory.GetTang().NewInjector(configBuilder.Build()).GetInstance<HelloREEFYarn>().Run();
         }
     }
-}
\ No newline at end of file
+}


[13/28] reef git commit: [REEF-1947] xUnit error in TestHelloREEFClient

Posted by do...@apache.org.
[REEF-1947] xUnit error in TestHelloREEFClient

The test is for yarn and called from a test console. So remove `[Fact]` for now.

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

This closes #1410


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

Branch: refs/heads/REEF-335
Commit: e36ee921d77ca6b88512ad1b8f3a1ad9752ed939
Parents: 7b81827
Author: jwang98052 <ju...@apache.org>
Authored: Thu Nov 2 20:32:27 2017 -0700
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Fri Nov 3 18:54:46 2017 -0700

----------------------------------------------------------------------
 .../Performance/TestHelloREEF/TestHelloREEFClient.cs         | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/e36ee921/lang/cs/Org.Apache.REEF.Tests/Performance/TestHelloREEF/TestHelloREEFClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Performance/TestHelloREEF/TestHelloREEFClient.cs b/lang/cs/Org.Apache.REEF.Tests/Performance/TestHelloREEF/TestHelloREEFClient.cs
index 454cd32..babdb2d 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Performance/TestHelloREEF/TestHelloREEFClient.cs
+++ b/lang/cs/Org.Apache.REEF.Tests/Performance/TestHelloREEF/TestHelloREEFClient.cs
@@ -39,7 +39,7 @@ namespace Org.Apache.REEF.Tests.Performance.TestHelloREEF
     /// <summary>
     /// Test Hello REEF for scalability
     /// </summary>
-    [Collection("PerformanceTests")]
+    [Collection("FunctionalTests")]
     public class TestHelloREEFClient : ReefFunctionalTest
     {
         private static readonly Logger Logger = Logger.GetLogger(typeof(TestHelloREEFClient));
@@ -68,16 +68,16 @@ namespace Org.Apache.REEF.Tests.Performance.TestHelloREEF
 
         /// <summary>
         /// Test HelloREEF on YARN. 
-        /// The parameter are provided on command line arguments: token password numberOfContainers
+        /// The test can be modified to pass parameters through command line arguments: token password numberOfContainers
         /// e.g. TestDriver.exe TrustedApplication001 none 2000 
         /// </summary>
-        /// <param name="args"></param>
         [Fact]
         [Trait("Environment", "Yarn")]
         [Trait("Priority", "1")]
         [Trait("Description", "Run CLR Test on Yarn")]
-        public void TestHelloREEFOnYarn(string[] args)
+        public void TestHelloREEFOnYarn()
         {
+            string[] args = { "TrustedApplication001", "none", "2000" };
             TestRun(GetRuntimeConfigurationForYarn(args), 10240);
         }
 


[04/28] reef git commit: [REEF-1937] Use Verbose log level in C# HelloREEF example

Posted by do...@apache.org.
[REEF-1937] Use Verbose log level in C# HelloREEF example

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

Pull Request:
  This closes #1403


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

Branch: refs/heads/REEF-335
Commit: e90ffbee09919aa6cfadc999cc82843b331e2445
Parents: 48e65c0
Author: Sergiy Matusevych <mo...@apache.org>
Authored: Fri Oct 27 15:14:39 2017 -0700
Committer: Markus Weimer <we...@apache.org>
Committed: Sun Oct 29 08:52:33 2017 -0700

----------------------------------------------------------------------
 lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/e90ffbee/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 20c80fa..1164719 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
@@ -26,6 +26,7 @@ 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.Utilities.Logging;
 
 namespace Org.Apache.REEF.Examples.HelloREEF
 {
@@ -64,6 +65,7 @@ namespace Org.Apache.REEF.Examples.HelloREEF
                 .AddDriverConfiguration(helloDriverConfiguration)
                 .AddGlobalAssemblyForType(typeof(HelloDriver))
                 .SetJobIdentifier("HelloREEF")
+                .SetJavaLogLevel(JavaLoggingSetting.Verbose)
                 .Build();
 
             _reefClient.Submit(helloJobRequest);


[10/28] reef git commit: [REEF-1942] Add ability for .NET Client to wait for Driver to complete.

Posted by do...@apache.org.
[REEF-1942] Add ability for .NET Client to wait for Driver to complete.

  * Added `IJobSubmissionResult.WaitForDriverToFinish()` which blocks
    until the Driver is done. This uses a new HTTP handler on the Java
    side of the Driver.
  * Added `DriverStatusHTTPHandler` to REEF / Java to query the Driver's
    status via HTTP and added that Handler to the Driver Configuration
    used by REEF.NET.
  * Added a call to block at the end of HelloREEF.

This change also adds a bunch of infrastructure to make this more
informative in the future, e.g. `DriverStatus` in .NET. To keep this
focused, I did not expose that machinery.

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

Pull Request:
  This closes #1408


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

Branch: refs/heads/REEF-335
Commit: fece629495a8a4946136577658d3d81356166354
Parents: bee789a
Author: Markus Weimer <we...@apache.org>
Authored: Wed Nov 1 10:28:14 2017 -0700
Committer: jwang98052 <ju...@apache.org>
Committed: Thu Nov 2 18:05:44 2017 -0700

----------------------------------------------------------------------
 .../Org.Apache.REEF.Client/API/DriverStatus.cs  |  96 ++++++++++++++
 .../Common/IJobSubmissionResult.cs              |   7 +
 .../Common/JobSubmissionResult.cs               |  40 +++++-
 .../Org.Apache.REEF.Client.csproj               |   1 +
 .../HelloREEF.cs                                |   6 +-
 .../apache/reef/bridge/client/Constants.java    |  10 ++
 .../bridge/client/DriverStatusHTTPHandler.java  | 129 +++++++++++++++++++
 .../client/TestDriverStatusHTTPHandler.java     | 122 ++++++++++++++++++
 8 files changed, 405 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/fece6294/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs b/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
new file mode 100644
index 0000000..d9934a9
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Client/API/DriverStatus.cs
@@ -0,0 +1,96 @@
+// 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>
+    /// Represents the status of a Driver.
+    /// </summary>
+    internal enum DriverStatus
+    {
+        /// <summary>
+        /// Driver is initializing.
+        /// </summary>
+        INIT,
+
+        /// <summary>
+        /// Driver is running.
+        /// </summary>
+        RUNNING,
+
+        /// <summary>
+        /// Driver has cleanly exited.
+        /// </summary>
+        DONE,
+
+        /// <summary>
+        /// Driver is suspended.
+        /// </summary>
+        SUSPEND,
+
+        /// <summary>
+        /// Driver was killed.
+        /// </summary>
+        KILLED,
+
+        /// <summary>
+        /// Driver Failed.
+        /// </summary>
+        FAILED,
+
+        /// <summary>
+        /// Driver was RUNNING, but is no longer reachable for unknown reasons.
+        /// </summary>
+        UNKNOWN_EXITED
+    }
+
+    /// <summary>
+    /// Extension methods for DriverStatus
+    /// </summary>
+    internal static class DriverStatusMethods
+    {
+        /// <summary>
+        /// Indicates that the Driver is active: INIT or RUNNING.
+        /// </summary>
+        /// <param name="status"></param>
+        /// <returns>Whether the Driver is active.</returns>
+        public static bool IsActive(this DriverStatus status)
+        {
+            return status == DriverStatus.RUNNING || status == DriverStatus.INIT;
+        }
+
+        /// <summary>
+        /// Indicates that the Driver is not active (INIT or RUNNING).
+        /// </summary>
+        /// <param name="status"></param>
+        /// <returns>Whether the driver is not active.</returns>
+        public static bool IsNotActive(this DriverStatus status)
+        {
+            return !IsActive(status);
+        }
+
+        /// <summary>
+        /// Parses a string representation of a DriverStatus.
+        /// </summary>
+        /// <param name="statusString">The string to parse.</param>
+        /// <returns>The DriverStatus represented in the string.</returns>
+        public static DriverStatus Parse(string statusString)
+        {
+            return (DriverStatus)System.Enum.Parse(typeof(DriverStatus), statusString);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/fece6294/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
index 325d011..328677a 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/IJobSubmissionResult.cs
@@ -45,5 +45,12 @@ namespace Org.Apache.REEF.Client.Common
         /// Get Yarn application id after Job is submited
         /// </summary>
         string AppId { get; }
+
+        /// <summary>
+        /// Waits for the Driver to complete.
+        /// </summary>
+        /// <exception cref="System.Net.WebException">If the Driver cannot be reached.</exception>
+        [Unstable("0.17", "Uses the HTTP server in the Java Driver. Might not work if that cannot be reached.")]
+        void WaitForDriverToFinish();
     }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/fece6294/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs b/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
index 7c8a7d4..3880433 100644
--- a/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
+++ b/lang/cs/Org.Apache.REEF.Client/Common/JobSubmissionResult.cs
@@ -63,7 +63,7 @@ namespace Org.Apache.REEF.Client.Common
         /// <summary>
         /// Returns http end point of the web server running in the driver
         /// </summary>
-        public string DriverUrl 
+        public string DriverUrl
         {
             get { return _driverUrl; }
         }
@@ -96,6 +96,36 @@ namespace Org.Apache.REEF.Client.Common
             return task.Result;
         }
 
+        public void WaitForDriverToFinish()
+        {
+            DriverStatus status = FetchDriverStatus();
+            
+            while (status.IsActive())
+            {
+                try
+                {
+                    status = FetchDriverStatus();
+                }
+                catch (System.Net.WebException)
+                {
+                    // If we no longer can reach the Driver, it must have exited.
+                    status = DriverStatus.UNKNOWN_EXITED;
+                }
+            }
+        }
+
+        private DriverStatus FetchDriverStatus()
+        {
+            string statusUrl = DriverUrl + "driverstatus/v1";
+            HttpWebRequest request = (HttpWebRequest)WebRequest.Create(statusUrl);
+            using (StreamReader reader = new StreamReader(request.GetResponse().GetResponseStream()))
+            {
+                string statusString = reader.ReadToEnd();
+                LOGGER.Log(Level.Verbose, "Status received: {0}", statusString);
+                return DriverStatusMethods.Parse(statusString);
+            }
+        }
+
         protected abstract string GetDriverUrl(string filepath);
 
         enum UrlResultKind
@@ -116,7 +146,7 @@ namespace Org.Apache.REEF.Client.Common
                 var rmList = new List<string>();
                 var rmUri = sr.ReadLine();
                 while (rmUri != null)
-                {                    
+                {
                     rmList.Add(rmUri);
                     rmUri = sr.ReadLine();
                 }
@@ -134,12 +164,12 @@ namespace Org.Apache.REEF.Client.Common
             LOGGER.Log(Level.Warning, "CallUrl result " + result.Item2);
             return result.Item2;
         }
-        
+
         internal async Task<string> GetAppIdTrackingUrl(string url)
         {
             var result = await TryGetUri(url);
-            if (HasCommandFailed(result) ||  
-                result.Item2 == null)                
+            if (HasCommandFailed(result) ||
+                result.Item2 == null)
             {
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/reef/blob/fece6294/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 a0b4f75..00c6d87 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
@@ -65,6 +65,7 @@ under the License.
     <Compile Include="API\AppParameters.cs" />
     <Compile Include="API\AppParametersBuilder.cs" />
     <Compile Include="API\ClientFactory.cs" />
+    <Compile Include="API\DriverStatus.cs" />
     <Compile Include="API\Exceptions\ClasspathException.cs" />
     <Compile Include="API\Exceptions\JavaNotFoundException.cs" />
     <Compile Include="API\IREEFClient.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/fece6294/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 1164719..b1d0cd3 100644
--- a/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
+++ b/lang/cs/Org.Apache.REEF.Examples.HelloREEF/HelloREEF.cs
@@ -27,6 +27,7 @@ using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Tang.Interface;
 using Org.Apache.REEF.Tang.Util;
 using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Client.Common;
 
 namespace Org.Apache.REEF.Examples.HelloREEF
 {
@@ -68,7 +69,10 @@ namespace Org.Apache.REEF.Examples.HelloREEF
                 .SetJavaLogLevel(JavaLoggingSetting.Verbose)
                 .Build();
 
-            _reefClient.Submit(helloJobRequest);
+            IJobSubmissionResult jobSubmissionResult = _reefClient.SubmitAndGetJobStatus(helloJobRequest);
+
+            // Wait for the Driver to complete.
+            jobSubmissionResult.WaitForDriverToFinish();
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/reef/blob/fece6294/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Constants.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Constants.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Constants.java
index ac41f9f..0eaef00 100644
--- a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Constants.java
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/Constants.java
@@ -23,8 +23,10 @@ import org.apache.reef.client.DriverServiceConfiguration;
 import org.apache.reef.client.DriverRestartConfiguration;
 import org.apache.reef.io.network.naming.NameServerConfiguration;
 import org.apache.reef.javabridge.generic.JobDriver;
+import org.apache.reef.runtime.common.driver.client.JobStatusHandler;
 import org.apache.reef.tang.Configuration;
 import org.apache.reef.tang.Configurations;
+import org.apache.reef.tang.Tang;
 import org.apache.reef.webserver.HttpHandlerConfiguration;
 import org.apache.reef.webserver.HttpServerReefEventHandler;
 import org.apache.reef.webserver.ReefEventStateManager;
@@ -34,6 +36,8 @@ import org.apache.reef.webserver.ReefEventStateManager;
  */
 public final class Constants {
 
+  private static final Tang TANG = Tang.Factory.getTang();
+
   /**
    * Contains all bindings of event handlers to the bridge.
    */
@@ -60,6 +64,8 @@ public final class Constants {
   public static final Configuration HTTP_SERVER_CONFIGURATION = Configurations.merge(
       HttpHandlerConfiguration.CONF
           .set(HttpHandlerConfiguration.HTTP_HANDLERS, HttpServerReefEventHandler.class)
+          // Add the http status handler.
+          .set(HttpHandlerConfiguration.HTTP_HANDLERS, DriverStatusHTTPHandler.class)
           .build(),
       DriverServiceConfiguration.CONF
           .set(DriverServiceConfiguration.ON_EVALUATOR_ALLOCATED,
@@ -76,6 +82,10 @@ public final class Constants {
               ReefEventStateManager.DriverRestartActiveContextStateHandler.class)
           .set(DriverRestartConfiguration.ON_DRIVER_RESTART_TASK_RUNNING,
               ReefEventStateManager.DriverRestartTaskRunningStateHandler.class)
+          .build(),
+      // Bind the HTTP handler for job status
+      TANG.newConfigurationBuilder()
+          .bindImplementation(JobStatusHandler.class, DriverStatusHTTPHandler.class)
           .build()
   );
 

http://git-wip-us.apache.org/repos/asf/reef/blob/fece6294/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
new file mode 100644
index 0000000..058f565
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/main/java/org/apache/reef/bridge/client/DriverStatusHTTPHandler.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.bridge.client;
+
+import org.apache.reef.proto.ReefServiceProtos;
+import org.apache.reef.runtime.common.driver.client.JobStatusHandler;
+import org.apache.reef.webserver.HttpHandler;
+import org.apache.reef.webserver.ParsedHttpRequest;
+
+import javax.inject.Inject;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+final class DriverStatusHTTPHandler implements HttpHandler, JobStatusHandler {
+
+  private static final Logger LOG = Logger.getLogger(DriverStatusHTTPHandler.class.getName());
+
+  /**
+   * The URI under which this handler answers.
+   */
+  private String uriSpecification = "driverstatus";
+
+  /**
+   * A queue of messages to be sent to the client.
+   */
+  private final Queue<ReefServiceProtos.JobStatusProto> statusMessagesToSend = new LinkedList<>();
+
+  /**
+   * The last status received by this object in its role as JobStatusHandler.
+   */
+  private ReefServiceProtos.JobStatusProto lastStatus = null;
+
+  @Inject
+  DriverStatusHTTPHandler(){
+  }
+
+  @Override
+  public String getUriSpecification() {
+    return uriSpecification;
+  }
+
+  @Override
+  public void setUriSpecification(final String newUriSpecification) {
+    this.uriSpecification = newUriSpecification;
+  }
+
+  @Override
+  public void onHttpRequest(final ParsedHttpRequest parsedHttpRequest, final HttpServletResponse response)
+      throws IOException, ServletException {
+    try (final PrintWriter writer = response.getWriter()) {
+      writer.write(waitAndGetMessage());
+    }
+  }
+
+  @Override
+  public void onNext(final ReefServiceProtos.JobStatusProto value) {
+    LOG.log(Level.INFO, "Received status: {0}", value.getState().name());
+    // Record the status received and notify the thread to send an answer.
+    synchronized (this) {
+      this.statusMessagesToSend.add(value);
+      this.lastStatus = value;
+      this.notifyAll();
+    }
+  }
+
+  @Override
+  public ReefServiceProtos.JobStatusProto getLastStatus() {
+    return this.lastStatus;
+  }
+
+  @Override
+  public String toString() {
+    return "DriverStatusHTTPHandler{uriSpec=" + getUriSpecification() + "}";
+  }
+
+  /**
+   * Waits for a status message to be available and returns it.
+   *
+   * @return the first available status message.
+   */
+  String waitAndGetMessage() {
+    synchronized (this) {
+      // Wait for a message to send.
+      while (this.statusMessagesToSend.isEmpty()) {
+        try {
+          this.wait();
+        } catch (final InterruptedException e) {
+          LOG.log(Level.FINE, "Interrupted. Ignoring.");
+        }
+      }
+
+      // Send the message
+      return getMessageForStatus(this.statusMessagesToSend.poll());
+    }
+  }
+
+  /**
+   * Generates a string to be sent to the client based on a
+   * {@link org.apache.reef.proto.ReefServiceProtos.JobStatusProto}.
+   *
+   * @param status the status to be converted to String.
+   * @return the string to be sent back to the HTTP client.
+   */
+  static String getMessageForStatus(final ReefServiceProtos.JobStatusProto status) {
+    return status.getState().name();
+  }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/fece6294/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java b/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
new file mode 100644
index 0000000..222832a
--- /dev/null
+++ b/lang/java/reef-bridge-client/src/test/java/org/apache/reef/bridge/client/TestDriverStatusHTTPHandler.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.reef.bridge.client;
+
+import org.apache.reef.proto.ReefServiceProtos;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.logging.Logger;
+
+/**
+ * Tests for {@link DriverStatusHTTPHandler}.
+ */
+public final class TestDriverStatusHTTPHandler {
+
+  private static final Logger LOG = Logger.getLogger(TestDriverStatusHTTPHandler.class.getName());
+  private static final String TEST_DRIVER_ID = "TestDriver";
+
+  /**
+   * An array of all statuses to test.
+   */
+  private final ReefServiceProtos.JobStatusProto[] allStatuses = new ReefServiceProtos.JobStatusProto[] {
+      ReefServiceProtos.JobStatusProto.newBuilder()
+          .setIdentifier(TEST_DRIVER_ID)
+          .setState(ReefServiceProtos.State.INIT).build(),
+      ReefServiceProtos.JobStatusProto.newBuilder()
+          .setIdentifier(TEST_DRIVER_ID)
+          .setState(ReefServiceProtos.State.RUNNING).build(),
+      ReefServiceProtos.JobStatusProto.newBuilder()
+          .setIdentifier(TEST_DRIVER_ID)
+          .setState(ReefServiceProtos.State.DONE).build(),
+      ReefServiceProtos.JobStatusProto.newBuilder()
+          .setIdentifier(TEST_DRIVER_ID)
+          .setState(ReefServiceProtos.State.SUSPEND).build(),
+      ReefServiceProtos.JobStatusProto.newBuilder()
+          .setIdentifier(TEST_DRIVER_ID)
+          .setState(ReefServiceProtos.State.FAILED).build(),
+      ReefServiceProtos.JobStatusProto.newBuilder()
+          .setIdentifier(TEST_DRIVER_ID)
+          .setState(ReefServiceProtos.State.KILLED).build()
+  };
+
+  /**
+   * Make sure we get the right strings for the driver status.
+   */
+  @Test
+  public void testMessageForProto() {
+    for (final ReefServiceProtos.JobStatusProto status : allStatuses) {
+      Assert.assertEquals(status.getState().name(), DriverStatusHTTPHandler.getMessageForStatus(status));
+    }
+  }
+
+  /**
+   * Make sure {@link DriverStatusHTTPHandler} implements
+   * {@link org.apache.reef.runtime.common.driver.client.JobStatusHandler}.
+   */
+  @Test
+  public void testLastStatus() {
+    final DriverStatusHTTPHandler tester = new DriverStatusHTTPHandler();
+
+    for (final ReefServiceProtos.JobStatusProto status : allStatuses) {
+      tester.onNext(status);
+      Assert.assertSame(status, tester.getLastStatus());
+    }
+  }
+
+  /**
+   * Test the wait and notify for correctness.
+   */
+  @Test
+  public void testAsyncCalls() throws InterruptedException {
+    final DriverStatusHTTPHandler tester = new DriverStatusHTTPHandler();
+
+    final WaitingRunnable waiter = new WaitingRunnable(tester);
+
+    for (final ReefServiceProtos.JobStatusProto status : allStatuses) {
+      final Thread waitingThread = new Thread(waiter);
+      waitingThread.start();
+      Assert.assertTrue(waitingThread.isAlive());
+      Assert.assertNull(waiter.getResult());
+      tester.onNext(status);
+      waitingThread.join();
+      Assert.assertEquals(DriverStatusHTTPHandler.getMessageForStatus(status), waiter.getResult());
+    }
+  }
+
+  private final class WaitingRunnable implements Runnable {
+    private final DriverStatusHTTPHandler handler;
+    private String result = null;
+
+    private WaitingRunnable(final DriverStatusHTTPHandler handler) {
+      this.handler = handler;
+    }
+
+    @Override
+    public synchronized void run() {
+      result = handler.waitAndGetMessage();
+    }
+
+    public synchronized String getResult() {
+      final String returnValue = result;
+      result = null;
+      return returnValue;
+    }
+  }
+}


[27/28] reef git commit: [REEF-1977] Fix for OSUtils.getPID() random fails

Posted by do...@apache.org.
[REEF-1977] Fix for OSUtils.getPID() random fails

   * Add return value check for `process.waitFor()`
   * Log error stream if getting PID failed

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

Pull request:
  This closes #1430


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

Branch: refs/heads/REEF-335
Commit: baa5be82b349ff94113f1b9ff51af2cce53ff097
Parents: 0348e54
Author: Chenxi Zhao <33...@users.noreply.github.com>
Authored: Tue Feb 6 08:41:09 2018 -0800
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Wed Feb 14 13:51:09 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/reef/util/OSUtils.java | 42 +++++++++++---------
 1 file changed, 24 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/baa5be82/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java b/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java
index 25e6531..2c1a1db 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java
@@ -18,7 +18,10 @@
  */
 package org.apache.reef.util;
 
+import java.io.BufferedReader;
 import java.io.IOException;
+import java.io.InputStreamReader;
+import java.lang.management.ManagementFactory;
 import java.nio.charset.StandardCharsets;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -27,6 +30,7 @@ import java.util.logging.Logger;
  * OS utils.
  */
 public final class OSUtils {
+
   private static final Logger LOG = Logger.getLogger(OSUtils.class.getName());
 
   private OSUtils() {
@@ -75,34 +79,37 @@ public final class OSUtils {
     if (isUnix()) {
       try {
         final Process process = new ProcessBuilder()
-            .command("bash", "-c", "echo $PPID")
+            .command("/bin/sh", "-c", "echo $PPID")
             .start();
+        final int exitCode = process.waitFor();
+        if (exitCode != 0) {
+          LOG.log(Level.SEVERE, "Unable to determine PID. Exit code = {0}", exitCode);
+          final StringBuilder errorMsg = new StringBuilder();
+          try (final BufferedReader reader = new BufferedReader(
+              new InputStreamReader(process.getErrorStream(), StandardCharsets.UTF_8))) {
+            for (int i = 0; i < 10 && reader.ready(); ++i) { // Read the first 10 lines from stderr
+              errorMsg.append(reader.readLine()).append('\n');
+            }
+          }
+          LOG.log(Level.SEVERE, "Error:\n{0}", errorMsg);
+          return -1;
+        }
         final byte[] returnBytes = new byte[128];
         if (process.getInputStream().read(returnBytes) == -1) {
           LOG.log(Level.FINE, "No data read because end of stream was reached");
         }
-        final Long result = Long.valueOf(new String(returnBytes, StandardCharsets.UTF_8).trim());
         process.destroy();
-        return result;
-      } catch (final Exception e) {
+        return Long.parseLong(new String(returnBytes, StandardCharsets.UTF_8).trim());
+      } catch (final IOException | InterruptedException e) {
         LOG.log(Level.SEVERE, "Unable to determine PID", e);
         return -1;
       }
     } else if (isWindows()) {
+      final String name = ManagementFactory.getRuntimeMXBean().getName();
       try {
-        final Process process = new ProcessBuilder()
-            .command("powershell.exe", "-NoProfile", "-Command",
-                "wmic process where processid=$pid get parentprocessid")
-            .start();
-        final byte[] returnBytes = new byte[128];
-        if (process.getInputStream().read(returnBytes) == -1) {
-          LOG.log(Level.FINE, "No data read because end of stream was reached");
-        }
-        final Long result = Long.valueOf(new String(returnBytes, StandardCharsets.UTF_8).split("\n")[1].trim());
-        process.destroy();
-        return result;
-      } catch (final Exception e) {
-        LOG.log(Level.SEVERE, "Unable to determine PID", e);
+        return Long.parseLong(name.split("@")[0]);
+      } catch (final NumberFormatException e) {
+        LOG.log(Level.SEVERE, "Unable to parse PID from string " + name, e);
         return -1;
       }
     } else {
@@ -147,5 +154,4 @@ public final class OSUtils {
       return "$" + variableName;
     }
   }
-
 }


[07/28] reef git commit: [REEF-1945] Unit test TestMetricsMessages fails in master

Posted by do...@apache.org.
[REEF-1945] Unit test TestMetricsMessages fails in master

Change the logging level of a message in `MetricsService` event handler from `Verbose` to `Info`.
That makes the unit test that checks for that message in the logs pass.

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

Closes #1409


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

Branch: refs/heads/REEF-335
Commit: eb522e923453ae7c7a42f3be8884e66a6e5cc3ce
Parents: 49fbac2
Author: Sergiy Matusevych <mo...@apache.com>
Authored: Wed Nov 1 17:57:24 2017 -0700
Committer: jwang98052 <ju...@apache.org>
Committed: Wed Nov 1 19:31:59 2017 -0700

----------------------------------------------------------------------
 lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/eb522e92/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs
index d0bf196..44d2b32 100644
--- a/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs
@@ -75,7 +75,7 @@ namespace Org.Apache.REEF.Common.Telemetry
             var msgReceived = ByteUtilities.ByteArraysToString(contextMessage.Message);
             var counters = new EvaluatorMetrics(msgReceived).GetMetricsCounters();
 
-            Logger.Log(Level.Verbose, "Received {0} counters with context message: {1}.",
+            Logger.Log(Level.Info, "Received {0} counters with context message: {1}.",
                 counters.GetCounters().Count(), msgReceived);
 
             _countersData.Update(counters);


[21/28] reef git commit: [REEF-1964] Allow IMRU jobs in REEF.NET to be executed from any directory

Posted by do...@apache.org.
[REEF-1964] Allow IMRU jobs in REEF.NET to be executed from any directory

This addressed the issue by
  * Making the IMRU client explicitly add the assemblies from the
    executing directory. This changes the default behavior but maintains
    backwards compatibility.

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

Pull request:
  This closes #1420


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

Branch: refs/heads/REEF-335
Commit: 8c1fbf1fe8beaf0fc7bda975c36fffcb97c2ffb4
Parents: 8d5f861
Author: RoganC <ro...@microsoft.com>
Authored: Mon Dec 11 16:29:54 2017 -0800
Committer: Markus Weimer <we...@apache.org>
Committed: Tue Dec 12 14:17:08 2017 -0800

----------------------------------------------------------------------
 lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClient.cs | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/8c1fbf1f/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClient.cs b/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClient.cs
index a39762e..44e3f55 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClient.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClient.cs
@@ -162,6 +162,7 @@ namespace Org.Apache.REEF.IMRU.OnREEF.Client
             var imruJobSubmission = _jobRequestBuilder
                 .AddDriverConfiguration(imruDriverConfiguration)
                 .AddGlobalAssemblyForType(typeof(IMRUDriver<TMapInput, TMapOutput, TResult, TPartitionType>))
+                .AddGlobalAssembliesInDirectoryOfExecutingAssembly()
                 .SetJobIdentifier(jobDefinition.JobName)
                 .SetDriverMemory(5000)
                 .Build();


[12/28] reef git commit: [REEF-1939] Implement network transport for the Java-CLR bridge in C#

Posted by do...@apache.org.
[REEF-1939] Implement network transport for the Java-CLR bridge in C#

Summary of changes:
   * Implement `NetworkTransport` wrapper and `LocalObserver` handler for incoming messages
   * Add Java bridge endpoint file name to `REEFFileNames` constants
   * Refactor the Avro protocol for the bridge
   * Minor refactoring and code cleanup
   * The bulk of the classes in REEF-1939, 1938, and 1936
     were pulled from an uncommited pull request for REEF-1763
     implemented by Doug Service.

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

Pull Request:
  Closes #1406


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

Branch: refs/heads/REEF-335
Commit: 7b81827ff0f00ccf86dcd938c8fbaadaf6bc084a
Parents: 7a2aa5c
Author: Sergiy Matusevych <mo...@apache.com>
Authored: Thu Oct 26 15:21:14 2017 -0700
Committer: Doug Service <do...@apache.org>
Committed: Fri Nov 3 18:40:20 2017 +0000

----------------------------------------------------------------------
 .../Org.Apache.REEF.Bridge.CLR/LocalObserver.cs |  89 ++++++++++++++
 .../Message/BridgeInterop.cs                    |  83 -------------
 .../Message/BridgeProtocol.cs                   |  66 +++++++++++
 .../Message/MessageType.cs                      |  47 --------
 .../Message/Protocol.cs                         |  66 -----------
 .../NetworkTransport.cs                         | 117 +++++++++++++++++++
 .../Org.Apache.REEF.Bridge.CLR.csproj           |   8 +-
 .../Files/REEFFileNames.cs                      |  16 ++-
 8 files changed, 288 insertions(+), 204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/7b81827f/lang/cs/Org.Apache.REEF.Bridge.CLR/LocalObserver.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/LocalObserver.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/LocalObserver.cs
new file mode 100644
index 0000000..d76e937
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/LocalObserver.cs
@@ -0,0 +1,89 @@
+// 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.Annotations;
+using Org.Apache.REEF.Tang.Implementations.InjectionPlan;
+using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Wake.Avro;
+using Org.Apache.REEF.Wake.Remote;
+
+namespace Org.Apache.REEF.Bridge
+{
+    /// <summary>
+    /// The Local Observer class receives byte buffer messages from the transport layer,
+    /// deserializes the messages into Avro C# classes, and invokes the appropriate
+    /// IObserver callback on the Avro message observer.
+    /// </summary>
+    public sealed class LocalObserver : IObserver<IRemoteMessage<byte[]>>
+    {
+        /// <summary>
+        /// Specify the class that will be called to process deserialied Avro messages.
+        /// </summary>
+        [NamedParameter(Documentation =
+            "Must implement IObserver<IMessageInstance<TMessage>> for messages to be received.")]
+        public class MessageObserver : Name<object>
+        {
+        }
+
+        private static readonly Logger Logger = Logger.GetLogger(typeof(LocalObserver));
+
+        private readonly ProtocolSerializer _serializer;
+        private readonly IInjectionFuture<object> _fMessageObserver;
+
+        /// <param name="serializer">The protocol serializer instance to be used.</param>
+        /// <param name="fMessageObserver">An injection future with message observer to be
+        /// called to process Avro messages from the Java bridge.</param>
+        [Inject]
+        private LocalObserver(
+            ProtocolSerializer serializer,
+            [Parameter(typeof(MessageObserver))] IInjectionFuture<object> fMessageObserver)
+        {
+            _serializer = serializer;
+            _fMessageObserver = fMessageObserver;
+        }
+
+        /// <summary>
+        /// Called by the remote manager to process messages received from the java bridge.
+        /// </summary>
+        /// <param name="message">A byte buffer containing a serialzied message.</param>
+        public void OnNext(IRemoteMessage<byte[]> message)
+        {
+            Logger.Log(Level.Verbose, "Message received: {0}", message.Identifier);
+
+            // Deserialize the message and invoke the appropriate handler.
+            _serializer.Read(message.Message, _fMessageObserver.Get());
+        }
+
+        /// <summary>
+        /// Handles error conditions in the low transport layer.
+        /// </summary>
+        /// <param name="error">The exception generated in the transport layer.</param>
+        public void OnError(Exception error)
+        {
+            Logger.Log(Level.Error, "Error in the transport layer", error);
+        }
+
+        /// <summary>
+        /// Notification that no more message processing is required.
+        /// </summary>
+        public void OnCompleted()
+        {
+            Logger.Log(Level.Info, "Completed");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/7b81827f/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeInterop.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeInterop.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeInterop.cs
deleted file mode 100644
index 2e862a7..0000000
--- a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeInterop.cs
+++ /dev/null
@@ -1,83 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-//<auto-generated />
-namespace org.apache.reef.bridge.message
-{
-    using System;
-    using System.Collections.Generic;
-    using System.Runtime.Serialization;
-    using Microsoft.Hadoop.Avro;
-
-    /// <summary>
-    /// Used to serialize and deserialize Avro record org.apache.reef.bridge.message.BridgeInterop.
-    /// </summary>
-    [DataContract(Namespace = "org.apache.reef.bridge.message")]
-    public partial class BridgeInterop
-    {
-        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.bridge.message.BridgeInterop"",""doc"":""Container message for all Java/CLR bridge messages in the protocol."",""fields"":[{""name"":""sequence"",""doc"":""The unique squence identifier of the message in the protocol stream."",""type"":""long""},{""name"":""messageType"",""doc"":""The type of the bridge Java/CLR interop message."",""type"":{""type"":""enum"",""name"":""org.apache.reef.bridge.message.MessageType"",""doc"":""An enumeration of all possible Java/C# bridge protocol messages."",""symbols"":[""SetupBridge"",""SystemOnStart"",""Acknowledgement""]}},{""name"":""message"",""doc"":""A union which contains the actual message."",""type"":[{""type"":""record"",""name"":""org.apache.reef.bridge.message.SetupBridge"",""doc"":""Notify the C# bridge of the http port of the Java bridge webserver."",""fields"":[{""name"":""httpServerPortNumber"",""doc"":""The Java bridge http server port number.
 "",""type"":""int""}]},{""type"":""record"",""name"":""org.apache.reef.bridge.message.SystemOnStart"",""doc"":""Notify the C# bridge the system is now running."",""fields"":[{""name"":""dateTime"",""doc"":""Date time in seconds as a long since January 1, 1970"",""type"":""long""}]},{""type"":""record"",""name"":""org.apache.reef.bridge.message.Acknowledgement"",""doc"":""The Acknowledgement message is sent to the Java bridge to acknowledge receipt and processing of a specific message."",""fields"":[{""name"":""messageIdentifier"",""doc"":""The message identifier of the message that was successfully processed."",""type"":""long""}]},{""type"":""array"",""items"":""bytes""}]}]}";
-
-        /// <summary>
-        /// Gets the schema.
-        /// </summary>
-        public static string Schema
-        {
-            get
-            {
-                return JsonSchema;
-            }
-        }
-      
-        /// <summary>
-        /// Gets or sets the sequence field.
-        /// </summary>
-        [DataMember]
-        public long sequence { get; set; }
-              
-        /// <summary>
-        /// Gets or sets the messageType field.
-        /// </summary>
-        [DataMember]
-        public org.apache.reef.bridge.message.MessageType messageType { get; set; }
-              
-        /// <summary>
-        /// Gets or sets the message field.
-        /// </summary>
-        [DataMember]
-        [AvroUnion(typeof(org.apache.reef.bridge.message.SetupBridge), typeof(org.apache.reef.bridge.message.SystemOnStart), typeof(org.apache.reef.bridge.message.Acknowledgement), typeof(List<byte[]>))]
-        public object message { get; set; }
-                
-        /// <summary>
-        /// Initializes a new instance of the <see cref="BridgeInterop"/> class.
-        /// </summary>
-        public BridgeInterop()
-        {
-        }
-
-        /// <summary>
-        /// Initializes a new instance of the <see cref="BridgeInterop"/> class.
-        /// </summary>
-        /// <param name="sequence">The sequence.</param>
-        /// <param name="messageType">The messageType.</param>
-        /// <param name="message">The message.</param>
-        public BridgeInterop(long sequence, org.apache.reef.bridge.message.MessageType messageType, object message)
-        {
-            this.sequence = sequence;
-            this.messageType = messageType;
-            this.message = message;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/7b81827f/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeProtocol.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeProtocol.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeProtocol.cs
new file mode 100644
index 0000000..deab9df
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/BridgeProtocol.cs
@@ -0,0 +1,66 @@
+// 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.
+//<auto-generated />
+namespace org.apache.reef.bridge.message
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Runtime.Serialization;
+    using Microsoft.Hadoop.Avro;
+
+    /// <summary>
+    /// Used to serialize and deserialize Avro record org.apache.reef.bridge.message.BridgeProtocol.
+    /// </summary>
+    [DataContract(Namespace = "org.apache.reef.bridge.message")]
+    public partial class BridgeProtocol
+    {
+        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.bridge.message.BridgeProtocol"",""doc"":""Negotiate Java/C# bridge protocol messages."",""fields"":[{""name"":""offset"",""doc"":""The index offset of the message identifiers."",""type"":""int""}]}";
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public static string Schema
+        {
+            get
+            {
+                return JsonSchema;
+            }
+        }
+      
+        /// <summary>
+        /// Gets or sets the offset field.
+        /// </summary>
+        [DataMember]
+        public int offset { get; set; }
+                
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BridgeProtocol"/> class.
+        /// </summary>
+        public BridgeProtocol()
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BridgeProtocol"/> class.
+        /// </summary>
+        /// <param name="offset">The offset.</param>
+        public BridgeProtocol(int offset)
+        {
+            this.offset = offset;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/7b81827f/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/MessageType.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/MessageType.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/MessageType.cs
deleted file mode 100644
index be59777..0000000
--- a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/MessageType.cs
+++ /dev/null
@@ -1,47 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-//<auto-generated />
-namespace org.apache.reef.bridge.message
-{
-    using System.Runtime.Serialization;
-
-    /// <summary>
-    /// Used to serialize and deserialize Avro enum org.apache.reef.bridge.message.MessageType.
-    /// </summary>
-    [DataContract(Namespace = "org.apache.reef.bridge.message")]
-    public enum MessageType
-    {
-        /// <summary>
-        /// The SetupBridge symbol.
-        /// </summary>
-        [EnumMember]
-        SetupBridge,
-
-        /// <summary>
-        /// The SystemOnStart symbol.
-        /// </summary>
-        [EnumMember]
-        SystemOnStart,
-
-        /// <summary>
-        /// The Acknowledgement symbol.
-        /// </summary>
-        [EnumMember]
-        Acknowledgement,
-
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/7b81827f/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/Protocol.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/Protocol.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/Protocol.cs
deleted file mode 100644
index 2f88621..0000000
--- a/lang/cs/Org.Apache.REEF.Bridge.CLR/Message/Protocol.cs
+++ /dev/null
@@ -1,66 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-//<auto-generated />
-namespace org.apache.reef.bridge.message
-{
-    using System;
-    using System.Collections.Generic;
-    using System.Runtime.Serialization;
-    using Microsoft.Hadoop.Avro;
-
-    /// <summary>
-    /// Used to serialize and deserialize Avro record org.apache.reef.bridge.message.Protocol.
-    /// </summary>
-    [DataContract(Namespace = "org.apache.reef.bridge.message")]
-    public partial class Protocol
-    {
-        private const string JsonSchema = @"{""type"":""record"",""name"":""org.apache.reef.bridge.message.Protocol"",""doc"":""Negotiate Java/C# bridge protocol messages."",""fields"":[{""name"":""offset"",""doc"":""The index offset of the message identifiers."",""type"":""int""}]}";
-
-        /// <summary>
-        /// Gets the schema.
-        /// </summary>
-        public static string Schema
-        {
-            get
-            {
-                return JsonSchema;
-            }
-        }
-      
-        /// <summary>
-        /// Gets or sets the offset field.
-        /// </summary>
-        [DataMember]
-        public int offset { get; set; }
-                
-        /// <summary>
-        /// Initializes a new instance of the <see cref="Protocol"/> class.
-        /// </summary>
-        public Protocol()
-        {
-        }
-
-        /// <summary>
-        /// Initializes a new instance of the <see cref="Protocol"/> class.
-        /// </summary>
-        /// <param name="offset">The offset.</param>
-        public Protocol(int offset)
-        {
-            this.offset = offset;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/reef/blob/7b81827f/lang/cs/Org.Apache.REEF.Bridge.CLR/NetworkTransport.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/NetworkTransport.cs b/lang/cs/Org.Apache.REEF.Bridge.CLR/NetworkTransport.cs
new file mode 100644
index 0000000..ed23a90
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/NetworkTransport.cs
@@ -0,0 +1,117 @@
+// 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.Concurrent;
+using System.IO;
+using System.Net;
+using org.apache.reef.bridge.message;
+using Org.Apache.REEF.Common.Files;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Utilities.Logging;
+using Org.Apache.REEF.Wake.Avro;
+using Org.Apache.REEF.Wake.Remote;
+using Org.Apache.REEF.Wake.Remote.Impl;
+
+namespace Org.Apache.REEF.Bridge
+{
+    /// <summary>
+    /// The CLR Bridge Network class aggregates a RemoteManager and
+    /// Protocol Serializer to provide a simple send/receive interface
+    /// between the CLR and Java bridges.
+    /// </summary>
+    public sealed class NetworkTransport
+    {
+        private static readonly Logger Logger = Logger.GetLogger(typeof(NetworkTransport));
+
+        private readonly BlockingCollection<byte[]> _queue = new BlockingCollection<byte[]>();
+
+        private readonly ProtocolSerializer _serializer;
+        private readonly IRemoteManager<byte[]> _remoteManager;
+        private readonly IObserver<byte[]> _remoteObserver;
+        private readonly REEFFileNames _fileNames;
+
+        /// <summary>
+        /// Construct a network stack using the wake remote manager.
+        /// </summary>
+        /// <param name="localAddressProvider">An address provider used to obtain
+        /// a local IP address on an open port.</param>
+        /// <param name="serializer">Serializer/deserializer of the bridge messages.</param>
+        /// <param name="localObserver">Handler of the incoming bridge messages.</param>
+        /// <param name="remoteManagerFactory">RemoteManager factory.
+        /// We need a new instance of the RM to communicate with the Java side of the bridge.</param>
+        /// <param name="filenames">Collection of global constants for file paths and such.</param>
+        [Inject]
+        private NetworkTransport(
+            ILocalAddressProvider localAddressProvider,
+            ProtocolSerializer serializer,
+            LocalObserver localObserver,
+            IRemoteManagerFactory remoteManagerFactory,
+            REEFFileNames fileNames)
+        {
+            _serializer = serializer;
+            _fileNames = fileNames;
+
+            // Instantiate the remote manager.
+            _remoteManager = remoteManagerFactory.GetInstance(localAddressProvider.LocalAddress, new ByteCodec());
+
+            // Listen to the java bridge on the local end point.
+            _remoteManager.RegisterObserver(localObserver);
+            Logger.Log(Level.Info, "Local observer listening to java bridge on: [{0}]", _remoteManager.LocalEndpoint);
+
+            // Instantiate a remote observer to send messages to the java bridge.
+            IPEndPoint javaIpEndPoint = GetJavaBridgeEndpoint();
+            Logger.Log(Level.Info, "Connecting to java bridge on: [{0}]", javaIpEndPoint);
+            _remoteObserver = _remoteManager.GetRemoteObserver(javaIpEndPoint);
+
+            // Negotiate the protocol.
+            Send(0, new BridgeProtocol(100));
+        }
+
+        /// <summary>
+        /// Send a message to the java side of the bridge.
+        /// </summary>
+        /// <param name="identifier">A long value that which is the unique sequence identifier of the message.</param>
+        /// <param name="message">An object reference to a message in the org.apache.reef.bridge.message package.</param>
+        public void Send(long identifier, object message)
+        {
+            Logger.Log(Level.Verbose, "Sending message: {0}", message);
+            _remoteObserver.OnNext(_serializer.Write(message, identifier));
+        }
+
+        /// <summary>
+        /// Retrieves the address of the java bridge.
+        /// </summary>
+        /// <returns>IP address and port of the Java bridge.</returns>
+        private IPEndPoint GetJavaBridgeEndpoint()
+        {
+            using (FileStream stream = File.Open(_fileNames.DriverJavaBridgeEndpointFileName, FileMode.Open))
+            {
+                using (StreamReader reader = new StreamReader(stream))
+                {
+                    string javaBridgeAddress = reader.ReadToEnd();
+                    Logger.Log(Level.Info, "Java bridge address: {0}", javaBridgeAddress);
+
+                    string[] javaAddressStrs = javaBridgeAddress.Split(':');
+                    IPAddress javaBridgeIpAddress = IPAddress.Parse(javaAddressStrs[0]);
+                    int port = int.Parse(javaAddressStrs[1]);
+                    return new IPEndPoint(javaBridgeIpAddress, port);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/7b81827f/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj b/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
index dbbe571..20500c3 100644
--- a/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
+++ b/lang/cs/Org.Apache.REEF.Bridge.CLR/Org.Apache.REEF.Bridge.CLR.csproj
@@ -53,12 +53,12 @@ under the License.
     <Compile Include="$(SolutionDir)\SharedAssemblyInfo.cs">
       <Link>Properties\SharedAssemblyInfo.cs</Link>
     </Compile>
+    <Compile Include="LocalObserver.cs" />
     <Compile Include="Message\Acknowledgement.cs" />
-    <Compile Include="Message\BridgeInterop.cs" />
-    <Compile Include="Message\MessageType.cs" />
-    <Compile Include="Message\Protocol.cs" />
+    <Compile Include="Message\BridgeProtocol.cs" />
     <Compile Include="Message\SetupBridge.cs" />
     <Compile Include="Message\SystemOnStart.cs" />
+    <Compile Include="NetworkTransport.cs" />
   </ItemGroup>
   <ItemGroup>
     <None Include="Message\README.md" />
@@ -92,4 +92,4 @@ under the License.
   <Import Project="$(SolutionDir)\AvroCodeGeneration.Targets" Condition="Exists('$(SolutionDir)\AvroCodeGeneration.Targets')" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
   <Import Project="$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets" Condition="Exists('$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets')" />
-</Project>
\ No newline at end of file
+</Project>

http://git-wip-us.apache.org/repos/asf/reef/blob/7b81827f/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
index 5cb272e..4c21d1e 100644
--- a/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Files/REEFFileNames.cs
@@ -50,6 +50,7 @@ namespace Org.Apache.REEF.Common.Files
         private const string CLR_DRIVER_CONFIGURATION_NAME = "clrdriver.conf";
         private const string CLR_BRIDGE_CONFIGURATION_NAME = "clrBridge.config";
         private const string DRIVER_HTTP_ENDPOINT_FILE_NAME = "DriverHttpEndpoint.txt";
+        private const string DRIVER_JAVA_BRIDGE_ENDPOINT_FILE_NAME = "DriverJavaBridgeEndpoint.txt";
         private const string BRIDGE_EXE_NAME = "Org.Apache.REEF.Bridge.exe";
         private const string BRIDGE_EXE_CONFIG_NAME = "Org.Apache.REEF.Bridge.exe.config";
         private const string SECURITY_TOKEN_IDENTIFIER_FILE = "SecurityTokenId";
@@ -338,12 +339,19 @@ namespace Org.Apache.REEF.Common.Files
         }
 
         /// <summary>
+        /// Name of the file that contains the driver name server address and port.
         /// </summary>
+        /// <returns>File name that contains the dfs path for the DriverNameServerEndpoint</returns>
+        public string DriverJavaBridgeEndpointFileName
+        {
+            get { return DRIVER_JAVA_BRIDGE_ENDPOINT_FILE_NAME; }
+        }
+
         /// <returns>File name that contains the dfs path for the DriverHttpEndpoint</returns>
         [Unstable("0.13", "Working in progress for what to return after submit")]
-        public string DriverHttpEndpoint 
-        { 
-            get { return DRIVER_HTTP_ENDPOINT_FILE_NAME; } 
+        public string DriverHttpEndpoint
+        {
+            get { return DRIVER_HTTP_ENDPOINT_FILE_NAME; }
         }
 
         private static readonly string GLOBAL_FOLDER_PATH = Path.Combine(REEF_BASE_FOLDER, GLOBAL_FOLDER);
@@ -358,4 +366,4 @@ namespace Org.Apache.REEF.Common.Files
         private static readonly string EVALUATOR_CONFIGURATION_PATH =
             Path.Combine(LOCAL_FOLDER_PATH, EVALUATOR_CONFIGURATION_NAME);
     }
-}
\ No newline at end of file
+}


[26/28] reef git commit: [REEF-1975] Support stream option for Azure Blob storage

Posted by do...@apache.org.
[REEF-1975] Support stream option for Azure Blob storage

   * Support stream option for Azure Blob storage
   * Updated Open test to assert on stream Stream to blobs (#8)
   * Implement Create method to create a blob and return a write stream to the blob

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

Closes #1426


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

Branch: refs/heads/REEF-335
Commit: 0348e54ec13bd8f267670167af55b8f6797b24bd
Parents: d459549
Author: dwaijam <dw...@gmail.com>
Authored: Thu Jan 18 10:30:16 2018 -0800
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Mon Feb 12 12:57:17 2018 -0800

----------------------------------------------------------------------
 .../TestAzureBlockBlobFileSystem.cs             |  20 +++-
 .../TestAzureBlockBlobFileSystemE2E.cs          | 102 ++++++++++++-------
 .../AzureBlob/AzureBlockBlobFileSystem.cs       |   8 +-
 .../FileSystem/AzureBlob/AzureCloudBlockBlob.cs |  10 ++
 .../FileSystem/AzureBlob/ICloudBlockBlob.cs     |  14 +++
 5 files changed, 111 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/0348e54e/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
index a1f9b34..dc48d5e 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystem.cs
@@ -36,18 +36,24 @@ namespace Org.Apache.REEF.IO.Tests
     /// </summary>
     public sealed class TestAzureBlockBlobFileSystem
     {
-        private readonly static Uri FakeUri = new Uri("http://fake.com");
+        private static readonly Uri FakeUri = new Uri("http://fake.com");
 
         [Fact]
-        public void TestCreateNotSupported()
+        public void TestCreate()
         {
-            Assert.Throws<NotSupportedException>(() => new TestContext().GetAzureFileSystem().Create(FakeUri));
+            var testContext = new TestContext();
+            Stream stream = testContext.GetAzureFileSystem().Create(new Uri(FakeUri, "container/file"));
+            testContext.TestCloudBlockBlob.Received(1).Create();
+            Assert.Equal(testContext.TestCreateStream, stream);
         }
 
         [Fact]
-        public void TestOpenNotSupported()
+        public void TestOpen()
         {
-            Assert.Throws<NotSupportedException>(() => new TestContext().GetAzureFileSystem().Open(FakeUri));
+            var testContext = new TestContext();
+            Stream stream = testContext.GetAzureFileSystem().Open(new Uri(FakeUri, "container/file"));
+            testContext.TestCloudBlockBlob.Received(1).Open();
+            Assert.Equal(testContext.TestOpenStream, stream);
         }
 
         [Fact]
@@ -133,6 +139,8 @@ namespace Org.Apache.REEF.IO.Tests
             public readonly ICloudBlockBlob TestCloudBlockBlob = Substitute.For<ICloudBlockBlob>();
             public readonly ICloudBlobContainer TestCloudBlobContainer = Substitute.For<ICloudBlobContainer>();
             public readonly ICloudBlobDirectory TestCloudBlobDirectory = Substitute.For<ICloudBlobDirectory>();
+            public readonly Stream TestOpenStream = Substitute.For<Stream>();
+            public readonly Stream TestCreateStream = Substitute.For<Stream>();
 
             public IFileSystem GetAzureFileSystem()
             {
@@ -144,6 +152,8 @@ namespace Org.Apache.REEF.IO.Tests
                 injector.BindVolatileInstance(TestCloudBlobClient);
                 var fs = injector.GetInstance<AzureBlockBlobFileSystem>();
                 TestCloudBlobClient.BaseUri.ReturnsForAnyArgs(FakeUri);
+                TestCloudBlockBlob.Open().Returns(TestOpenStream);
+                TestCloudBlockBlob.Create().Returns(TestCreateStream);
                 TestCloudBlobClient.GetBlockBlobReference(FakeUri).ReturnsForAnyArgs(TestCloudBlockBlob);
                 TestCloudBlobClient.GetContainerReference("container").ReturnsForAnyArgs(TestCloudBlobContainer);
                 TestCloudBlobContainer.GetDirectoryReference("directory").ReturnsForAnyArgs(TestCloudBlobDirectory);

http://git-wip-us.apache.org/repos/asf/reef/blob/0348e54e/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
index 8e8a708..7b749e4 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
@@ -34,6 +34,7 @@ namespace Org.Apache.REEF.IO.Tests
     /// </summary>
     public sealed class TestAzureBlockBlobFileSystemE2E : IDisposable
     {
+        private const string SkipMessage = "Fill in credentials before running test"; // Use null to run tests
         private const string HelloFile = "hello";
         private IFileSystem _fileSystem;
         private CloudBlobContainer _container;
@@ -41,14 +42,14 @@ namespace Org.Apache.REEF.IO.Tests
         public TestAzureBlockBlobFileSystemE2E()
         {
             // Fill in before running test!
-            const string connectionString = "DefaultEndpointsProtocol=http;AccountName=myAccount;AccountKey=myKey;";
+            const string ConnectionString = "DefaultEndpointsProtocol=http;AccountName=myAccount;AccountKey=myKey;";
             var defaultContainerName = "reef-test-container-" + Guid.NewGuid();
             var conf = AzureBlockBlobFileSystemConfiguration.ConfigurationModule
-                .Set(AzureBlockBlobFileSystemConfiguration.ConnectionString, connectionString)
+                .Set(AzureBlockBlobFileSystemConfiguration.ConnectionString, ConnectionString)
                 .Build();
 
             _fileSystem = TangFactory.GetTang().NewInjector(conf).GetInstance<AzureBlockBlobFileSystem>();
-            _container = CloudStorageAccount.Parse(connectionString).CreateCloudBlobClient().GetContainerReference(defaultContainerName);
+            _container = CloudStorageAccount.Parse(ConnectionString).CreateCloudBlobClient().GetContainerReference(defaultContainerName);
             _container.CreateIfNotExistsAsync().Wait();
         }
 
@@ -88,7 +89,40 @@ namespace Org.Apache.REEF.IO.Tests
             return task.Result;
         }
 
-        [Fact(Skip = "Fill in credentials before running test")]
+        [Fact(Skip = SkipMessage)]
+        public void TestOpenE2E()
+        {
+            const string Text = "hello";
+            var blob = _container.GetBlockBlobReference(HelloFile);
+            UploadFromString(blob, Text);
+            Assert.True(CheckBlobExists(blob));
+            using (var reader = new StreamReader(_fileSystem.Open(PathToFile(HelloFile))))
+            {
+                string streamText = reader.ReadToEnd();
+                Assert.Equal(Text, streamText);
+            }
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestCreateE2E()
+        {
+            const string Text = "Hello Azure Blob";
+            var blob = _container.GetBlockBlobReference(HelloFile);
+            Assert.False(CheckBlobExists(blob));
+            using (var streamWriter = new StreamWriter(_fileSystem.Create(PathToFile(HelloFile))))
+            {
+                streamWriter.Write(Text);
+            }
+            blob = _container.GetBlockBlobReference(HelloFile);
+            Assert.True(CheckBlobExists(blob));
+            using (var reader = new StreamReader(blob.OpenRead()))
+            {
+                string streamText = reader.ReadToEnd();
+                Assert.Equal(Text, streamText);
+            }
+        }
+
+        [Fact(Skip = SkipMessage)]
         public void TestDeleteE2E()
         {
             var blob = _container.GetBlockBlobReference(HelloFile);
@@ -98,7 +132,7 @@ namespace Org.Apache.REEF.IO.Tests
             Assert.False(CheckBlobExists(blob));
         }
 
-        [Fact(Skip = "Fill in credentials before running test")]
+        [Fact(Skip = SkipMessage)]
         public void TestExistsE2E()
         {
             var helloFilePath = PathToFile(HelloFile);
@@ -109,39 +143,39 @@ namespace Org.Apache.REEF.IO.Tests
             Assert.False(_fileSystem.Exists(helloFilePath));
         }
 
-        [Fact(Skip = "Fill in credentials before running test")]
+        [Fact(Skip = SkipMessage)]
         public void TestCopyE2E()
         {
-            const string srcFileName = "src";
-            const string destFileName = "dest";
-            var srcFilePath = PathToFile(srcFileName);
-            var destFilePath = PathToFile(destFileName);
-            ICloudBlob srcBlob = _container.GetBlockBlobReference(srcFileName);
+            const string SrcFileName = "src";
+            const string DestFileName = "dest";
+            var srcFilePath = PathToFile(SrcFileName);
+            var destFilePath = PathToFile(DestFileName);
+            ICloudBlob srcBlob = _container.GetBlockBlobReference(SrcFileName);
             UploadFromString(srcBlob, "hello");
             Assert.True(CheckBlobExists(srcBlob));
-            ICloudBlob destBlob = _container.GetBlockBlobReference(destFileName);
+            ICloudBlob destBlob = _container.GetBlockBlobReference(DestFileName);
             Assert.False(CheckBlobExists(destBlob));
             _fileSystem.Copy(srcFilePath, destFilePath);
-            destBlob = GetBlobReferenceFromServer(_container, destFileName);
+            destBlob = GetBlobReferenceFromServer(_container, DestFileName);
             Assert.True(CheckBlobExists(destBlob));
-            srcBlob = GetBlobReferenceFromServer(_container, srcFileName);
+            srcBlob = GetBlobReferenceFromServer(_container, SrcFileName);
             Assert.True(CheckBlobExists(srcBlob));
-            Assert.Equal(DownloadText(_container.GetBlockBlobReference(srcFileName)), DownloadText(_container.GetBlockBlobReference(destFileName)));
+            Assert.Equal(DownloadText(_container.GetBlockBlobReference(SrcFileName)), DownloadText(_container.GetBlockBlobReference(DestFileName)));
         }
 
-        [Fact(Skip = "Fill in credentials before running test")]
+        [Fact(Skip = SkipMessage)]
         public void TestCopyToLocalE2E()
         {
             var helloFilePath = PathToFile(HelloFile);
             var blob = _container.GetBlockBlobReference(HelloFile);
             var tempFilePath = GetTempFilePath();
-            const string text = "hello";
+            const string Text = "hello";
             try
             {
-                UploadFromString(blob, text);
+                UploadFromString(blob, Text);
                 _fileSystem.CopyToLocal(helloFilePath, tempFilePath);
                 Assert.True(File.Exists(tempFilePath));
-                Assert.Equal(text, File.ReadAllText(tempFilePath));
+                Assert.Equal(Text, File.ReadAllText(tempFilePath));
             }
             finally
             {
@@ -149,17 +183,17 @@ namespace Org.Apache.REEF.IO.Tests
             }
         }
 
-        [Fact(Skip = "Fill in credentials before running test")]
+        [Fact(Skip = SkipMessage)]
         public void TestCopyFromLocalE2E()
         {
             var helloFilePath = PathToFile(HelloFile);
             ICloudBlob blob = _container.GetBlockBlobReference(HelloFile);
             Assert.False(CheckBlobExists(blob));
             var tempFilePath = GetTempFilePath();
-            const string text = "hello";
+            const string Text = "hello";
             try
             {
-                File.WriteAllText(tempFilePath, text);
+                File.WriteAllText(tempFilePath, Text);
                 _fileSystem.CopyFromLocal(tempFilePath, helloFilePath);
                 blob = GetBlobReferenceFromServer(_container, HelloFile);
                 Assert.True(CheckBlobExists(blob));
@@ -171,7 +205,7 @@ namespace Org.Apache.REEF.IO.Tests
                     using (var sr = new StreamReader(stream))
                     {
                         var matchingText = sr.ReadToEnd();
-                        Assert.Equal(text, matchingText);
+                        Assert.Equal(Text, matchingText);
                     }
                 }
             }
@@ -181,29 +215,29 @@ namespace Org.Apache.REEF.IO.Tests
             }
         }
 
-        [Fact(Skip = "Fill in credentials before running test")]
+        [Fact(Skip = SkipMessage)]
         public void TestDeleteDirectoryAtContainerE2E()
         {
             _fileSystem.DeleteDirectory(_container.Uri);
             Assert.False(CheckContainerExists(_container));
         }
 
-        [Fact(Skip = "Fill in credentials before running test")]
+        [Fact(Skip = SkipMessage)]
         public void TestDeleteDirectoryFirstLevelE2E()
         {
-            const string directory = "dir";
+            const string Directory = "dir";
             var blockBlobs = new List<CloudBlockBlob>(); 
 
             for (var i = 0; i < 3; i++)
             {
-                var filePath = directory + '/' + i;
+                var filePath = Directory + '/' + i;
                 var blockBlob = _container.GetBlockBlobReference(filePath);
                 UploadFromString(blockBlob, "hello");
                 Assert.True(CheckBlobExists(blockBlob));
                 blockBlobs.Add(blockBlob);
             }
 
-            _fileSystem.DeleteDirectory(PathToFile(directory));
+            _fileSystem.DeleteDirectory(PathToFile(Directory));
 
             foreach (var blockBlob in blockBlobs)
             {
@@ -213,18 +247,18 @@ namespace Org.Apache.REEF.IO.Tests
             Assert.True(CheckContainerExists(_container));
         }
 
-        [Fact(Skip = "Fill in credentials before running test")]
+        [Fact(Skip = SkipMessage)]
         public void TestDeleteDirectorySecondLevelE2E()
         {
-            const string directory1 = "dir1";
-            const string directory2 = "dir2";
+            const string Directory1 = "dir1";
+            const string Directory2 = "dir2";
             var blockBlobs1 = new List<CloudBlockBlob>();
             var blockBlobs2 = new List<CloudBlockBlob>();
 
             for (var i = 0; i < 3; i++)
             {
-                var filePath1 = directory1 + '/' + i;
-                var filePath2 = directory1 + '/' + directory2 + '/' + i;
+                var filePath1 = Directory1 + '/' + i;
+                var filePath2 = Directory1 + '/' + Directory2 + '/' + i;
                 var blockBlob1 = _container.GetBlockBlobReference(filePath1);
                 var blockBlob2 = _container.GetBlockBlobReference(filePath2);
                 UploadFromString(blockBlob1, "hello");
@@ -235,7 +269,7 @@ namespace Org.Apache.REEF.IO.Tests
                 blockBlobs2.Add(blockBlob2);
             }
 
-            _fileSystem.DeleteDirectory(PathToFile(directory1 + '/' + directory2));
+            _fileSystem.DeleteDirectory(PathToFile(Directory1 + '/' + Directory2));
 
             foreach (var blockBlob in blockBlobs2)
             {

http://git-wip-us.apache.org/repos/asf/reef/blob/0348e54e/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
index b5f659c..81a7c1f 100644
--- a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureBlockBlobFileSystem.cs
@@ -42,19 +42,19 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
         }
 
         /// <summary>
-        /// Not supported for Azure Blobs, will throw <see cref="NotSupportedException"/>.
+        /// Returns a Stream object to the blob specified by the fileUri.
         /// </summary>
         public Stream Open(Uri fileUri)
         {
-            throw new NotSupportedException("Open is not supported for AzureBlockBlobFileSystem.");
+            return _client.GetBlockBlobReference(fileUri).Open();
         }
 
         /// <summary>
-        /// Not supported for Azure Blobs, will throw <see cref="NotSupportedException"/>.
+        /// Creates a blob for the specified fileUri and returns a write Stream object to it.
         /// </summary>
         public Stream Create(Uri fileUri)
         {
-            throw new NotSupportedException("Open is not supported for AzureBlockBlobFileSystem.");
+            return _client.GetBlockBlobReference(fileUri).Create();
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/reef/blob/0348e54e/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs
index c4b9c6d..b2289b6 100644
--- a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/AzureCloudBlockBlob.cs
@@ -59,6 +59,16 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
             _blob = new CloudBlockBlob(uri, credentials);
         }
 
+        public Stream Open()
+        {
+            return _blob.OpenRead();
+        }
+
+        public Stream Create()
+        {
+            return _blob.OpenWrite();
+        }
+
         public bool Exists()
         {
             var task = _blob.ExistsAsync();

http://git-wip-us.apache.org/repos/asf/reef/blob/0348e54e/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs
index 2941e6e..ec102cc 100644
--- a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureBlob/ICloudBlockBlob.cs
@@ -46,6 +46,20 @@ namespace Org.Apache.REEF.IO.FileSystem.AzureBlob
         CopyState CopyState { get; }
 
         /// <summary>
+        /// Opens a stream to the blob content.
+        /// </summary>
+        /// <returns>System.IO.Stream object.</returns>
+        /// <exception cref="StorageException">If blob does not exist</exception>
+        Stream Open();
+
+        /// <summary>
+        /// Creates a blob and returns a write Stream object to it.
+        /// </summary>
+        /// <returns>System.IO.Stream object.</returns>
+        /// <exception cref="StorageException">If blob cannot be created</exception>
+        Stream Create();
+
+        /// <summary>
         /// Makes a round trip to the server to test if the blob exists.
         /// </summary>
         /// <returns>True if exists. False otherwise.</returns>


[05/28] reef git commit: [REEF-1733] Define Driver Metrics and observer

Posted by do...@apache.org.
[REEF-1733] Define Driver Metrics and observer

   * Add interface for DriverMetrics and impl
   * Add DriverMetricsObservers and configuration module
   * Let MetricsService implement IDriverMetrics observer
   * Update test driver and test case

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

This closes #1342


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

Branch: refs/heads/REEF-335
Commit: af6c39a214321f7c1969baf694641af3ee9ce828
Parents: e90ffbe
Author: Julia Wang <ju...@apache.org>
Authored: Tue Jul 25 19:29:42 2017 -0700
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Wed Nov 1 12:06:55 2017 -0700

----------------------------------------------------------------------
 .../Org.Apache.REEF.Common.csproj               |  4 ++
 .../Telemetry/CountersData.cs                   |  9 +--
 .../Telemetry/DefaultMetricsSink.cs             |  4 +-
 .../Telemetry/DriverMetrics.cs                  | 39 +++++++++++++
 .../DriverMetricsObserverConfigurationModule.cs | 45 +++++++++++++++
 .../Telemetry/DriverMetricsObservers.cs         | 31 ++++++++++
 .../Telemetry/IDriverMetrics.cs                 | 38 +++++++++++++
 .../Telemetry/IMetricsSink.cs                   |  2 +-
 .../Telemetry/MetricsService.cs                 | 34 ++++++++---
 .../Functional/Telemetry/MetricsDriver.cs       | 60 ++++++++++++++++++--
 .../Functional/Telemetry/TestMetricsMessage.cs  | 13 ++++-
 11 files changed, 255 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/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 518bdfb..8a82948 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
@@ -253,9 +253,13 @@ under the License.
     <Compile Include="Telemetry\Counters.cs" />
     <Compile Include="Telemetry\CounterSinkThreshold.cs" />
     <Compile Include="Telemetry\DefaultMetricsSink.cs" />
+    <Compile Include="Telemetry\DriverMetrics.cs" />
+    <Compile Include="Telemetry\DriverMetricsObserverConfigurationModule.cs" />
+    <Compile Include="Telemetry\DriverMetricsObservers.cs" />
     <Compile Include="Telemetry\EvaluatorMetrics.cs" />
     <Compile Include="Telemetry\ICounter.cs" />
     <Compile Include="Telemetry\ICounters.cs" />
+    <Compile Include="Telemetry\IDriverMetrics.cs" />
     <Compile Include="Telemetry\IEvaluatorMetrics.cs" />
     <Compile Include="Telemetry\IMetricsSink.cs" />
     <Compile Include="Telemetry\MessageSenderConfigurationModule.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Common/Telemetry/CountersData.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/CountersData.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/CountersData.cs
index 55393b0..b8c22c8 100644
--- a/lang/cs/Org.Apache.REEF.Common/Telemetry/CountersData.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/CountersData.cs
@@ -81,14 +81,9 @@ namespace Org.Apache.REEF.Common.Telemetry
         /// Convert the counter data into ISet for sink
         /// </summary>
         /// <returns></returns>
-        internal ISet<KeyValuePair<string, string>> GetCounterData()
+        internal IEnumerable<KeyValuePair<string, string>> GetCounterData()
         {
-            var set = new HashSet<KeyValuePair<string, string>>();
-            foreach (var c in _counterMap)
-            {
-                set.Add(c.Value.GetKeyValuePair());
-            }
-            return set;
+            return _counterMap.Select(counter => counter.Value.GetKeyValuePair());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Common/Telemetry/DefaultMetricsSink.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/DefaultMetricsSink.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/DefaultMetricsSink.cs
index d302812..7f4fd95 100644
--- a/lang/cs/Org.Apache.REEF.Common/Telemetry/DefaultMetricsSink.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/DefaultMetricsSink.cs
@@ -38,8 +38,8 @@ namespace Org.Apache.REEF.Common.Telemetry
         /// <summary>
         /// Simple sink for metrics data
         /// </summary>
-        /// <param name="metrics"></param>
-        public void Sink(ISet<KeyValuePair<string, string>> metrics)
+        /// <param name="metrics">A collection of metrics data in Key value pair format.</param>
+        public void Sink(IEnumerable<KeyValuePair<string, string>> metrics)
         {
             foreach (var m in metrics)
             {

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetrics.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetrics.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetrics.cs
new file mode 100644
index 0000000..2d634e3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetrics.cs
@@ -0,0 +1,39 @@
+// 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;
+
+namespace Org.Apache.REEF.Common.Telemetry
+{
+    /// <summary>
+    /// A simple driver metrics.
+    /// It contains system state for now.
+    /// It can be extended later to include more driver metrics data.
+    /// </summary>
+    public sealed class DriverMetrics : IDriverMetrics
+    {
+        public DriverMetrics(string systemState, DateTime timeUpdated)
+        {
+            SystemState = systemState;
+            TimeUpdated = timeUpdated;
+        }
+
+        public string SystemState { get; private set; }
+
+        public DateTime TimeUpdated { get; private set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetricsObserverConfigurationModule.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetricsObserverConfigurationModule.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetricsObserverConfigurationModule.cs
new file mode 100644
index 0000000..d7e00e5
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetricsObserverConfigurationModule.cs
@@ -0,0 +1,45 @@
+// 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.Formats;
+using Org.Apache.REEF.Tang.Util;
+
+namespace Org.Apache.REEF.Common.Telemetry
+{
+    /// <summary>
+    /// It provides ConfigurationModule for DriverMetrics observers
+    /// </summary>
+    public sealed class DriverMetricsObserverConfigurationModule : ConfigurationModuleBuilder
+    {
+        /// <summary>
+        /// Observer of driver metrics
+        /// </summary>
+        public static readonly OptionalImpl<IObserver<IDriverMetrics>> OnDriverMetrics =
+            new OptionalImpl<IObserver<IDriverMetrics>>();
+
+        /// <summary>
+        /// Configuration module for driver metrics observer.
+        /// MetricsService is added as an observer.
+        /// User can set more observers with this configuration module.
+        /// </summary>
+        public readonly static ConfigurationModule ConfigurationModule = new DriverMetricsObserverConfigurationModule()
+            .BindSetEntry(GenericType<DriverMetricsObservers>.Class, OnDriverMetrics)
+            .BindSetEntry<DriverMetricsObservers, MetricsService, IObserver<IDriverMetrics>>(GenericType<DriverMetricsObservers>.Class, GenericType<MetricsService>.Class)
+            .Build();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetricsObservers.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetricsObservers.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetricsObservers.cs
new file mode 100644
index 0000000..21a3f4a
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/DriverMetricsObservers.cs
@@ -0,0 +1,31 @@
+// 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 Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.Common.Telemetry
+{
+    /// <summary>
+    /// Named parameter for a set of driver metrics observers
+    /// </summary>
+    [NamedParameter(documentation: "Observers for Driver Metrics", shortName: "DriverMetricsObservers")]
+    public class DriverMetricsObservers : Name<ISet<IObserver<IDriverMetrics>>>
+    {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Common/Telemetry/IDriverMetrics.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/IDriverMetrics.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/IDriverMetrics.cs
new file mode 100644
index 0000000..4f2c05d
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/IDriverMetrics.cs
@@ -0,0 +1,38 @@
+// 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.Annotations;
+using Org.Apache.REEF.Utilities.Attributes;
+
+namespace Org.Apache.REEF.Common.Telemetry
+{
+    [Unstable("0.16", "This is to build a simple metrics with system state only. More metrics will be added in future.")]
+    [DefaultImplementation(typeof(DriverMetrics))]
+    public interface IDriverMetrics
+    {
+        /// <summary>
+        /// System state
+        /// </summary>
+        string SystemState { get; }
+
+        /// <summary>
+        /// DateTime that the system state is updated
+        /// </summary>
+        DateTime TimeUpdated { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Common/Telemetry/IMetricsSink.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/IMetricsSink.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/IMetricsSink.cs
index b27bd3d..eef54db 100644
--- a/lang/cs/Org.Apache.REEF.Common/Telemetry/IMetricsSink.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/IMetricsSink.cs
@@ -28,6 +28,6 @@ namespace Org.Apache.REEF.Common.Telemetry
     [DefaultImplementation(typeof(DefaultMetricsSink))]
     public interface IMetricsSink : IDisposable
     {
-        void Sink(ISet<KeyValuePair<string, string>> metrics);
+        void Sink(IEnumerable<KeyValuePair<string, string>> metrics);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs b/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs
index 75c8cc2..d0bf196 100644
--- a/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs
+++ b/lang/cs/Org.Apache.REEF.Common/Telemetry/MetricsService.cs
@@ -31,7 +31,7 @@ namespace Org.Apache.REEF.Common.Telemetry
     /// Metrics service. It is also a context message handler.
     /// </summary>
     [Unstable("0.16", "This is a simple MetricsService. More functionalities will be added.")]
-    internal sealed class MetricsService : IObserver<IContextMessage>
+    internal sealed class MetricsService : IObserver<IContextMessage>, IObserver<IDriverMetrics>
     {
         private static readonly Logger Logger = Logger.GetLogger(typeof(MetricsService));
 
@@ -47,7 +47,8 @@ namespace Org.Apache.REEF.Common.Telemetry
 
         /// <summary>
         /// The threshold that triggers the sinks. 
-        /// Currently only one threshold is defined for all the counters. Later, it can be extended to define a threshold per counter.
+        /// Currently only one threshold is defined for all the counters.
+        /// Later, it can be extended to define a threshold per counter.
         /// </summary>
         private readonly int _counterSinkThreshold;
 
@@ -73,7 +74,9 @@ namespace Org.Apache.REEF.Common.Telemetry
         {
             var msgReceived = ByteUtilities.ByteArraysToString(contextMessage.Message);
             var counters = new EvaluatorMetrics(msgReceived).GetMetricsCounters();
-            Logger.Log(Level.Info, "Received {0} counters with context message: {1}.", counters.GetCounters().Count(), msgReceived);
+
+            Logger.Log(Level.Verbose, "Received {0} counters with context message: {1}.",
+                counters.GetCounters().Count(), msgReceived);
 
             _countersData.Update(counters);
 
@@ -87,17 +90,17 @@ namespace Org.Apache.REEF.Common.Telemetry
         /// <summary>
         /// Call each Sink to sink the data in the counters
         /// </summary>
-        private void Sink(ISet<KeyValuePair<string, string>> set)
+        private void Sink(IEnumerable<KeyValuePair<string, string>> metrics)
         {
             foreach (var s in _metricsSinks)
             {
                 try
                 {
-                    Task.Run(() => s.Sink(set));
+                    Task.Run(() => s.Sink(metrics));
                 }
                 catch (Exception e)
                 {
-                    Logger.Log(Level.Error, "Exception happens during the sink for Sink {0} with Exception: {1}.", s.GetType().AssemblyQualifiedName, e);
+                    Logger.Log(Level.Error, "Exception in Sink " + s.GetType().AssemblyQualifiedName, e);
                 }
                 finally
                 {
@@ -108,10 +111,27 @@ namespace Org.Apache.REEF.Common.Telemetry
 
         public void OnCompleted()
         {
+            Logger.Log(Level.Info, "Completed");
         }
 
         public void OnError(Exception error)
         {
+            Logger.Log(Level.Error, "MetricService error", error);
+        }
+
+        /// <summary>
+        /// Observer of IDriverMetrics.
+        /// When Driver metrics data is changed, this method will be called.
+        /// It calls Sink to store/log the metrics data.
+        /// </summary>
+        /// <param name="driverMetrics">driver metrics data.</param>
+        public void OnNext(IDriverMetrics driverMetrics)
+        {
+            Sink(new Dictionary<string, string>()
+            {
+                { "SystemState", driverMetrics.SystemState },
+                { "TimeUpdated", driverMetrics.TimeUpdated.ToLongTimeString() }
+            });
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/MetricsDriver.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/MetricsDriver.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/MetricsDriver.cs
index eb88cf0..73df585 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/MetricsDriver.cs
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/MetricsDriver.cs
@@ -16,6 +16,7 @@
 // under the License.
 
 using System;
+using System.Collections.Generic;
 using Org.Apache.REEF.Common.Context;
 using Org.Apache.REEF.Common.Services;
 using Org.Apache.REEF.Common.Tasks;
@@ -23,6 +24,7 @@ using Org.Apache.REEF.Common.Telemetry;
 using Org.Apache.REEF.Driver;
 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.Implementations.Configuration;
 using Org.Apache.REEF.Tang.Util;
@@ -31,22 +33,42 @@ using Org.Apache.REEF.Utilities.Logging;
 
 namespace Org.Apache.REEF.Tests.Functional.Telemetry
 {
+    /// <summary>
+    /// Test driver to test metrics
+    /// </summary>
     class MetricsDriver :
         IObserver<IDriverStarted>,
         IObserver<IAllocatedEvaluator>,
-        IObserver<IActiveContext>
+        IObserver<IActiveContext>,
+        IObserver<ICompletedTask>
     {
         private static readonly Logger Logger = Logger.GetLogger(typeof(MessageDriver));
         private readonly IEvaluatorRequestor _evaluatorRequestor;
+        internal const string EventPrefix = "TestState";
 
+        /// <summary>
+        /// a set of driver metrics observers.
+        /// </summary>
+        private readonly ISet<IObserver<IDriverMetrics>> _driverMetricsObservers;
+
+        /// <summary>
+        /// This driver inject DriverMetricsObservers and IDriverMetrics.
+        /// It keeps updating the driver metrics when receiving events. 
+        /// </summary>
+        /// <param name="evaluatorRequestor"></param>
+        /// <param name="driverMetricsObservers"></param>
         [Inject]
-        public MetricsDriver(IEvaluatorRequestor evaluatorRequestor)
+        public MetricsDriver(IEvaluatorRequestor evaluatorRequestor,
+            [Parameter(typeof(DriverMetricsObservers))] ISet<IObserver<IDriverMetrics>> driverMetricsObservers)
         {
             _evaluatorRequestor = evaluatorRequestor;
+            _driverMetricsObservers = driverMetricsObservers;
         }
 
         public void OnNext(IDriverStarted value)
         {
+            UpdateMetrics(TestSystemState.DriverStarted);
+
             var request =
                 _evaluatorRequestor.NewBuilder()
                     .SetNumber(1)
@@ -61,14 +83,14 @@ namespace Org.Apache.REEF.Tests.Functional.Telemetry
         public void OnNext(IAllocatedEvaluator value)
         {
             Logger.Log(Level.Info, "Received IAllocatedEvaluator");
-            const string contextId = "ContextID";
+            UpdateMetrics(TestSystemState.EvaluatorAllocated);
 
+            const string contextId = "ContextID";
             var serviceConfiguration = ServiceConfiguration.ConfigurationModule
                 .Build();
 
             var contextConfiguration1 = ContextConfiguration.ConfigurationModule
                 .Set(ContextConfiguration.Identifier, contextId)
-                ////.Set(ContextConfiguration.OnSendMessage, GenericType<MetricsMessageSender>.Class)
                 .Build();
 
             var contextConfiguration2 = MessageSenderConfigurationModule.ConfigurationModule.Build();
@@ -80,6 +102,7 @@ namespace Org.Apache.REEF.Tests.Functional.Telemetry
         public void OnNext(IActiveContext activeContext)
         {
             Logger.Log(Level.Info, "Received IActiveContext");
+            UpdateMetrics(TestSystemState.ActiveContextReceived);
 
             const string taskId = "TaskID";
             var taskConfiguration = TaskConfiguration.ConfigurationModule
@@ -89,6 +112,14 @@ namespace Org.Apache.REEF.Tests.Functional.Telemetry
             activeContext.SubmitTask(taskConfiguration);
         }
 
+        public void OnNext(ICompletedTask value)
+        {
+            Logger.Log(Level.Info, "Received ICompletedTask");
+            UpdateMetrics(TestSystemState.TaskCompleted);
+
+            value.ActiveContext.Dispose();
+        }
+
         public void OnCompleted()
         {
             throw new NotImplementedException();
@@ -98,5 +129,26 @@ namespace Org.Apache.REEF.Tests.Functional.Telemetry
         {
             throw new NotImplementedException();
         }
+
+        /// <summary>
+        /// Call metrics observers with driver metrics data
+        /// </summary>
+        private void UpdateMetrics(TestSystemState systemState)
+        {
+            var driverMetrics = new DriverMetrics(EventPrefix + systemState, DateTime.Now);
+
+            foreach (var metricsObserver in _driverMetricsObservers)
+            {
+                metricsObserver.OnNext(driverMetrics);
+            }
+        }
+    }
+
+    internal enum TestSystemState
+    {
+        DriverStarted,
+        EvaluatorAllocated,
+        ActiveContextReceived,
+        TaskCompleted
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/reef/blob/af6c39a2/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/TestMetricsMessage.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/TestMetricsMessage.cs b/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/TestMetricsMessage.cs
index f447f75..0f10ac8 100644
--- a/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/TestMetricsMessage.cs
+++ b/lang/cs/Org.Apache.REEF.Tests/Functional/Telemetry/TestMetricsMessage.cs
@@ -40,24 +40,31 @@ namespace Org.Apache.REEF.Tests.Functional.Telemetry
             string[] lines = ReadLogFile(DriverStdout, "driver", testFolder, 240);
             var receivedCounterMessage = GetMessageCount(lines, "Received 2 counters with context message:");
             Assert.True(receivedCounterMessage > 1);
+
+            var messageCount = GetMessageCount(lines, MetricsDriver.EventPrefix);
+            Assert.Equal(4, messageCount);
+
             CleanUp(testFolder);
         }
 
         private static IConfiguration DriverConfigurations()
         {
-            var c1 = DriverConfiguration.ConfigurationModule
+            var driverBasicConfig = DriverConfiguration.ConfigurationModule
                 .Set(DriverConfiguration.OnDriverStarted, GenericType<MetricsDriver>.Class)
                 .Set(DriverConfiguration.OnEvaluatorAllocated, GenericType<MetricsDriver>.Class)
                 .Set(DriverConfiguration.OnContextActive, GenericType<MetricsDriver>.Class)
+                .Set(DriverConfiguration.OnTaskCompleted, GenericType<MetricsDriver>.Class)
                 .Set(DriverConfiguration.CustomTraceLevel, Level.Info.ToString())
                 .Build();
 
-            var c2 = MetricsServiceConfigurationModule.ConfigurationModule
+            var metricServiceConfig = MetricsServiceConfigurationModule.ConfigurationModule
                 .Set(MetricsServiceConfigurationModule.OnMetricsSink, GenericType<DefaultMetricsSink>.Class)
                 .Set(MetricsServiceConfigurationModule.CounterSinkThreshold, "5")
                 .Build();
 
-            return Configurations.Merge(c1, c2);
+            var driverMetricConfig = DriverMetricsObserverConfigurationModule.ConfigurationModule.Build();
+
+            return Configurations.Merge(driverBasicConfig, metricServiceConfig, driverMetricConfig);
         }
     }
 }


[03/28] reef git commit: [REEF-1935] Improve logging in Wake NettyMessagingTransport and related classes

Posted by do...@apache.org.
[REEF-1935] Improve logging in Wake NettyMessagingTransport and related classes

Summary of changes:
   * Implement `.toString()` for `LinkReference`
   * Implement `.toString()` for `RemoteEventEncoder`
   * Implement `.toString()` for `NettyMessagingTransport`
   * Improve logging and remove redundant code in `NettyMessagingTransport` constructor

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

Pull Request:
  This closes #1401


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

Branch: refs/heads/REEF-335
Commit: 48e65c0dcb52a47774dbb09300c0a142becf52d5
Parents: daf33d5
Author: Sergiy Matusevych <mo...@apache.com>
Authored: Wed Oct 25 16:37:35 2017 -0700
Committer: Markus Weimer <we...@apache.org>
Committed: Wed Oct 25 20:38:33 2017 -0700

----------------------------------------------------------------------
 .../wake/remote/impl/RemoteEventDecoder.java    |  4 +
 .../wake/remote/impl/RemoteEventEncoder.java    |  4 +
 .../remote/transport/netty/LinkReference.java   |  5 ++
 .../netty/NettyMessagingTransport.java          | 84 +++++++++-----------
 4 files changed, 49 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/48e65c0d/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventDecoder.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventDecoder.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventDecoder.java
index fdf10b2..f65646f 100644
--- a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventDecoder.java
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventDecoder.java
@@ -59,4 +59,8 @@ public class RemoteEventDecoder<T> implements Decoder<RemoteEvent<T>> {
     }
   }
 
+  @Override
+  public String toString() {
+    return String.format("RemoteEventDecoder: { decoder: %s }", this.decoder);
+  }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/48e65c0d/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventEncoder.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventEncoder.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventEncoder.java
index 29e3be6..4a3fa0b 100644
--- a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventEncoder.java
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/impl/RemoteEventEncoder.java
@@ -61,4 +61,8 @@ public class RemoteEventEncoder<T> implements Encoder<RemoteEvent<T>> {
     return builder.build().toByteArray();
   }
 
+  @Override
+  public String toString() {
+    return String.format("RemoteEventEncoder: { encoder: %s }", this.encoder);
+  }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/48e65c0d/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/LinkReference.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/LinkReference.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/LinkReference.java
index 6e780b8..21cd4d5 100644
--- a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/LinkReference.java
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/LinkReference.java
@@ -49,4 +49,9 @@ final class LinkReference {
   AtomicInteger getConnectInProgress() {
     return this.connectInProgress;
   }
+
+  @Override
+  public String toString() {
+    return String.format("LinkReference: { link: %s }", this.link); // NettyLink has a good .toString() implementation
+  }
 }

http://git-wip-us.apache.org/repos/asf/reef/blob/48e65c0d/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/NettyMessagingTransport.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/NettyMessagingTransport.java b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/NettyMessagingTransport.java
index 2643030..ea221ec 100644
--- a/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/NettyMessagingTransport.java
+++ b/lang/java/reef-wake/wake/src/main/java/org/apache/reef/wake/remote/transport/netty/NettyMessagingTransport.java
@@ -54,7 +54,6 @@ import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -86,14 +85,12 @@ public final class NettyMessagingTransport implements Transport {
   private final EventLoopGroup serverWorkerGroup;
 
   private final Bootstrap clientBootstrap;
-  private final ServerBootstrap serverBootstrap;
   private final Channel acceptor;
 
   private final ChannelGroup clientChannelGroup = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
   private final ChannelGroup serverChannelGroup = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
 
-  private final int serverPort;
-  private final SocketAddress localAddress;
+  private final InetSocketAddress localAddress;
 
   private final NettyClientEventListener clientEventListener;
   private final NettyServerEventListener serverEventListener;
@@ -105,7 +102,7 @@ public final class NettyMessagingTransport implements Transport {
    * Constructs a messaging transport.
    *
    * @param hostAddress   the server host address
-   * @param port          the server listening port; when it is 0, randomly assign a port number
+   * @param listenPort    the server listening port; when it is 0, randomly assign a port number
    * @param clientStage   the client-side stage that handles transport events
    * @param serverStage   the server-side stage that handles transport events
    * @param numberOfTries the number of tries of connection
@@ -115,7 +112,7 @@ public final class NettyMessagingTransport implements Transport {
   @Inject
   private NettyMessagingTransport(
       @Parameter(RemoteConfiguration.HostAddress.class) final String hostAddress,
-      @Parameter(RemoteConfiguration.Port.class) final int port,
+      @Parameter(RemoteConfiguration.Port.class) final int listenPort,
       @Parameter(RemoteConfiguration.RemoteClientStage.class) final EStage<TransportEvent> clientStage,
       @Parameter(RemoteConfiguration.RemoteServerStage.class) final EStage<TransportEvent> serverStage,
       @Parameter(RemoteConfiguration.NumberOfTries.class) final int numberOfTries,
@@ -123,9 +120,8 @@ public final class NettyMessagingTransport implements Transport {
       final TcpPortProvider tcpPortProvider,
       final LocalAddressProvider localAddressProvider) {
 
-    int p = port;
-    if (p < 0) {
-      throw new RemoteRuntimeException("Invalid server port: " + p);
+    if (listenPort < 0) {
+      throw new RemoteRuntimeException("Invalid server port: " + listenPort);
     }
 
     final String host = UNKNOWN_HOST_NAME.equals(hostAddress) ? localAddressProvider.getLocalAddress() : hostAddress;
@@ -142,16 +138,16 @@ public final class NettyMessagingTransport implements Transport {
     this.clientWorkerGroup = new NioEventLoopGroup(CLIENT_WORKER_NUM_THREADS,
         new DefaultThreadFactory(CLASS_NAME + ":ClientWorker"));
 
-    this.clientBootstrap = new Bootstrap();
-    this.clientBootstrap.group(this.clientWorkerGroup)
+    this.clientBootstrap = new Bootstrap()
+        .group(this.clientWorkerGroup)
         .channel(NioSocketChannel.class)
         .handler(new NettyChannelInitializer(new NettyDefaultChannelHandlerFactory("client",
             this.clientChannelGroup, this.clientEventListener)))
         .option(ChannelOption.SO_REUSEADDR, true)
         .option(ChannelOption.SO_KEEPALIVE, true);
 
-    this.serverBootstrap = new ServerBootstrap();
-    this.serverBootstrap.group(this.serverBossGroup, this.serverWorkerGroup)
+    final ServerBootstrap serverBootstrap = new ServerBootstrap()
+        .group(this.serverBossGroup, this.serverWorkerGroup)
         .channel(NioServerSocketChannel.class)
         .childHandler(new NettyChannelInitializer(new NettyDefaultChannelHandlerFactory("server",
             this.serverChannelGroup, this.serverEventListener)))
@@ -159,56 +155,43 @@ public final class NettyMessagingTransport implements Transport {
         .option(ChannelOption.SO_REUSEADDR, true)
         .childOption(ChannelOption.SO_KEEPALIVE, true);
 
-    LOG.log(Level.FINE, "Binding to {0}", p);
+    LOG.log(Level.FINE, "Binding to {0}:{1}", new Object[] {host, listenPort});
 
-    Channel acceptorFound = null;
     try {
-      if (p > 0) {
-        acceptorFound = this.serverBootstrap.bind(new InetSocketAddress(host, p)).sync().channel();
+      if (listenPort > 0) {
+        this.localAddress = new InetSocketAddress(host, listenPort);
+        this.acceptor = serverBootstrap.bind(this.localAddress).sync().channel();
       } else {
-        final Iterator<Integer> ports = tcpPortProvider.iterator();
-        while (acceptorFound == null) {
-          if (!ports.hasNext()) {
-            throw new IllegalStateException("tcpPortProvider cannot find a free port.");
-          }
-          p = ports.next();
-          LOG.log(Level.FINEST, "Try port {0}", p);
+        InetSocketAddress socketAddr = null;
+        Channel acceptorFound = null;
+        for (int port : tcpPortProvider) {
+          LOG.log(Level.FINEST, "Try port {0}", port);
           try {
-            acceptorFound = this.serverBootstrap.bind(new InetSocketAddress(host, p)).sync().channel();
+            socketAddr = new InetSocketAddress(host, port);
+            acceptorFound = serverBootstrap.bind(socketAddr).sync().channel();
+            break;
           } catch (final Exception ex) {
-            if (ex instanceof BindException) {
-              LOG.log(Level.FINEST, "The port {0} is already bound. Try again", p);
+            if (ex instanceof BindException) { // Not visible to catch :(
+              LOG.log(Level.FINEST, "The port {0} is already bound. Try again", port);
             } else {
               throw ex;
             }
           }
         }
+        if (acceptorFound == null) {
+          throw new IllegalStateException("TcpPortProvider could not find a free port.");
+        }
+        this.localAddress = socketAddr;
+        this.acceptor = acceptorFound;
       }
-    } catch (final IllegalStateException ex) {
-      final RuntimeException transportException =
-                new TransportRuntimeException("tcpPortProvider failed to return free ports.", ex);
-      LOG.log(Level.SEVERE, "Cannot find a free port with " + tcpPortProvider, transportException);
-
+    } catch (final IllegalStateException | InterruptedException ex) {
+      LOG.log(Level.SEVERE, "Cannot bind to port " + listenPort, ex);
       this.clientWorkerGroup.shutdownGracefully();
       this.serverBossGroup.shutdownGracefully();
       this.serverWorkerGroup.shutdownGracefully();
-      throw transportException;
-
-    } catch (final Exception ex) {
-      final RuntimeException transportException =
-          new TransportRuntimeException("Cannot bind to port " + p, ex);
-      LOG.log(Level.SEVERE, "Cannot bind to port " + p, ex);
-
-      this.clientWorkerGroup.shutdownGracefully();
-      this.serverBossGroup.shutdownGracefully();
-      this.serverWorkerGroup.shutdownGracefully();
-      throw transportException;
+      throw new TransportRuntimeException("Cannot bind to port " + listenPort, ex);
     }
 
-    this.acceptor = acceptorFound;
-    this.serverPort = p;
-    this.localAddress = new InetSocketAddress(host, this.serverPort);
-
     LOG.log(Level.FINE, "Starting netty transport socket address: {0}", this.localAddress);
   }
 
@@ -372,7 +355,7 @@ public final class NettyMessagingTransport implements Transport {
    */
   @Override
   public int getListeningPort() {
-    return this.serverPort;
+    return this.localAddress.getPort();
   }
 
   /**
@@ -385,4 +368,9 @@ public final class NettyMessagingTransport implements Transport {
     this.clientEventListener.registerErrorHandler(handler);
     this.serverEventListener.registerErrorHandler(handler);
   }
+
+  @Override
+  public String toString() {
+    return String.format("NettyMessagingTransport: { address: %s }", this.localAddress);
+  }
 }


[06/28] reef git commit: [REEF-1944] Remove Oracle JDK 7 from Travis CI

Posted by do...@apache.org.
[REEF-1944] Remove Oracle JDK 7 from Travis CI

Also, add a note about JDK versions used in CI

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

Pull request:
   This closes #1407


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

Branch: refs/heads/REEF-335
Commit: 49fbac27548793f4a036b1aa461d1e7cb3c1d33b
Parents: af6c39a
Author: Mariia Mykhailova <mi...@gmail.com>
Authored: Wed Nov 1 11:27:34 2017 -0700
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Wed Nov 1 13:37:16 2017 -0700

----------------------------------------------------------------------
 .travis.yml        | 4 +---
 lang/java/BUILD.md | 2 +-
 2 files changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/49fbac27/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index e460051..3965f34 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -14,13 +14,11 @@
 # limitations under the License.
 
 sudo: false
+dist: trusty
 
 language: java
 
-dist: precise
-
 jdk:
-  - oraclejdk7
   - oraclejdk8
   - openjdk7
 

http://git-wip-us.apache.org/repos/asf/reef/blob/49fbac27/lang/java/BUILD.md
----------------------------------------------------------------------
diff --git a/lang/java/BUILD.md b/lang/java/BUILD.md
index b3633fa..d8c4e95 100644
--- a/lang/java/BUILD.md
+++ b/lang/java/BUILD.md
@@ -92,7 +92,7 @@ Continuous Integration
 ------------
 
 We use [Travis CI](https://travis-ci.org/) to run continuous integration for REEF Java code (i.e. build and run tests
-for all pull requests and commits to master branch).
+for all pull requests and commits to master branch). Our current setup runs 2 builds for each commit: Oracle JDK 8 and OpenJDK 7; REEF can run on Oracle JDK 7 but it has been dropped from Travis build images.
 
 It can be convenient to set up Travis for your fork of REEF repository, for example, to reproduce a test failure which
 can't be reproduced locally.


[16/28] reef git commit: [REEF-1959] Fix bug in ClassPathBuilder#addAllToSuffix

Posted by do...@apache.org.
[REEF-1959] Fix bug in ClassPathBuilder#addAllToSuffix

This fixes a wrong method call inside of the addAllToSuffix method

JIRA:
  [REEF-1959](https://issues.apache.org/jira/projects/REEF/issues/REEF-1959)

Pull Request:
  Closes #1416


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

Branch: refs/heads/REEF-335
Commit: aeb730aa3b27cee4815093e1961aad0c5a425462
Parents: 3b89926
Author: Geon-Woo Kim <gw...@toss.im>
Authored: Sun Nov 12 17:47:32 2017 +0900
Committer: John Yang <jo...@apache.org>
Committed: Sun Nov 12 19:53:29 2017 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/reef/runtime/yarn/ClassPathBuilder.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/aeb730aa/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/ClassPathBuilder.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/ClassPathBuilder.java b/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/ClassPathBuilder.java
index 72f14f5..9b53d8c 100644
--- a/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/ClassPathBuilder.java
+++ b/lang/java/reef-runtime-yarn/src/main/java/org/apache/reef/runtime/yarn/ClassPathBuilder.java
@@ -100,7 +100,7 @@ final class ClassPathBuilder {
    */
   void addAllToSuffix(final String... entries) {
     for (final String classPathEntry : entries) {
-      this.addToPrefix(classPathEntry);
+      this.addToSuffix(classPathEntry);
     }
   }
 


[18/28] reef git commit: [REEF-1954] Require Visual Studio 2017

Posted by do...@apache.org.
[REEF-1954] Require Visual Studio 2017

This change

   * Updates `lang/cs/build.md` to require Visual Studio 2017
   * Updates `appveyor.yml` to only build on Visual Studio 2017
   * Updated the solutions file to require Visual Studio 2017

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

Pull Request:
  This closes #1415


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

Branch: refs/heads/REEF-335
Commit: cc1073f6c008da08d0580d2b0c15febe95515b47
Parents: 821ef2a
Author: Markus Weimer <we...@apache.org>
Authored: Wed Nov 8 16:08:19 2017 -0800
Committer: Sergiy Matusevych <mo...@apache.com>
Committed: Thu Nov 16 16:43:09 2017 -0800

----------------------------------------------------------------------
 appveyor.yml                |   2 --
 lang/cs/BUILD.md            |   4 ++--
 lang/cs/Org.Apache.REEF.sln | Bin 37118 -> 37102 bytes
 3 files changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/cc1073f6/appveyor.yml
----------------------------------------------------------------------
diff --git a/appveyor.yml b/appveyor.yml
index c882e23..e28c0b0 100644
--- a/appveyor.yml
+++ b/appveyor.yml
@@ -18,8 +18,6 @@ version: "{build}-{branch}"
 shallow_clone: true
 
 os:
- - Visual Studio 2013
- - Visual Studio 2015
  - Visual Studio 2017
 
 platform: x64

http://git-wip-us.apache.org/repos/asf/reef/blob/cc1073f6/lang/cs/BUILD.md
----------------------------------------------------------------------
diff --git a/lang/cs/BUILD.md b/lang/cs/BUILD.md
index dad75d5..00d4b17 100644
--- a/lang/cs/BUILD.md
+++ b/lang/cs/BUILD.md
@@ -34,9 +34,9 @@ Prerequisites
   * Add the following items to the environment variable `PATH`:
      * Add the location of the `protoc.exe` executable in the windows path
      * Add `%JAVA_HOME%/bin` and `%M2_HOME%/bin` to the windows path as well
-  * [Visual Studio](http://www.visualstudio.com) 2015 (preferred) or 2013. Most REEF developers use the free Community
+  * [Visual Studio](http://www.visualstudio.com) 2017 V. 15.4.3 or newer. Most REEF developers use the free Community
     Edition.
-  * NuGet 2.8.6 or later. (Included in VS 2015)
+  * NuGet 4.0.0 or later. (Included in VS 2017)
   * xunit.runner.console.2.1.0 package (installing it might require manual restore of NuGet packages).
 
 

http://git-wip-us.apache.org/repos/asf/reef/blob/cc1073f6/lang/cs/Org.Apache.REEF.sln
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.sln b/lang/cs/Org.Apache.REEF.sln
index 3889f24..b4cbccd 100644
Binary files a/lang/cs/Org.Apache.REEF.sln and b/lang/cs/Org.Apache.REEF.sln differ


[28/28] reef git commit: [REEF-1976] Implement Azure Data Lake Storage FileSystem

Posted by do...@apache.org.
[REEF-1976] Implement Azure Data Lake Storage FileSystem

   * Implement Azure Data Lake Storage FileSystem
   * Add unit tests checking exceptions for azure data lake filesystem methods (#9)
   * Add unit tests checking exceptions for azure data lake filesystem methods
   * upgrade Newtonsoft.Json to 10.0.0.3
   * add doc link for AD app creation

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

Closes #1427


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

Branch: refs/heads/REEF-335
Commit: b759764c7b67afd6cd37044029901ec8a9adba72
Parents: baa5be8
Author: dwaijam <dw...@gmail.com>
Authored: Thu Jan 25 16:46:08 2018 -0800
Committer: Sergiy Matusevych <mo...@apache.org>
Committed: Wed Feb 21 13:01:53 2018 -0800

----------------------------------------------------------------------
 .../Org.Apache.REEF.IO.Tests.csproj             |  35 ++-
 .../TestAzureBlockBlobFileSystemE2E.cs          |  10 +-
 .../TestAzureDataLakeFileSystem.cs              | 234 +++++++++++++++++
 .../TestAzureDataLakeFileSystemE2E.cs           | 251 +++++++++++++++++++
 .../cs/Org.Apache.REEF.IO.Tests/packages.config |   8 +-
 .../AzureDataLake/AzureDataLakeFileSystem.cs    | 232 +++++++++++++++++
 .../AzureDataLakeFileSystemConfiguration.cs     |  71 ++++++
 ...reDataLakeFileSystemConfigurationProvider.cs |  61 +++++
 .../AzureDataLake/AzureDataLakeStoreClient.cs   |  44 ++++
 .../AzureDataLake/IAdlsCredentials.cs           |  28 +++
 .../AzureDataLake/IDataLakeStoreClient.cs       |  36 +++
 .../AzureDataLake/Parameters/ClientId.cs        |  29 +++
 .../Parameters/DataLakeStorageAccountName.cs    |  29 +++
 .../AzureDataLake/Parameters/SecretKey.cs       |  29 +++
 .../AzureDataLake/Parameters/Tenant.cs          |  29 +++
 .../AzureDataLake/Parameters/TokenAudience.cs   |  29 +++
 .../AzureDataLake/SecretKeyAdlsCredentials.cs   |  45 ++++
 .../Hadoop/Parameters/CommandTimeOut.cs         |   2 +-
 .../Org.Apache.REEF.IO.csproj                   |  43 +++-
 lang/cs/Org.Apache.REEF.IO/packages.config      |   7 +
 20 files changed, 1240 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj b/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
index 5a7e323..5d688cc 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/Org.Apache.REEF.IO.Tests.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -34,8 +34,34 @@ under the License.
     <BuildPackage>false</BuildPackage>
   </PropertyGroup>
   <ItemGroup>
+    <Reference Include="Microsoft.Azure.DataLake.Store, Version=1.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Azure.DataLake.Store.1.0.4\lib\net452\Microsoft.Azure.DataLake.Store.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.IdentityModel.Clients.ActiveDirectory, Version=2.28.3.860, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.IdentityModel.Clients.ActiveDirectory.2.28.3\lib\net45\Microsoft.IdentityModel.Clients.ActiveDirectory.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.IdentityModel.Clients.ActiveDirectory.WindowsForms, Version=2.28.3.860, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.IdentityModel.Clients.ActiveDirectory.2.28.3\lib\net45\Microsoft.IdentityModel.Clients.ActiveDirectory.WindowsForms.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.Rest.ClientRuntime, Version=2.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Rest.ClientRuntime.2.3.10\lib\net452\Microsoft.Rest.ClientRuntime.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Rest.ClientRuntime.Azure.Authentication, Version=2.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Rest.ClientRuntime.Azure.Authentication.2.3.1\lib\net452\Microsoft.Rest.ClientRuntime.Azure.Authentication.dll</HintPath>
+    </Reference>
+    <Reference Include="Newtonsoft.Json">
+      <HintPath>$(PackagesDir)\Newtonsoft.Json.$(NewtonsoftJsonVersion)\lib\net45\Newtonsoft.Json.dll</HintPath>
+    </Reference>
+    <Reference Include="NLog, Version=4.0.0.0, Culture=neutral, PublicKeyToken=5120e14c03d0593c, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\NLog.4.4.12\lib\net45\NLog.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
+    <Reference Include="System.Management" />
+    <Reference Include="System.Net.Http" />
     <Reference Include="System.Xml.Linq" />
     <Reference Include="System.Data.DataSetExtensions" />
     <Reference Include="Microsoft.CSharp" />
@@ -58,7 +84,9 @@ under the License.
     <Compile Include="FileSystemTestUtilities.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
     <Compile Include="TestAzureBlockBlobFileSystem.cs" />
+    <Compile Include="TestAzureDataLakeFileSystemE2E.cs" />
     <Compile Include="TestAzureBlockBlobFileSystemE2E.cs" />
+    <Compile Include="TestAzureDataLakeFileSystem.cs" />
     <Compile Include="TestFilePartitionInputDataSet.cs" />
     <Compile Include="TestHadoopFileSystem.cs" />
     <Compile Include="TestLocalFileSystem.cs" />
@@ -91,7 +119,10 @@ under the License.
     <Service Include="{82A7F48D-3B50-4B1E-B82E-3ADA8210C358}" />
   </ItemGroup>
   <ItemGroup>
-    <None Include="packages.config" />
+    <None Include="$(SolutionDir)\App.config" />
+    <None Include="packages.config">
+      <SubType>Designer</SubType>
+    </None>
   </ItemGroup>
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
index 7b749e4..f5665b6 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureBlockBlobFileSystemE2E.cs
@@ -168,8 +168,9 @@ namespace Org.Apache.REEF.IO.Tests
         {
             var helloFilePath = PathToFile(HelloFile);
             var blob = _container.GetBlockBlobReference(HelloFile);
-            var tempFilePath = GetTempFilePath();
+            var tempFilePath = Path.GetTempFileName();
             const string Text = "hello";
+            
             try
             {
                 UploadFromString(blob, Text);
@@ -189,7 +190,7 @@ namespace Org.Apache.REEF.IO.Tests
             var helloFilePath = PathToFile(HelloFile);
             ICloudBlob blob = _container.GetBlockBlobReference(HelloFile);
             Assert.False(CheckBlobExists(blob));
-            var tempFilePath = GetTempFilePath();
+            var tempFilePath = Path.GetTempFileName();
             const string Text = "hello";
             try
             {
@@ -284,11 +285,6 @@ namespace Org.Apache.REEF.IO.Tests
             Assert.True(CheckContainerExists(_container));
         }
 
-        private static string GetTempFilePath()
-        {
-            return Path.Combine(Path.GetTempPath(), Guid.NewGuid().ToString());
-        }
-
         private static void UploadFromString(ICloudBlob blob, string str)
         {
             var byteArray = Encoding.UTF8.GetBytes(str);

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystem.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystem.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystem.cs
new file mode 100644
index 0000000..821e6ae
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystem.cs
@@ -0,0 +1,234 @@
+// 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 Microsoft.Azure.DataLake.Store;
+using NSubstitute;
+using Org.Apache.REEF.IO.FileSystem.AzureDataLake;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Xunit;
+
+namespace Org.Apache.REEF.IO.Tests
+{
+    public sealed class TestAzureDataLakeFileSystem
+    {
+        private static readonly Uri FakeBaseUri = new Uri("http://fakeadls.com");
+        private static readonly Uri FakeDirUri = new Uri(FakeBaseUri, "dir");
+        private static readonly Uri FakeFileUri = new Uri($"{FakeDirUri}/fakefile");
+        private readonly TestContext _context = new TestContext();
+        private readonly AzureDataLakeFileSystem _fs;
+
+        public TestAzureDataLakeFileSystem()
+        {
+            _fs = _context.GetAdlsFileSystem();
+        }
+
+        [Fact]
+        public void TestOpen()
+        {
+            _context.MockAdlsClient.CreateFile(FakeFileUri.AbsolutePath, IfExists.Overwrite);
+            var stream = _fs.Open(FakeBaseUri);
+            Assert.IsAssignableFrom<AdlsInputStream>(stream);
+        }
+
+        [Fact]
+        public void TestOpenException()
+        {
+            // Open a file that doesn't exist.
+            Exception ex = Assert.Throws<AdlsException>(() => _fs.Open(FakeFileUri));
+            Assert.IsAssignableFrom<IOException>(ex);
+        }
+
+        [Fact]
+        public void TestCreate()
+        {
+            _fs.Create(FakeFileUri);
+            Assert.True(_context.MockAdlsClient.CheckExists(FakeFileUri.AbsolutePath));
+            var directoryEntry = _context.MockAdlsClient.GetDirectoryEntry(FakeFileUri.AbsolutePath);
+            Assert.Equal(DirectoryEntryType.FILE, directoryEntry.Type);
+        }
+
+        [Fact]
+        public void TestCreateFileUnderDirectory()
+        {
+            // Checks when file is created, directory in path was properly created too
+            _fs.Create(FakeFileUri);
+            Assert.True(_context.MockAdlsClient.CheckExists(FakeDirUri.AbsolutePath));
+            var directoryEntry = _context.MockAdlsClient.GetDirectoryEntry(FakeDirUri.AbsolutePath);
+            Assert.Equal(DirectoryEntryType.DIRECTORY, directoryEntry.Type);
+        }
+
+        [Fact]
+        public void TestDelete()
+        {
+            _context.MockAdlsClient.CreateFile(FakeFileUri.AbsolutePath, IfExists.Overwrite);
+            _fs.Delete(FakeFileUri);
+            Assert.False(_context.MockAdlsClient.CheckExists(FakeFileUri.AbsolutePath));
+        }
+        
+        [Fact]
+        public void TestDeleteException()
+        {
+            // Delete a file that doesn't exist.
+            Assert.Throws<IOException>(() => _fs.Delete(FakeFileUri));
+        }
+
+        [Fact]
+        public void TestFileDoesNotExists()
+        {
+            Assert.False(_context.GetAdlsFileSystem().Exists(FakeFileUri));
+        }
+
+        [Fact]
+        public void TestExists()
+        {
+            _context.MockAdlsClient.CreateFile(FakeFileUri.AbsolutePath, IfExists.Overwrite);
+            Assert.True(_fs.Exists(FakeFileUri));
+        }
+
+        [Fact]
+        public void TestCopy()
+        {
+            // Setup
+            Uri src = new Uri($"{FakeDirUri}/copyfile");
+            _context.MockAdlsClient.CreateFile(src.AbsolutePath, IfExists.Fail);
+            Assert.True(_context.MockAdlsClient.CheckExists(src.AbsolutePath));
+            Assert.False(_context.MockAdlsClient.CheckExists(FakeFileUri.AbsolutePath));
+
+            _fs.Copy(src, FakeFileUri);
+            Assert.True(_context.MockAdlsClient.CheckExists(FakeFileUri.AbsolutePath));
+        }
+
+        [Fact]
+        public void TestCopyException()
+        {
+            // Source file does not exist
+            Uri src = new Uri($"{FakeDirUri}/copyfile");
+            Assert.Throws<IOException>(() => _fs.Copy(src, FakeFileUri));
+        }
+
+        [Fact(Skip = "This test is failing during appveyor build saying 'Currently not supported for folder' which might be because of a bug in MockAdlsClient.")]
+        public void TestCopyFromLocal()
+        {
+            Assert.False(_context.MockAdlsClient.CheckExists(FakeFileUri.AbsolutePath));
+            _fs.CopyFromLocal("fakefile", FakeFileUri);
+            Assert.True(_context.MockAdlsClient.CheckExists(FakeFileUri.AbsolutePath));
+        }
+
+        [Fact]
+        public void TestCopyToLocal()
+        {
+            _context.MockAdlsClient.CreateFile(FakeFileUri.AbsolutePath, IfExists.Overwrite);
+            _fs.CopyToLocal(FakeFileUri, Path.GetFileName(FakeFileUri.LocalPath));
+            Assert.True(File.Exists(Path.GetFileName(FakeFileUri.LocalPath)));
+        }
+
+        [Fact]
+        public void TestCopyToLocalException()
+        {
+            // Source file does not exist
+            Assert.Throws<IOException>(() => _fs.CopyToLocal(FakeFileUri, "fileName"));
+        }
+
+        [Fact]
+        public void TestCreateDirectory()
+        {
+            _fs.CreateDirectory(FakeDirUri);
+            Assert.True(_context.MockAdlsClient.CheckExists(FakeDirUri.AbsolutePath));
+            
+            // check if it is a directory and not a file
+            var directoryEntry = _context.MockAdlsClient.GetDirectoryEntry(FakeDirUri.AbsolutePath);
+            Assert.Equal(DirectoryEntryType.DIRECTORY, directoryEntry.Type); 
+        }
+
+        [Fact]
+        public void TestDeleteDirectory()
+        {
+            _context.MockAdlsClient.CreateDirectory(FakeDirUri.AbsolutePath);
+            Assert.True(_context.MockAdlsClient.CheckExists(FakeDirUri.AbsolutePath), "Test setup failed: did not successfully create directory to delete.");
+            _fs.Delete(FakeDirUri);
+            Assert.False(_context.MockAdlsClient.CheckExists(FakeDirUri.AbsolutePath), "Test to delete adls directory failed.");
+        }
+
+        [Fact]
+        public void TestDeleteDirectoryException()
+        {
+            // Delete a directory that doesn't exist.
+            Assert.Throws<IOException>(() => _fs.DeleteDirectory(FakeDirUri));
+        }
+
+        [Fact]
+        public void TestGetChildren()
+        {
+            _context.MockAdlsClient.CreateDirectory(FakeDirUri.AbsolutePath);
+            var children = _fs.GetChildren(FakeDirUri);
+            int count = children.Count();
+            Assert.Equal(0, count);
+
+            _context.MockAdlsClient.CreateFile(FakeFileUri.AbsolutePath, IfExists.Overwrite);
+            children = _fs.GetChildren(FakeDirUri);
+            count = children.Count();
+            Assert.Equal(1, count);
+        }
+
+        [Fact]
+        public void TestGetChildrenException()
+        {
+            // Search a directory that doesn't exist.
+            Assert.Throws<IOException>(() => _fs.GetChildren(FakeFileUri).ToList());
+        }
+
+        [Fact]
+        public void TestCreateUriForPath()
+        {
+            string dirStructure = FakeFileUri.AbsolutePath;
+            Uri createdUri = _fs.CreateUriForPath(dirStructure);
+            Assert.Equal(createdUri, new Uri($"adl://{_context.AdlAccountName}{dirStructure}"));
+        }
+
+        [Fact]
+        public void TestGetFileStatusThrowsException()
+        {
+            Assert.Throws<ArgumentNullException>(() => _fs.GetFileStatus(null));
+        }
+
+        private sealed class TestContext
+        {
+            public readonly string AdlAccountName = "adlAccount";
+            public readonly AdlsClient MockAdlsClient = Microsoft.Azure.DataLake.Store.MockAdlsFileSystem.MockAdlsClient.GetMockClient();
+
+            public AzureDataLakeFileSystem GetAdlsFileSystem()
+            {
+                var conf = AzureDataLakeFileSystemConfiguration.ConfigurationModule
+                     .Set(AzureDataLakeFileSystemConfiguration.DataLakeStorageAccountName, "adlsAccountName")
+                    .Set(AzureDataLakeFileSystemConfiguration.Tenant, "tenant")
+                    .Set(AzureDataLakeFileSystemConfiguration.ClientId, "clientId")
+                    .Set(AzureDataLakeFileSystemConfiguration.SecretKey, "secretKey")
+                    .Build();
+                var injector = TangFactory.GetTang().NewInjector(conf);
+                var testDataLakeStoreClient = Substitute.For<IDataLakeStoreClient>();
+                injector.BindVolatileInstance(testDataLakeStoreClient);
+                testDataLakeStoreClient.GetAdlsClient().ReturnsForAnyArgs(MockAdlsClient);
+                testDataLakeStoreClient.AccountFQDN.Returns(AdlAccountName);
+                var fs = injector.GetInstance<AzureDataLakeFileSystem>();
+                return fs;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystemE2E.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystemE2E.cs b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystemE2E.cs
new file mode 100644
index 0000000..a0aaad0
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/TestAzureDataLakeFileSystemE2E.cs
@@ -0,0 +1,251 @@
+// 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 Microsoft.Rest;
+using Microsoft.Rest.Azure.Authentication;
+using Org.Apache.REEF.IO.FileSystem;
+using Org.Apache.REEF.Tang.Implementations.Tang;
+using Xunit;
+using System.Threading;
+using Org.Apache.REEF.IO.FileSystem.AzureDataLake;
+using Microsoft.Azure.DataLake.Store;
+using Org.Apache.REEF.Tang.Interface;
+using System.IO;
+using System.Linq;
+
+namespace Org.Apache.REEF.IO.Tests
+{
+    /// <summary>
+    /// E2E tests for AzureDataLakeFileSystem.
+    /// These tests require the person running the test to fill in credentials.
+    /// </summary>
+    public sealed class TestAzureDataLakeFileSystemE2E : IDisposable
+    {
+        private const string SkipMessage = "Fill in credentials before running test"; // Use null to run tests
+        private const string ContentsText = "hello";
+        private readonly IFileSystem _fileSystem;
+        private readonly AdlsClient _adlsClient;
+        private readonly string _defaultFolderName;
+
+        public TestAzureDataLakeFileSystemE2E()
+        {
+            // Service principal / application authentication with client secret / key
+            // Use the application ID of an existing AAD "Web App" application as the ClientId and
+            // use its authentication key as the SecretKey
+            // https://docs.microsoft.com/en-us/azure/azure-resource-manager/resource-group-create-service-principal-portal#get-application-id-and-authentication-key
+            // Fill in before running test!
+            const string AdlsAccountName = "#####.azuredatalakestore.net";
+            const string Tenant = "microsoft.onmicrosoft.com";
+            const string TokenAudience = @"https://datalake.azure.net/";
+            const string ClientId = "########-####-####-####-############"; // e.g. "c2897d56-5eef-4030-8b7a-46b5c0acd05c"
+            const string SecretKey = "##########"; // e.g. "SecretKey1234!"
+
+            _defaultFolderName = "reef-test-folder-" + Guid.NewGuid();
+
+            IConfiguration conf = AzureDataLakeFileSystemConfiguration.ConfigurationModule
+                .Set(AzureDataLakeFileSystemConfiguration.DataLakeStorageAccountName, AdlsAccountName)
+                .Set(AzureDataLakeFileSystemConfiguration.Tenant, Tenant)
+                .Set(AzureDataLakeFileSystemConfiguration.ClientId, ClientId)
+                .Set(AzureDataLakeFileSystemConfiguration.SecretKey, SecretKey)
+                .Build();
+
+            _fileSystem = TangFactory.GetTang().NewInjector(conf).GetInstance<AzureDataLakeFileSystem>();
+
+            ServiceClientCredentials adlCreds = GetCredsSpiSecretKey(Tenant, new Uri(TokenAudience), ClientId, SecretKey);
+            _adlsClient = AdlsClient.CreateClient(AdlsAccountName, adlCreds);
+        }
+
+        public void Dispose()
+        {
+            _adlsClient?.DeleteRecursive($"/{_defaultFolderName}");
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestOpenE2E()
+        {
+            string fileName = UploadFromString(ContentsText);
+            using (var reader = new StreamReader(_fileSystem.Open(PathToFile(fileName))))
+            {
+                string streamText = reader.ReadToEnd();
+                Assert.Equal(ContentsText, streamText);
+            }
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestCreateE2E()
+        {
+            string fileName = $"/{_defaultFolderName}/TestCreateE2E.txt";
+            var stream = _fileSystem.Create(PathToFile(fileName));
+            Assert.True(_adlsClient.CheckExists(fileName));
+            Assert.IsType<AdlsOutputStream>(stream);
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestDeleteE2E()
+        {
+            string fileName = UploadFromString(ContentsText);
+            Assert.True(_adlsClient.CheckExists(fileName));
+            _fileSystem.Delete(PathToFile(fileName));
+            Assert.False(_adlsClient.CheckExists(fileName));
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestDeleteExceptionE2E()
+        {
+            Assert.Throws<IOException>(() => _fileSystem.Delete(PathToFile("fileName")));
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestExistsE2E()
+        {
+            string fileName = UploadFromString(ContentsText);
+            Assert.True(_fileSystem.Exists(PathToFile(fileName)));
+            _adlsClient.Delete(fileName);
+            Assert.False(_fileSystem.Exists(PathToFile(fileName)));
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestCopyE2E()
+        {
+            var sourceTempFilePath = Path.GetTempFileName();
+            var destTempFilePath = Path.GetTempFileName();
+            try
+            {
+                string fileName = UploadFromString("CopyThis", 1);
+                var sourceUri = PathToFile(fileName);
+                string copyToFile = $"/{_defaultFolderName}/testFile2.txt";
+                var destUri = PathToFile(copyToFile);
+                _fileSystem.Copy(sourceUri, destUri);
+                _adlsClient.BulkDownload(sourceUri.AbsolutePath, sourceTempFilePath);
+                _adlsClient.BulkDownload(destUri.AbsolutePath, destTempFilePath);
+                FileSystemTestUtilities.HaveSameContent(sourceTempFilePath, destTempFilePath);
+            }
+            finally
+            {
+                try
+                {
+                    File.Delete(sourceTempFilePath);
+                }
+                finally
+                {
+                    File.Delete(destTempFilePath);
+                }
+            }
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestCopyToLocalE2E()
+        {         
+            var tempFilePath = Path.GetTempFileName();
+            try
+            {
+                string fileName = UploadFromString(ContentsText);
+                _fileSystem.CopyToLocal(PathToFile(fileName), tempFilePath);
+                Assert.True(File.Exists(tempFilePath));
+                Assert.Equal(ContentsText, File.ReadAllText(tempFilePath));
+            }
+            finally
+            {
+                File.Delete(tempFilePath);
+            }
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestCopyFromLocalE2E()
+        {
+            var tempFilePath = Path.GetTempFileName();
+            var tempFileName = Path.GetFileName(tempFilePath);
+            try
+            {
+                File.WriteAllText(tempFilePath, ContentsText);
+                Uri remoteFileUri = PathToFile($"/{_defaultFolderName}/{tempFileName}");
+                _fileSystem.CopyFromLocal(tempFilePath, remoteFileUri);
+                Assert.True(_adlsClient.CheckExists($"/{_defaultFolderName}/{tempFileName}"));
+                var stream = _fileSystem.Open(remoteFileUri);
+                string streamText = new StreamReader(stream).ReadToEnd();
+                Assert.Equal(ContentsText, streamText);
+            }
+            finally
+            {
+                File.Delete(tempFilePath);
+            }
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestCreateDirectoryE2E()
+        {
+            string dirName = $"/{_defaultFolderName}";
+            _fileSystem.CreateDirectory(PathToFile(dirName));
+            Assert.True(_adlsClient.CheckExists(dirName));
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestDeleteDirectoryE2E()
+        {
+            string dirName = $"/{_defaultFolderName}";
+            _adlsClient.CreateDirectory(dirName);
+            Assert.True(_adlsClient.CheckExists(dirName));
+            _fileSystem.Delete(PathToFile(dirName));
+            Assert.False(_adlsClient.CheckExists(dirName));
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestGetChildrenE2E()
+        {
+            string fileName1 = UploadFromString("file1", 1);
+            string fileName2 = UploadFromString("file2", 2);
+            string dirName = $"/{_defaultFolderName}";
+            var childUris = _fileSystem.GetChildren(PathToFile(dirName)).ToList();
+            Assert.Equal(2, childUris.Count);
+            Assert.Equal(new[] { PathToFile(fileName1), PathToFile(fileName2) }, childUris);
+        }
+
+        [Fact(Skip = SkipMessage)]
+        public void TestGetFileStatusE2E()
+        {
+            string fileName = UploadFromString(ContentsText);
+            var fileStatus = _fileSystem.GetFileStatus(PathToFile(fileName));
+            Assert.Equal(ContentsText.Length, fileStatus.LengthBytes);
+        }
+
+        private static ServiceClientCredentials GetCredsSpiSecretKey(string tenant, Uri tokenAudience, string clientId, string secretKey)
+        {
+            SynchronizationContext.SetSynchronizationContext(new SynchronizationContext());
+
+            var serviceSettings = ActiveDirectoryServiceSettings.Azure;
+            serviceSettings.TokenAudience = tokenAudience;
+
+            return ApplicationTokenProvider.LoginSilentAsync(tenant, clientId, secretKey, serviceSettings).Result;
+        }
+
+        private string UploadFromString(string str, int fileIndex = 1)
+        {
+            string fileName = $"/{_defaultFolderName}/testFile{fileIndex}.txt";
+            using (var streamWriter = new StreamWriter(_adlsClient.CreateFile(fileName, IfExists.Overwrite)))
+            {
+                streamWriter.Write(str);
+            }
+            return fileName;
+        }
+
+        private Uri PathToFile(string filePath)
+        {
+            return _fileSystem.CreateUriForPath(filePath);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO.Tests/packages.config b/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
index 6cae99e..cc3ad3b 100644
--- a/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
+++ b/lang/cs/Org.Apache.REEF.IO.Tests/packages.config
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -18,6 +18,12 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
+  <package id="Microsoft.Azure.DataLake.Store" version="1.0.4" targetFramework="net452" />
+  <package id="Microsoft.IdentityModel.Clients.ActiveDirectory" version="2.28.3" targetFramework="net452" />
+  <package id="Microsoft.Rest.ClientRuntime" version="2.3.10" targetFramework="net452" />
+  <package id="Microsoft.Rest.ClientRuntime.Azure.Authentication" version="2.3.1" targetFramework="net452" />
+  <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net452" />
+  <package id="NLog" version="4.4.12" targetFramework="net452" />
   <package id="NSubstitute" version="1.8.2.0" targetFramework="net45" />
   <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="xunit" version="2.1.0" targetFramework="net45" />

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystem.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystem.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystem.cs
new file mode 100644
index 0000000..39d1f04
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystem.cs
@@ -0,0 +1,232 @@
+// 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 System.Linq;
+using Microsoft.Azure.DataLake.Store;
+using Microsoft.Azure.DataLake.Store.FileTransfer;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
+{
+    /// <summary>
+    /// An IFileSystem implementation for Azure Data Lake Store.
+    /// </summary>
+    internal sealed class AzureDataLakeFileSystem : IFileSystem
+    {
+        private readonly IDataLakeStoreClient _client;
+        private readonly AdlsClient _adlsClient;
+
+        [Inject]
+        private AzureDataLakeFileSystem(IDataLakeStoreClient client)
+        {
+            _client = client;
+            _adlsClient = _client.GetAdlsClient();
+        }
+
+        /// <summary>
+        /// Opens the given URI for reading
+        /// </summary>
+        /// <exception cref="AdlsException">If the URI couldn't be opened.</exception>
+        public Stream Open(Uri fileUri)
+        {
+            return _adlsClient.GetReadStream(fileUri.AbsolutePath);
+        }
+
+        /// <summary>
+        /// Creates a new file under the given URI.
+        /// </summary>
+        /// <exception cref="AdlsException">If the URI couldn't be created.</exception>
+        public Stream Create(Uri fileUri)
+        {
+            return _adlsClient.CreateFile(fileUri.AbsolutePath, IfExists.Overwrite);
+        }
+
+        /// <summary>
+        /// Deletes the file under the given URI.
+        /// </summary>
+        /// <exception cref="IOException">If the specified file cannot be deleted</exception>
+        public void Delete(Uri fileUri)
+        {
+            bool deleteStatus = _adlsClient.Delete(fileUri.AbsolutePath);
+            if (!deleteStatus)
+            {
+                throw new IOException($"Cannot delete directory/file specified by {fileUri}");
+            }
+        }
+
+        /// <summary>
+        /// Determines whether a file exists under the given URI.
+        /// </summary>
+        public bool Exists(Uri fileUri)
+        {
+            return _adlsClient.CheckExists(fileUri.AbsolutePath);
+        }
+
+        /// <summary>
+        /// Copies the file referenced by sourceUri to destinationUri.
+        /// Note : This method reads from the input stream of sourceUri locally and
+        /// writes to the output stream of destinationUri.
+        /// This is time consuming and not recommended for large file transfers.
+        /// </summary>
+        /// <exception cref="IOException">If copy process encounters any exceptions</exception>
+        public void Copy(Uri sourceUri, Uri destinationUri)
+        {
+            try
+            {
+                using (var readStream = Open(sourceUri))
+                {
+                    readStream.Position = 0;
+                    using (var writeStream = Create(destinationUri))
+                    {
+                        readStream.CopyTo(writeStream);
+                    }
+                }
+            }
+            catch (Exception ex)
+            {
+                throw new IOException($"Error copying {sourceUri} to {destinationUri}", ex);
+            }
+        }
+
+        /// <summary>
+        /// Copies the remote file to a local file.
+        /// </summary>
+        /// <exception cref="IOException">If copy process encounters any exceptions</exception>
+        public void CopyToLocal(Uri remoteFileUri, string localFileName)
+        {
+            TransferStatus status;
+            try
+            {
+                status = _adlsClient.BulkDownload(remoteFileUri.AbsolutePath, localFileName); // throws KeyNotFoundException
+            }
+            catch (Exception ex)
+            {
+                throw new IOException($"Error in bulk download from {remoteFileUri} to {localFileName}", ex);
+            }
+            if (status.EntriesFailed.Count != 0)
+            {
+                throw new IOException($"{status.EntriesFailed.Count} entries did not get transferred correctly");
+            }
+        }
+
+        /// <summary>
+        /// Copies the specified file to the remote location.
+        /// </summary>
+        /// <exception cref="IOException">If copy process encounters any exception</exception>
+        public void CopyFromLocal(string localFileName, Uri remoteFileUri)
+        {
+            TransferStatus status;
+            try
+            {
+                status = status = _adlsClient.BulkUpload(localFileName, remoteFileUri.AbsolutePath);
+            }
+            catch (Exception ex)
+            {
+                throw new IOException($"Error in bulk upload from {localFileName} to {remoteFileUri}", ex);
+            }
+            if (status.EntriesFailed.Count != 0)
+            {
+                throw new IOException($"{status.EntriesFailed.Count} entries did not get transferred correctly");
+            }
+        }
+
+        /// <summary>
+        /// Creates a new directory.
+        /// </summary>
+        /// <exception cref="IOException">If directory cannot be created</exception>
+        public void CreateDirectory(Uri directoryUri)
+        {
+            bool createDirStatus = _adlsClient.CreateDirectory(directoryUri.AbsolutePath);
+            if (!createDirStatus)
+            {
+                throw new IOException($"Cannot create directory specified by {directoryUri}");
+            }
+        }
+
+        /// <summary>
+        /// Deletes a directory.
+        /// </summary>
+        /// <exception cref="IOException">If directory cannot be deleted</exception>
+        public void DeleteDirectory(Uri directoryUri)
+        {
+            bool deleteStatus = Exists(directoryUri) && 
+                _adlsClient.GetDirectoryEntry(directoryUri.AbsolutePath).Type == DirectoryEntryType.DIRECTORY &&
+                _adlsClient.DeleteRecursive(directoryUri.AbsolutePath);
+            if (!deleteStatus)
+            {
+                throw new IOException($"Cannot delete directory specified by {directoryUri}");
+            }
+        }
+
+        /// <summary>
+        /// Get the children on the given URI, if that refers to a directory.
+        /// </summary>
+        /// <exception cref="IOException">If directory does not exist</exception>
+        public IEnumerable<Uri> GetChildren(Uri directoryUri)
+        {
+            if (!Exists(directoryUri) || _adlsClient.GetDirectoryEntry(directoryUri.AbsolutePath).Type != DirectoryEntryType.DIRECTORY)
+            {
+                throw new IOException($"Cannot find directory specified by {directoryUri}");
+            }
+
+            return _adlsClient.EnumerateDirectory(directoryUri.AbsolutePath).Select(entry => CreateUriForPath(entry.FullName));
+        }
+
+        /// <summary>
+        /// Create Uri from a given file path.
+        /// The file path can be full with prefix or relative without prefix.
+        /// If null is passed as the path, ArgumentException will be thrown.
+        /// </summary>
+        /// <exception cref="ArgumentNullException">If specified path is null</exception>
+        public Uri CreateUriForPath(string path)
+        {
+            if (path == null)
+            {
+                throw new ArgumentNullException(nameof(path), "Specified path is null");
+            }
+            return new Uri($"{GetUriPrefix()}/{path.TrimStart('/')}");
+        }
+
+        /// <summary>
+        /// Gets the FileStatus for remote file.
+        /// </summary>
+        /// <exception cref="ArgumentNullException">If remote file URI is null</exception>
+        /// <returns>FileStatus</returns>
+        public FileStatus GetFileStatus(Uri remoteFileUri)
+        {
+            if (remoteFileUri == null)
+            {
+                throw new ArgumentNullException(nameof(remoteFileUri), "Specified uri is null");
+            }
+            var entrySummary = _adlsClient.GetDirectoryEntry(remoteFileUri.AbsolutePath);
+            if (!entrySummary.LastModifiedTime.HasValue)
+            {
+                throw new IOException($"File/Directory at {remoteFileUri} does not have a last modified time. It may have been deleted.");
+            }
+
+            return new FileStatus(entrySummary.LastModifiedTime.Value, entrySummary.Length);
+        }
+
+        private string GetUriPrefix()
+        {
+            return $"adl://{_client.AccountFQDN}";
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystemConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystemConfiguration.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystemConfiguration.cs
new file mode 100644
index 0000000..6c386f6
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystemConfiguration.cs
@@ -0,0 +1,71 @@
+// 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.Common.Client.Parameters;
+using Org.Apache.REEF.Tang.Formats;
+using Org.Apache.REEF.Tang.Interface;
+using Org.Apache.REEF.Tang.Util;
+using Org.Apache.REEF.IO.FileSystem.AzureDataLake.Parameters;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
+{
+    /// <summary>
+    /// Configuration Module for the Azure Data Lake (ADL) file system implementation of IFileSystem.
+    /// </summary>
+    public sealed class AzureDataLakeFileSystemConfiguration : ConfigurationModuleBuilder
+    {
+        /// <summary>
+        /// The account FQDN to be used to connect to the data lake store
+        /// </summary>
+        public static readonly RequiredParameter<string> DataLakeStorageAccountName = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Tenant to be used to authenticate with Azure
+        /// </summary>
+        public static readonly RequiredParameter<string> Tenant = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Application ID to be used to authenticate with Azure
+        /// </summary>
+        public static readonly RequiredParameter<string> ClientId = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The Client Secret to be used to authenticate with Azure
+        /// </summary>
+        public static readonly RequiredParameter<string> SecretKey = new RequiredParameter<string>();
+
+        /// <summary>
+        /// The ADL TokenAudience Uri to be used to authenticate with Data Lake Store
+        /// </summary>
+        public static readonly OptionalParameter<string> TokenAudience = new OptionalParameter<string>();
+
+        /// <summary>
+        /// Set AzureDataLakeFileSystemConfigurationProvider to DriverConfigurationProviders.
+        /// Set all the parameters needed for injecting AzureDataLakeFileSystemConfigurationProvider.
+        /// </summary>
+        public static readonly ConfigurationModule ConfigurationModule = new AzureDataLakeFileSystemConfiguration()
+            .BindSetEntry<DriverConfigurationProviders, AzureDataLakeFileSystemConfigurationProvider, IConfigurationProvider>(
+                GenericType<DriverConfigurationProviders>.Class, GenericType<AzureDataLakeFileSystemConfigurationProvider>.Class)
+            .BindImplementation(GenericType<IFileSystem>.Class, GenericType<AzureDataLakeFileSystem>.Class)
+            .BindNamedParameter(GenericType<DataLakeStorageAccountName>.Class, DataLakeStorageAccountName)
+            .BindNamedParameter(GenericType<Tenant>.Class, Tenant)
+            .BindNamedParameter(GenericType<ClientId>.Class, ClientId)
+            .BindNamedParameter(GenericType<SecretKey>.Class, SecretKey)
+            .BindNamedParameter(GenericType<TokenAudience>.Class, TokenAudience)
+            .Build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystemConfigurationProvider.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystemConfigurationProvider.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystemConfigurationProvider.cs
new file mode 100644
index 0000000..8e1564c
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeFileSystemConfigurationProvider.cs
@@ -0,0 +1,61 @@
+// 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.Common.Evaluator.Parameters;
+using Org.Apache.REEF.IO.FileSystem.AzureDataLake.Parameters;
+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;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
+{
+    /// <summary>
+    /// This provider provides configuration for AzureDataLakeFileSystem
+    /// The client that is going to use AzureDataLakeFileSystem in its driver and evaluators should set 
+    /// configuration data through AzureDataLakeFileSystemConfiguration module in the client's configuration
+    /// </summary>
+    internal sealed class AzureDataLakeFileSystemConfigurationProvider : IConfigurationProvider
+    {
+        private readonly IConfiguration _configuration;
+
+        [Inject]
+        private AzureDataLakeFileSystemConfigurationProvider(
+            [Parameter(typeof(DataLakeStorageAccountName))] string adlsAccountName,
+            [Parameter(typeof(Tenant))] string tenant,
+            [Parameter(typeof(ClientId))] string clientId,
+            [Parameter(typeof(SecretKey))] string secretKey,
+            [Parameter(typeof(TokenAudience))] string tokenAudience)
+        {
+            _configuration = TangFactory.GetTang().NewConfigurationBuilder()
+                .BindImplementation(GenericType<IFileSystem>.Class, GenericType<AzureDataLakeFileSystem>.Class)
+                .BindImplementation(GenericType<IAdlsCredentials>.Class, GenericType<SecretKeyAdlsCredentials>.Class)
+                .BindStringNamedParam<DataLakeStorageAccountName>(adlsAccountName)
+                .BindStringNamedParam<Tenant>(tenant)
+                .BindStringNamedParam<ClientId>(clientId)
+                .BindStringNamedParam<SecretKey>(secretKey)
+                .BindStringNamedParam<TokenAudience>(tokenAudience)
+                .BindSetEntry<EvaluatorConfigurationProviders, AzureDataLakeFileSystemConfigurationProvider, IConfigurationProvider>()
+                .Build();
+        }
+
+        public IConfiguration GetConfiguration()
+        {
+            return _configuration;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeStoreClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeStoreClient.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeStoreClient.cs
new file mode 100644
index 0000000..6367797
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/AzureDataLakeStoreClient.cs
@@ -0,0 +1,44 @@
+// 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 Microsoft.Azure.DataLake.Store;
+using Org.Apache.REEF.IO.FileSystem.AzureDataLake.Parameters;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
+{
+    /// <summary>
+    /// A proxy class for AdlsClient, mainly in order to fake for unit testing.
+    /// </summary>
+    internal sealed class AzureDataLakeStoreClient : IDataLakeStoreClient
+    {
+        private readonly AdlsClient _adlsClient;
+
+        [Inject]
+        private AzureDataLakeStoreClient([Parameter(typeof(DataLakeStorageAccountName))] string adlsAccountName, IAdlsCredentials adlsCredentials)
+        {
+            _adlsClient = AdlsClient.CreateClient(adlsAccountName, adlsCredentials.Credentials);
+        }
+
+        public AdlsClient GetAdlsClient()
+        {
+            return _adlsClient;
+        }
+
+        public string AccountFQDN => _adlsClient.AccountFQDN;
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/IAdlsCredentials.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/IAdlsCredentials.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/IAdlsCredentials.cs
new file mode 100644
index 0000000..6f217d9
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/IAdlsCredentials.cs
@@ -0,0 +1,28 @@
+// 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 Microsoft.Rest;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
+{
+    [DefaultImplementation(typeof(SecretKeyAdlsCredentials))]
+    internal interface IAdlsCredentials
+    {
+        ServiceClientCredentials Credentials { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/IDataLakeStoreClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/IDataLakeStoreClient.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/IDataLakeStoreClient.cs
new file mode 100644
index 0000000..da694c3
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/IDataLakeStoreClient.cs
@@ -0,0 +1,36 @@
+// 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 Microsoft.Azure.DataLake.Store;
+using Org.Apache.REEF.Tang.Annotations;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
+{
+    /// <summary>
+    /// A proxy interface for AzureDataLakeStoreClient, mainly in order to fake for unit testing.
+    /// </summary>
+    [DefaultImplementation(typeof(AzureDataLakeStoreClient))]
+    internal interface IDataLakeStoreClient
+    {
+        AdlsClient GetAdlsClient();
+
+        /// <summary>
+        /// Returns the account name for the AdlsClient.
+        /// </summary>
+        string AccountFQDN { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/ClientId.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/ClientId.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/ClientId.cs
new file mode 100644
index 0000000..c9c8d55
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/ClientId.cs
@@ -0,0 +1,29 @@
+// 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.IO.FileSystem.AzureDataLake.Parameters
+{
+    /// <summary>
+    /// The Application ID to be used to authenticate with Azure
+    /// </summary>
+    [NamedParameter("The Application ID to be used to authenticate with Azure")]
+    internal sealed class ClientId : Name<string>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/DataLakeStorageAccountName.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/DataLakeStorageAccountName.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/DataLakeStorageAccountName.cs
new file mode 100644
index 0000000..5d3f2ed
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/DataLakeStorageAccountName.cs
@@ -0,0 +1,29 @@
+// 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.IO.FileSystem.AzureDataLake.Parameters
+{
+    /// <summary>
+    /// The account FQDN to be used to connect to the data lake store
+    /// </summary>
+    [NamedParameter("The account FQDN to be used to connect to the data lake store")]
+    internal sealed class DataLakeStorageAccountName : Name<string>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/SecretKey.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/SecretKey.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/SecretKey.cs
new file mode 100644
index 0000000..21a23b4
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/SecretKey.cs
@@ -0,0 +1,29 @@
+// 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.IO.FileSystem.AzureDataLake.Parameters
+{
+    /// <summary>
+    /// The Client Secret to be used to authenticate with Azure
+    /// </summary>
+    [NamedParameter("The Client Secret to be used to authenticate with Azure")]
+    internal sealed class SecretKey : Name<string>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/Tenant.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/Tenant.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/Tenant.cs
new file mode 100644
index 0000000..a5c5cca
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/Tenant.cs
@@ -0,0 +1,29 @@
+// 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.IO.FileSystem.AzureDataLake.Parameters
+{
+    /// <summary>
+    /// The Tenant to be used to authenticate with Azure
+    /// </summary>
+    [NamedParameter("The Tenant to be used to authenticate with Azure")]
+    internal sealed class Tenant : Name<string>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/TokenAudience.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/TokenAudience.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/TokenAudience.cs
new file mode 100644
index 0000000..d15c6d0
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/Parameters/TokenAudience.cs
@@ -0,0 +1,29 @@
+// 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.IO.FileSystem.AzureDataLake.Parameters
+{
+    /// <summary>
+    /// The ADL TokenAudience Uri to be used to authenticate with Data Lake Store
+    /// </summary>
+    [NamedParameter("The ADL TokenAudience Uri to be used to authenticate with Data Lake Store", defaultValue: "https://datalake.azure.net/")]
+    internal sealed class TokenAudience : Name<string>
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/SecretKeyAdlsCredentials.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/SecretKeyAdlsCredentials.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/SecretKeyAdlsCredentials.cs
new file mode 100644
index 0000000..1f75319
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/AzureDataLake/SecretKeyAdlsCredentials.cs
@@ -0,0 +1,45 @@
+// 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.Threading;
+using Microsoft.Rest;
+using Org.Apache.REEF.Tang.Annotations;
+using Microsoft.Rest.Azure.Authentication;
+using Org.Apache.REEF.IO.FileSystem.AzureDataLake.Parameters;
+
+namespace Org.Apache.REEF.IO.FileSystem.AzureDataLake
+{
+    internal sealed class SecretKeyAdlsCredentials : IAdlsCredentials
+    {
+        public ServiceClientCredentials Credentials { get; }
+
+        [Inject]
+        private SecretKeyAdlsCredentials([Parameter(typeof(Tenant))] string tenant,
+            [Parameter(typeof(TokenAudience))] string tokenAudience,
+            [Parameter(typeof(ClientId))] string clientId,
+            [Parameter(typeof(SecretKey))] string secretKey)
+        {
+            SynchronizationContext.SetSynchronizationContext(new SynchronizationContext());
+
+            var serviceSettings = ActiveDirectoryServiceSettings.Azure;
+            serviceSettings.TokenAudience = new Uri(tokenAudience);
+
+            Credentials = ApplicationTokenProvider.LoginSilentAsync(tenant, clientId, secretKey, serviceSettings).Result;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/Parameters/CommandTimeOut.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/Parameters/CommandTimeOut.cs b/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/Parameters/CommandTimeOut.cs
index 315d5eb..75bb476 100644
--- a/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/Parameters/CommandTimeOut.cs
+++ b/lang/cs/Org.Apache.REEF.IO/FileSystem/Hadoop/Parameters/CommandTimeOut.cs
@@ -22,7 +22,7 @@ namespace Org.Apache.REEF.IO.FileSystem.Hadoop.Parameters
     /// <summary>
     /// The timeout (in milliseconds) for HDFS commands. Defaults to 300000 (5 minutes).
     /// </summary>
-    [NamedParameter("he timeout (in milliseconds) for HDFS commands.", defaultValue: "300000")]
+    [NamedParameter("The timeout (in milliseconds) for HDFS commands.", defaultValue: "300000")]
     internal sealed class CommandTimeOut : Name<int>
     {
     }

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj b/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
index 97a6565..055c139 100644
--- a/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
+++ b/lang/cs/Org.Apache.REEF.IO/Org.Apache.REEF.IO.csproj
@@ -1,4 +1,4 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?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
@@ -33,10 +33,17 @@ under the License.
   <Import Project="$(SolutionDir)\.nuget\NuGet.targets" Condition="Exists('$(SolutionDir)\.nuget\NuGet.targets')" />
   <Import Project="$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets" Condition="Exists('$(PackagesDir)\StyleCop.MSBuild.$(StyleCopVersion)\build\StyleCop.MSBuild.Targets')" />
   <ItemGroup>
+    <Reference Include="Microsoft.Azure.DataLake.Store, Version=1.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Azure.DataLake.Store.1.0.4\lib\net452\Microsoft.Azure.DataLake.Store.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="Microsoft.Azure.KeyVault.Core, Version=1.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
       <HintPath>$(PackagesDir)\Microsoft.Azure.KeyVault.Core.1.0.0\lib\net40\Microsoft.Azure.KeyVault.Core.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="Microsoft.Azure.Management.DataLake.Store, Version=2.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Azure.Management.DataLake.Store.2.2.1\lib\net452\Microsoft.Azure.Management.DataLake.Store.dll</HintPath>
+    </Reference>
     <Reference Include="Microsoft.Data.Edm, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
       <HintPath>$(PackagesDir)\Microsoft.Data.Edm.5.6.4\lib\net40\Microsoft.Data.Edm.dll</HintPath>
       <Private>True</Private>
@@ -49,16 +56,38 @@ under the License.
       <HintPath>$(PackagesDir)\Microsoft.Data.Services.Client.5.6.4\lib\net40\Microsoft.Data.Services.Client.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="Microsoft.Rest.ClientRuntime, Version=2.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Rest.ClientRuntime.2.3.10\lib\net452\Microsoft.Rest.ClientRuntime.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="Microsoft.Rest.ClientRuntime.Azure, Version=3.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Rest.ClientRuntime.Azure.3.3.7\lib\net452\Microsoft.Rest.ClientRuntime.Azure.dll</HintPath>
+    </Reference>
     <Reference Include="Microsoft.WindowsAzure.Storage, Version=6.1.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
       <HintPath>$(PackagesDir)\WindowsAzure.Storage.6.1.0\lib\net40\Microsoft.WindowsAzure.Storage.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="Microsoft.IdentityModel.Clients.ActiveDirectory, Version=2.28.3.860, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.IdentityModel.Clients.ActiveDirectory.2.28.3\lib\net45\Microsoft.IdentityModel.Clients.ActiveDirectory.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.IdentityModel.Clients.ActiveDirectory.WindowsForms, Version=2.28.3.860, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.IdentityModel.Clients.ActiveDirectory.2.28.3\lib\net45\Microsoft.IdentityModel.Clients.ActiveDirectory.WindowsForms.dll</HintPath>
+    </Reference>
+    <Reference Include="Microsoft.Rest.ClientRuntime.Azure.Authentication, Version=2.0.0.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\Microsoft.Rest.ClientRuntime.Azure.Authentication.2.3.1\lib\net452\Microsoft.Rest.ClientRuntime.Azure.Authentication.dll</HintPath>
+    </Reference>
     <Reference Include="Newtonsoft.Json">
       <HintPath>$(PackagesDir)\Newtonsoft.Json.$(NewtonsoftJsonVersion)\lib\net45\Newtonsoft.Json.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="NLog, Version=4.0.0.0, Culture=neutral, PublicKeyToken=5120e14c03d0593c, processorArchitecture=MSIL">
+      <HintPath>$(PackagesDir)\NLog.4.4.12\lib\net45\NLog.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="System" />
     <Reference Include="System.Core" />
+    <Reference Include="System.Management" />
+    <Reference Include="System.Net.Http" />
     <Reference Include="System.Spatial, Version=5.6.4.0, Culture=neutral, PublicKeyToken=31bf3856ad364e35, processorArchitecture=MSIL">
       <HintPath>$(PackagesDir)\System.Spatial.5.6.4\lib\net40\System.Spatial.dll</HintPath>
       <Private>True</Private>
@@ -86,6 +115,18 @@ under the License.
     <Compile Include="FileSystem\AzureBlob\ICloudBlobClient.cs" />
     <Compile Include="FileSystem\AzureBlob\AzureCloudBlobContainer.cs" />
     <Compile Include="FileSystem\AzureBlob\Parameters\AzureStorageConnectionString.cs" />
+    <Compile Include="FileSystem\AzureDataLake\IDataLakeStoreClient.cs" />
+    <Compile Include="FileSystem\AzureDataLake\AzureDataLakeStoreClient.cs" />
+    <Compile Include="FileSystem\AzureDataLake\AzureDataLakeFileSystem.cs" />
+    <Compile Include="FileSystem\AzureDataLake\AzureDataLakeFileSystemConfiguration.cs" />
+    <Compile Include="FileSystem\AzureDataLake\AzureDataLakeFileSystemConfigurationProvider.cs" />
+    <Compile Include="FileSystem\AzureDataLake\IAdlsCredentials.cs" />
+    <Compile Include="FileSystem\AzureDataLake\SecretKeyAdlsCredentials.cs" />
+    <Compile Include="FileSystem\AzureDataLake\Parameters\DataLakeStorageAccountName.cs" />
+    <Compile Include="FileSystem\AzureDataLake\Parameters\Tenant.cs" />
+    <Compile Include="FileSystem\AzureDataLake\Parameters\TokenAudience.cs" />
+    <Compile Include="FileSystem\AzureDataLake\Parameters\ClientId.cs" />
+    <Compile Include="FileSystem\AzureDataLake\Parameters\SecretKey.cs" />
     <Compile Include="FileSystem\FileStatus.cs" />
     <Compile Include="FileSystem\Hadoop\CommandResult.cs" />
     <Compile Include="FileSystem\Hadoop\HDFSConfigurationWithoutDriverBinding.cs" />

http://git-wip-us.apache.org/repos/asf/reef/blob/b759764c/lang/cs/Org.Apache.REEF.IO/packages.config
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IO/packages.config b/lang/cs/Org.Apache.REEF.IO/packages.config
index 4e1a691..b401637 100644
--- a/lang/cs/Org.Apache.REEF.IO/packages.config
+++ b/lang/cs/Org.Apache.REEF.IO/packages.config
@@ -18,11 +18,18 @@ specific language governing permissions and limitations
 under the License.
 -->
 <packages>
+  <package id="Microsoft.Azure.DataLake.Store" version="1.0.4" targetFramework="net452" />
   <package id="Microsoft.Azure.KeyVault.Core" version="1.0.0" targetFramework="net45" />
+  <package id="Microsoft.Azure.Management.DataLake.Store" version="2.2.1" targetFramework="net452" />
   <package id="Microsoft.Data.Edm" version="5.6.4" targetFramework="net45" />
   <package id="Microsoft.Data.OData" version="5.6.4" targetFramework="net45" />
   <package id="Microsoft.Data.Services.Client" version="5.6.4" targetFramework="net45" />
+  <package id="Microsoft.IdentityModel.Clients.ActiveDirectory" version="2.28.3" targetFramework="net452" />
+  <package id="Microsoft.Rest.ClientRuntime" version="2.3.10" targetFramework="net452" />
+  <package id="Microsoft.Rest.ClientRuntime.Azure" version="3.3.7" targetFramework="net452" />
+  <package id="Microsoft.Rest.ClientRuntime.Azure.Authentication" version="2.3.2" targetFramework="net452" />
   <package id="Newtonsoft.Json" version="10.0.3" targetFramework="net451" />
+  <package id="NLog" version="4.4.12" targetFramework="net452" />
   <package id="StyleCop.MSBuild" version="5.0.0" targetFramework="net45" developmentDependency="true" />
   <package id="System.Spatial" version="5.6.4" targetFramework="net45" />
   <package id="WindowsAzure.Storage" version="6.1.0" targetFramework="net45" />