You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by ju...@apache.org on 2015/10/13 09:27:31 UTC

incubator-reef git commit: [REEF-840] IIMRUClient does not need to have generic type

Repository: incubator-reef
Updated Branches:
  refs/heads/master a6e2218b5 -> 06db1ab1f


[REEF-840] IIMRUClient does not need to have generic type

This addressed the issue by
* removing generics from the IIMRU Client class definition and shifting it to Submit function

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

This closes #562


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

Branch: refs/heads/master
Commit: 06db1ab1fef6dd87e2eaa99b4c1b228c896a07ec
Parents: a6e2218
Author: Dhruv <dh...@apache.org>
Authored: Mon Oct 12 22:27:09 2015 -0700
Committer: Julia Wang <ju...@microsoft.com>
Committed: Tue Oct 13 00:12:43 2015 -0700

----------------------------------------------------------------------
 .../MapperCount/MapperCount.cs                   |  6 +++---
 .../OnREEFIMRURunTimeConfiguration.cs            |  4 ++--
 .../PipelinedBroadcastAndReduce.cs               |  6 +++---
 .../MapperCountTest.cs                           |  4 ++--
 lang/cs/Org.Apache.REEF.IMRU/API/IIMRUClient.cs  | 18 ++++++++----------
 .../InProcess/InProcessIMRUClient.cs             | 19 ++++++++++---------
 .../InProcess/InProcessIMRUConfiguration.cs      | 11 ++++-------
 .../OnREEF/Client/REEFIMRUClient.cs              | 14 +++++++-------
 .../OnREEF/Client/REEFIMRUClientConfiguration.cs |  8 ++++----
 9 files changed, 43 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/lang/cs/Org.Apache.REEF.IMRU.Examples/MapperCount/MapperCount.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU.Examples/MapperCount/MapperCount.cs b/lang/cs/Org.Apache.REEF.IMRU.Examples/MapperCount/MapperCount.cs
index 76b2e37..9fed04b 100644
--- a/lang/cs/Org.Apache.REEF.IMRU.Examples/MapperCount/MapperCount.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU.Examples/MapperCount/MapperCount.cs
@@ -31,10 +31,10 @@ namespace Org.Apache.REEF.IMRU.Examples.MapperCount
     /// </summary>
     public sealed class MapperCount
     {
-        private readonly IIMRUClient<int, int, int> _imruClient;
+        private readonly IIMRUClient _imruClient;
 
         [Inject]
-        private MapperCount(IIMRUClient<int, int, int> imruClient)
+        private MapperCount(IIMRUClient imruClient)
         {
             _imruClient = imruClient;
         }
@@ -45,7 +45,7 @@ namespace Org.Apache.REEF.IMRU.Examples.MapperCount
         /// <returns>The number of MapFunction instances that are part of the job.</returns>
         public int Run(int numberofMappers)
         {
-            var results = _imruClient.Submit(
+            var results = _imruClient.Submit<int, int, int>(
                 new IMRUJobDefinitionBuilder()
                     .SetMapFunctionConfiguration(IMRUMapConfiguration<int, int>.ConfigurationModule
                         .Set(IMRUMapConfiguration<int, int>.MapFunction, GenericType<IdentityMapFunction>.Class)

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/lang/cs/Org.Apache.REEF.IMRU.Examples/OnREEFIMRURunTimeConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU.Examples/OnREEFIMRURunTimeConfiguration.cs b/lang/cs/Org.Apache.REEF.IMRU.Examples/OnREEFIMRURunTimeConfiguration.cs
index eaf99b2..c13efa0 100644
--- a/lang/cs/Org.Apache.REEF.IMRU.Examples/OnREEFIMRURunTimeConfiguration.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU.Examples/OnREEFIMRURunTimeConfiguration.cs
@@ -43,7 +43,7 @@ namespace Org.Apache.REEF.IMRU.Examples
         {
             IConfiguration runtimeConfig;
             IConfiguration imruClientConfig =
-                REEFIMRUClientConfiguration<TMapInput, TMapOutput, TResult>.ConfigurationModule.Build();
+                REEFIMRUClientConfiguration.ConfigurationModule.Build();
 
             if (runTimeDir.Length != 0)
             {
@@ -71,7 +71,7 @@ namespace Org.Apache.REEF.IMRU.Examples
         internal static IConfiguration GetYarnIMRUConfiguration()
         {
             IConfiguration imruClientConfig =
-                REEFIMRUClientConfiguration<TMapInput, TMapOutput, TResult>.ConfigurationModule.Build();
+                REEFIMRUClientConfiguration.ConfigurationModule.Build();
 
             var fileSystemConfig = HadoopFileSystemConfiguration.ConfigurationModule.Build();
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/lang/cs/Org.Apache.REEF.IMRU.Examples/PipelinedBroadcastReduce/PipelinedBroadcastAndReduce.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU.Examples/PipelinedBroadcastReduce/PipelinedBroadcastAndReduce.cs b/lang/cs/Org.Apache.REEF.IMRU.Examples/PipelinedBroadcastReduce/PipelinedBroadcastAndReduce.cs
index 8f1cbc7..b642e37 100644
--- a/lang/cs/Org.Apache.REEF.IMRU.Examples/PipelinedBroadcastReduce/PipelinedBroadcastAndReduce.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU.Examples/PipelinedBroadcastReduce/PipelinedBroadcastAndReduce.cs
@@ -32,10 +32,10 @@ namespace Org.Apache.REEF.IMRU.Examples.PipelinedBroadcastReduce
     /// </summary>
     public sealed class PipelinedBroadcastAndReduce
     {
-        private readonly IIMRUClient<int[], int[], int[]> _imruClient;
+        private readonly IIMRUClient _imruClient;
 
         [Inject]
-        private PipelinedBroadcastAndReduce(IIMRUClient<int[], int[], int[]> imruClient)
+        private PipelinedBroadcastAndReduce(IIMRUClient imruClient)
         {
             _imruClient = imruClient;
         }
@@ -75,7 +75,7 @@ namespace Org.Apache.REEF.IMRU.Examples.PipelinedBroadcastReduce
                         chunkSize.ToString(CultureInfo.InvariantCulture))
                     .Build();
 
-            var results = _imruClient.Submit(
+            var results = _imruClient.Submit<int[], int[], int[]>(
                 new IMRUJobDefinitionBuilder()
                     .SetMapFunctionConfiguration(IMRUMapConfiguration<int[], int[]>.ConfigurationModule
                         .Set(IMRUMapConfiguration<int[], int[]>.MapFunction,

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/lang/cs/Org.Apache.REEF.IMRU.Tests/MapperCountTest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU.Tests/MapperCountTest.cs b/lang/cs/Org.Apache.REEF.IMRU.Tests/MapperCountTest.cs
index ad0e477..2874e13 100644
--- a/lang/cs/Org.Apache.REEF.IMRU.Tests/MapperCountTest.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU.Tests/MapperCountTest.cs
@@ -41,8 +41,8 @@ namespace Org.Apache.REEF.IMRU.Tests
             var tested =
                 TangFactory.GetTang()
                     .NewInjector(
-                        InProcessIMRUConfiguration<int, int, int>.ConfigurationModule
-                            .Set(InProcessIMRUConfiguration<int, int, int>.NumberOfMappers, NumberOfMappers.ToString())
+                        InProcessIMRUConfiguration.ConfigurationModule
+                            .Set(InProcessIMRUConfiguration.NumberOfMappers, NumberOfMappers.ToString())
                             .Build()
                     )
                     .GetInstance<MapperCount>();

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/lang/cs/Org.Apache.REEF.IMRU/API/IIMRUClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU/API/IIMRUClient.cs b/lang/cs/Org.Apache.REEF.IMRU/API/IIMRUClient.cs
index 247d721..51682de 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/API/IIMRUClient.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU/API/IIMRUClient.cs
@@ -21,18 +21,16 @@ using System.Collections.Generic;
 
 namespace Org.Apache.REEF.IMRU.API
 {
-    /// <summary>
-    /// Job submission interface for IMRU jobs.
-    /// </summary>
-    /// <typeparam name="TMapInput">The type of the side information provided to the Map function</typeparam>
-    /// <typeparam name="TMapOutput">The return type of the Map function</typeparam>
-    /// <typeparam name="TResult">The return type of the computation.</typeparam>
-    public interface IIMRUClient<TMapInput, TMapOutput, out TResult>
-    {
+    public interface IIMRUClient
+    {        
         /// <summary>
         /// Submit the given job for execution.
         /// </summary>
-        /// <param name="jobDefinition"></param>
-        IEnumerable<TResult> Submit(IMRUJobDefinition jobDefinition);
+        /// <typeparam name="TMapInput">The type of the side information provided to the Map function</typeparam>
+        /// <typeparam name="TMapOutput">The return type of the Map function</typeparam>
+        /// <typeparam name="TResult">The return type of the computation.</typeparam>
+        /// <param name="jobDefinition">IMRU job definition</param>
+        /// <returns>Result of IMRU</returns>
+        IEnumerable<TResult> Submit<TMapInput, TMapOutput, TResult>(IMRUJobDefinition jobDefinition);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUClient.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUClient.cs b/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUClient.cs
index 77f7d9b..08cf201 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUClient.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUClient.cs
@@ -40,14 +40,11 @@ namespace Org.Apache.REEF.IMRU.InProcess
     /// </summary>
     /// <remarks>
     /// This client assumes that all given Configurations can be merged in a conflict-free way.
-    /// </remarks>
-    /// <typeparam name="TMapInput">The type of the side information provided to the Map function</typeparam>
-    /// <typeparam name="TMapOutput">The return type of the Map function</typeparam>
-    /// <typeparam name="TResult">The return type of the computation.</typeparam>
-    public class InProcessIMRUClient<TMapInput, TMapOutput, TResult> : IIMRUClient<TMapInput, TMapOutput, TResult>
+    /// </remarks> 
+    public class InProcessIMRUClient : IIMRUClient
     {
         private static readonly Logger Logger =
-            Logger.GetLogger(typeof (InProcessIMRUClient<TMapInput, TMapOutput, TResult>));
+            Logger.GetLogger(typeof (InProcessIMRUClient));
 
         private readonly int _numberOfMappers;
 
@@ -65,9 +62,12 @@ namespace Org.Apache.REEF.IMRU.InProcess
         /// <summary>
         /// Submits the map job
         /// </summary>
+        /// <typeparam name="TMapInput">The type of the side information provided to the Map function</typeparam>
+        /// <typeparam name="TMapOutput">The return type of the Map function</typeparam>
+        /// <typeparam name="TResult">The return type of the computation.</typeparam>
         /// <param name="jobDefinition">Job definition given by the user</param>
         /// <returns>The result of the job</returns>
-        public IEnumerable<TResult> Submit(IMRUJobDefinition jobDefinition)
+        public IEnumerable<TResult> Submit<TMapInput, TMapOutput, TResult>(IMRUJobDefinition jobDefinition)
         {
             IConfiguration overallPerMapConfig = null;
             try
@@ -91,7 +91,8 @@ namespace Org.Apache.REEF.IMRU.InProcess
                 (ISet<IPerMapperConfigGenerator>) injector.GetNamedInstance(typeof (PerMapConfigGeneratorSet));
 
             injector.BindVolatileInstance(GenericType<MapFunctions<TMapInput, TMapOutput>>.Class,
-                MakeMapFunctions(jobDefinition.MapFunctionConfiguration, jobDefinition.PartitionedDatasetConfiguration, perMapConfigGenerators));
+                MakeMapFunctions<TMapInput, TMapOutput>(jobDefinition.MapFunctionConfiguration,
+                    jobDefinition.PartitionedDatasetConfiguration, perMapConfigGenerators));
 
             var runner = injector.GetInstance<IMRURunner<TMapInput, TMapOutput, TResult>>();
             return runner.Run();
@@ -104,7 +105,7 @@ namespace Org.Apache.REEF.IMRU.InProcess
         /// <param name="partitionedDataSetConfig">Partitioned dataset configuration</param>
         /// <param name="perMapConfigGenerators">Per map configuration generators</param>
         /// <returns></returns>
-        private MapFunctions<TMapInput, TMapOutput> MakeMapFunctions(IConfiguration mapConfiguration, IConfiguration partitionedDataSetConfig, ISet<IPerMapperConfigGenerator> perMapConfigGenerators)
+        private MapFunctions<TMapInput, TMapOutput> MakeMapFunctions<TMapInput, TMapOutput>(IConfiguration mapConfiguration, IConfiguration partitionedDataSetConfig, ISet<IPerMapperConfigGenerator> perMapConfigGenerators)
         {
             IPartitionedDataSet dataset =
                 TangFactory.GetTang().NewInjector(partitionedDataSetConfig).GetInstance<IPartitionedDataSet>();

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUConfiguration.cs b/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUConfiguration.cs
index 6a2f01b..76bbe36 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUConfiguration.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU/InProcess/InProcessIMRUConfiguration.cs
@@ -27,10 +27,7 @@ namespace Org.Apache.REEF.IMRU.InProcess
     /// <summary>
     /// Configuration module for the in-process IMRU.
     /// </summary>
-    /// <typeparam name="TMapInput">The type of the side information provided to the Map function</typeparam>
-    /// <typeparam name="TMapOutput">The return type of the Map function</typeparam>
-    /// <typeparam name="TResult">The return type of the computation.</typeparam>
-    public sealed class InProcessIMRUConfiguration<TMapInput, TMapOutput, TResult> : ConfigurationModuleBuilder
+    public sealed class InProcessIMRUConfiguration : ConfigurationModuleBuilder
     {
         /// <summary>
         /// The number of Mappers to instantiate.
@@ -41,9 +38,9 @@ namespace Org.Apache.REEF.IMRU.InProcess
         /// Configuration module
         /// </summary>
         public static ConfigurationModule ConfigurationModule =
-            new InProcessIMRUConfiguration<TMapInput, TMapOutput, TResult>()
-                .BindImplementation(GenericType<IIMRUClient<TMapInput, TMapOutput, TResult>>.Class,
-                    GenericType<InProcessIMRUClient<TMapInput, TMapOutput, TResult>>.Class)
+            new InProcessIMRUConfiguration()
+                .BindImplementation(GenericType<IIMRUClient>.Class,
+                    GenericType<InProcessIMRUClient>.Class)
                 .BindNamedParameter(GenericType<NumberOfMappers>.Class, NumberOfMappers)
                 .Build();
     }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/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 9e74f43..d178c4f 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClient.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClient.cs
@@ -43,12 +43,9 @@ namespace Org.Apache.REEF.IMRU.OnREEF.Client
     /// <summary>
     /// Implements the IMRU client API on REEF.
     /// </summary>
-    /// <typeparam name="TMapInput">The type of the side information provided to the Map function</typeparam>
-    /// <typeparam name="TMapOutput">The return type of the Map function</typeparam>
-    /// <typeparam name="TResult">The return type of the computation.</typeparam>
-    internal sealed class REEFIMRUClient<TMapInput, TMapOutput, TResult> : IIMRUClient<TMapInput, TMapOutput, TResult>
+    internal sealed class REEFIMRUClient : IIMRUClient
     {
-        private static readonly Logger Logger = Logger.GetLogger(typeof(IMRUDriver<TMapInput, TMapOutput, TResult>));
+        private static readonly Logger Logger = Logger.GetLogger(typeof (REEFIMRUClient));
 
         private readonly IREEFClient _reefClient;
         private readonly JobSubmissionBuilderFactory _jobSubmissionBuilderFactory;
@@ -66,13 +63,16 @@ namespace Org.Apache.REEF.IMRU.OnREEF.Client
         /// <summary>
         /// Submits the job to reefClient
         /// </summary>
+        /// <typeparam name="TMapInput">The type of the side information provided to the Map function</typeparam>
+        /// <typeparam name="TMapOutput">The return type of the Map function</typeparam>
+        /// <typeparam name="TResult">The return type of the computation.</typeparam>
         /// <param name="jobDefinition">IMRU job definition given by the user</param>
         /// <returns>Null as results will be later written to some directory</returns>
-        IEnumerable<TResult> IIMRUClient<TMapInput, TMapOutput, TResult>.Submit(IMRUJobDefinition jobDefinition)
+        IEnumerable<TResult> IIMRUClient.Submit<TMapInput, TMapOutput, TResult>(IMRUJobDefinition jobDefinition)
         {
             string driverId = string.Format("IMRU-{0}-Driver", jobDefinition.JobName);
             IConfiguration overallPerMapConfig = null;
-            
+
             try
             {
                 overallPerMapConfig = Configurations.Merge(jobDefinition.PerMapConfigGeneratorConfig.ToArray());

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/06db1ab1/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClientConfiguration.cs b/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClientConfiguration.cs
index ac3bef3..0f3aa0a 100644
--- a/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClientConfiguration.cs
+++ b/lang/cs/Org.Apache.REEF.IMRU/OnREEF/Client/REEFIMRUClientConfiguration.cs
@@ -27,15 +27,15 @@ namespace Org.Apache.REEF.IMRU.OnREEF.Client
     /// <summary>
     /// A configuration module for specifying REEFIMRUClient
     /// </summary>
-    public sealed class REEFIMRUClientConfiguration<TMapInput, TMapOutput, TResult> : ConfigurationModuleBuilder
+    public sealed class REEFIMRUClientConfiguration : ConfigurationModuleBuilder
     {
         /// <summary>
         /// Configuration module
         /// </summary>
         public static ConfigurationModule ConfigurationModule =
-            new REEFIMRUClientConfiguration<TMapInput, TMapOutput, TResult>()
-                .BindImplementation(GenericType<IIMRUClient<TMapInput, TMapOutput, TResult>>.Class,
-                    GenericType<REEFIMRUClient<TMapInput, TMapOutput, TResult>>.Class)
+            new REEFIMRUClientConfiguration()
+                .BindImplementation(GenericType<IIMRUClient>.Class,
+                    GenericType<REEFIMRUClient>.Class)
                 .Build();
     }
 }
\ No newline at end of file