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/05/05 00:17:43 UTC

incubator-reef git commit: [REEF-305] Introduce RemoteManagerFactry to .NET

Repository: incubator-reef
Updated Branches:
  refs/heads/master 33b6edf1d -> cc30a3589


[REEF-305] Introduce RemoteManagerFactry to .NET

This adds `IRemoteManagerFactory` as well as an implementation that
creates `DefaultRemoteManager` instances. Futher, this change marks
all constructors in that class as `[Obsolete]` and replaces all calls
to them within the REEF code base with respective calls to the
appropriate `IRemoteManagerFactory.GetInstance()` methods.

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

Pull Request:
  This closes #172

 Author:    Markus Weimer <we...@apache.org>


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

Branch: refs/heads/master
Commit: cc30a3589eff7acf784c44f5e6aab5229607bc83
Parents: 33b6edf
Author: Markus Weimer <we...@apache.org>
Authored: Mon May 4 10:46:42 2015 -0700
Committer: Julia Wang <jw...@yahoo.com>
Committed: Mon May 4 15:09:38 2015 -0700

----------------------------------------------------------------------
 lang/cs/Org.Apache.REEF.Evaluator/Evaluator.cs  |  2 +-
 .../GroupCommunicationTests.cs                  |  3 +-
 .../NetworkService/NetworkServiceTests.cs       |  7 +--
 .../NetworkService/NetworkService.cs            |  6 ++-
 .../RemoteManagerTest.cs                        | 39 +++++++-------
 .../Impl/DefaultRemoteManagerFactory.cs         | 53 ++++++++++++++++++++
 .../Org.Apache.REEF.Wake.csproj                 |  2 +
 .../Remote/IRemoteManagerFactory.cs             | 47 +++++++++++++++++
 .../Remote/Impl/DefaultRemoteManager.cs         |  6 ++-
 9 files changed, 139 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/lang/cs/Org.Apache.REEF.Evaluator/Evaluator.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Evaluator/Evaluator.cs b/lang/cs/Org.Apache.REEF.Evaluator/Evaluator.cs
index e09367e..5e094f6 100644
--- a/lang/cs/Org.Apache.REEF.Evaluator/Evaluator.cs
+++ b/lang/cs/Org.Apache.REEF.Evaluator/Evaluator.cs
@@ -109,7 +109,7 @@ namespace Org.Apache.REEF.Evaluator
                     Optional<ServiceConfiguration> rootServiceConfig = _evaluatorConfig.RootServiceConfiguration;
 
                     // remoteManager used as client-only in evaluator
-                    IRemoteManager<REEFMessage> remoteManager = new DefaultRemoteManager<REEFMessage>(new REEFMessageCodec());
+                    IRemoteManager<REEFMessage> remoteManager = _injector.GetInstance<IRemoteManagerFactory>().GetInstance((new REEFMessageCodec()));
                     IRemoteIdentifier remoteId = new SocketRemoteIdentifier(NetUtilities.ParseIpEndpoint(rId));
 
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/lang/cs/Org.Apache.REEF.Network.Tests/GroupCommunication/GroupCommunicationTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network.Tests/GroupCommunication/GroupCommunicationTests.cs b/lang/cs/Org.Apache.REEF.Network.Tests/GroupCommunication/GroupCommunicationTests.cs
index b9092aa..26fa16c 100644
--- a/lang/cs/Org.Apache.REEF.Network.Tests/GroupCommunication/GroupCommunicationTests.cs
+++ b/lang/cs/Org.Apache.REEF.Network.Tests/GroupCommunication/GroupCommunicationTests.cs
@@ -745,8 +745,9 @@ namespace Org.Apache.REEF.Network.Tests.GroupCommunication
         public static NetworkService<GroupCommunicationMessage> BuildNetworkService(
             IPEndPoint nameServerEndpoint, IObserver<NsMessage<GroupCommunicationMessage>> handler)
         {
+            var remoteManagerFactory = TangFactory.GetTang().NewInjector().GetInstance<IRemoteManagerFactory>();
             return new NetworkService<GroupCommunicationMessage>(
-                0, handler, new StringIdentifierFactory(), new GroupCommunicationMessageCodec(), new NameClient(nameServerEndpoint.Address.ToString(), nameServerEndpoint.Port));
+                0, handler, new StringIdentifierFactory(), new GroupCommunicationMessageCodec(), new NameClient(nameServerEndpoint.Address.ToString(), nameServerEndpoint.Port), remoteManagerFactory);
         }
 
         private GroupCommunicationMessage CreateGcm(string message, string from, string to)

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/lang/cs/Org.Apache.REEF.Network.Tests/NetworkService/NetworkServiceTests.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network.Tests/NetworkService/NetworkServiceTests.cs b/lang/cs/Org.Apache.REEF.Network.Tests/NetworkService/NetworkServiceTests.cs
index 1489b3c..e52a082 100644
--- a/lang/cs/Org.Apache.REEF.Network.Tests/NetworkService/NetworkServiceTests.cs
+++ b/lang/cs/Org.Apache.REEF.Network.Tests/NetworkService/NetworkServiceTests.cs
@@ -156,10 +156,11 @@ namespace Org.Apache.REEF.Network.Tests.NetworkService
                  nameServiceAddr)
              .BindImplementation(GenericType<INameClient>.Class, GenericType<NameClient>.Class)
              .Build();
-
-            var nameClient = TangFactory.GetTang().NewInjector(nameserverConf).GetInstance<NameClient>();
+            var injector = TangFactory.GetTang().NewInjector(nameserverConf);
+            var nameClient = injector.GetInstance<NameClient>();
+            var remoteManagerFactory = injector.GetInstance<IRemoteManagerFactory>();
             return new NetworkService<string>(networkServicePort,
-                handler, new StringIdentifierFactory(), new StringCodec(), nameClient);
+                handler, new StringIdentifierFactory(), new StringCodec(), nameClient, remoteManagerFactory);
         }
 
         private class MessageHandler : IObserver<NsMessage<string>>

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/lang/cs/Org.Apache.REEF.Network/NetworkService/NetworkService.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Network/NetworkService/NetworkService.cs b/lang/cs/Org.Apache.REEF.Network/NetworkService/NetworkService.cs
index 6d4c0a4..43e55c1 100644
--- a/lang/cs/Org.Apache.REEF.Network/NetworkService/NetworkService.cs
+++ b/lang/cs/Org.Apache.REEF.Network/NetworkService/NetworkService.cs
@@ -57,18 +57,20 @@ namespace Org.Apache.REEF.Network.NetworkService
         /// <param name="messageHandler">The observer to handle incoming messages</param>
         /// <param name="idFactory">The factory used to create IIdentifiers</param>
         /// <param name="codec">The codec used for serialization</param>
+        /// <param name="remoteManagerFactory">Used to instantiate remote manager instances.</param>
         [Inject]
         public NetworkService(
             [Parameter(typeof(NetworkServiceOptions.NetworkServicePort))] int nsPort,
             IObserver<NsMessage<T>> messageHandler,
             IIdentifierFactory idFactory,
             ICodec<T> codec,
-            INameClient nameClient)
+            INameClient nameClient,
+            IRemoteManagerFactory remoteManagerFactory)
         {
             _codec = new NsMessageCodec<T>(codec, idFactory);
 
             IPAddress localAddress = NetworkUtils.LocalIPAddress;
-            _remoteManager = new DefaultRemoteManager<NsMessage<T>>(localAddress, nsPort, _codec);
+            _remoteManager = remoteManagerFactory.GetInstance(localAddress, nsPort, _codec);
             _messageHandler = messageHandler;
 
             NamingClient = nameClient;

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/lang/cs/Org.Apache.REEF.Wake.Tests/RemoteManagerTest.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake.Tests/RemoteManagerTest.cs b/lang/cs/Org.Apache.REEF.Wake.Tests/RemoteManagerTest.cs
index 3b3ac6d..f24eb29 100644
--- a/lang/cs/Org.Apache.REEF.Wake.Tests/RemoteManagerTest.cs
+++ b/lang/cs/Org.Apache.REEF.Wake.Tests/RemoteManagerTest.cs
@@ -22,6 +22,7 @@ using System.Collections.Generic;
 using System.Net;
 using System.Reactive;
 using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Org.Apache.REEF.Tang.Implementations.Tang;
 using Org.Apache.REEF.Wake.Remote;
 using Org.Apache.REEF.Wake.Remote.Impl;
 using Org.Apache.REEF.Wake.Util;
@@ -31,6 +32,8 @@ namespace Org.Apache.REEF.Wake.Tests
     [TestClass]
     public class RemoteManagerTest
     {
+        private readonly IRemoteManagerFactory _remoteManagerFactory =
+            TangFactory.GetTang().NewInjector().GetInstance<IRemoteManagerFactory>();
         [TestMethod]
         public void TestOneWayCommunication()
         {
@@ -39,8 +42,8 @@ namespace Org.Apache.REEF.Wake.Tests
             BlockingCollection<string> queue = new BlockingCollection<string>();
             List<string> events = new List<string>();
 
-            using (var remoteManager1 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager2 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager1 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
             {
                 var observer = Observer.Create<string>(queue.Add);
                 IPEndPoint endpoint1 = new IPEndPoint(listeningAddress, 0);
@@ -68,8 +71,8 @@ namespace Org.Apache.REEF.Wake.Tests
             BlockingCollection<string> queue = new BlockingCollection<string>();
             List<string> events = new List<string>();
 
-            using (var remoteManager1 = new DefaultRemoteManager<string>(new StringCodec()))
-            using (var remoteManager2 = new DefaultRemoteManager<string>(listeningAddress, listeningPort, new StringCodec()))
+            using (var remoteManager1 = _remoteManagerFactory.GetInstance(new StringCodec()))
+            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, listeningPort, new StringCodec()))
             {
                 IPEndPoint remoteEndpoint = new IPEndPoint(listeningAddress, 0);
                 var observer = Observer.Create<string>(queue.Add);
@@ -98,8 +101,8 @@ namespace Org.Apache.REEF.Wake.Tests
             List<string> events1 = new List<string>();
             List<string> events2 = new List<string>();
 
-            using (var remoteManager1 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager2 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager1 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
             {
                 // Register observers for remote manager 1 and remote manager 2
                 var remoteEndpoint = new IPEndPoint(listeningAddress, 0);
@@ -143,9 +146,9 @@ namespace Org.Apache.REEF.Wake.Tests
             BlockingCollection<string> queue = new BlockingCollection<string>();
             List<string> events = new List<string>();
 
-            using (var remoteManager1 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager2 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager3 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager1 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager3 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
             {
                 var remoteEndpoint = new IPEndPoint(listeningAddress, 0);
                 var observer = Observer.Create<string>(queue.Add);
@@ -181,9 +184,9 @@ namespace Org.Apache.REEF.Wake.Tests
             List<string> events2 = new List<string>();
             List<string> events3 = new List<string>();
 
-            using (var remoteManager1 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager2 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager3 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager1 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager3 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
             {
                 var remoteEndpoint = new IPEndPoint(listeningAddress, 0);
 
@@ -241,8 +244,8 @@ namespace Org.Apache.REEF.Wake.Tests
             BlockingCollection<string> queue = new BlockingCollection<string>();
             List<string> events = new List<string>();
 
-            using (var remoteManager1 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager2 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager1 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
             {
                 // Register handler for when remote manager 2 receives events; respond
                 // with an ack
@@ -282,8 +285,8 @@ namespace Org.Apache.REEF.Wake.Tests
             BlockingCollection<string> queue = new BlockingCollection<string>();
             List<string> events = new List<string>();
 
-            using (var remoteManager1 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager2 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager1 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
             {
                 // RemoteManager2 listens and records events of type IRemoteEvent<string>
                 var observer = Observer.Create<IRemoteMessage<string>>(message => queue.Add(message.Message));
@@ -311,8 +314,8 @@ namespace Org.Apache.REEF.Wake.Tests
             BlockingCollection<string> queue = new BlockingCollection<string>();
             List<string> events = new List<string>();
 
-            using (var remoteManager1 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
-            using (var remoteManager2 = new DefaultRemoteManager<string>(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager1 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
+            using (var remoteManager2 = _remoteManagerFactory.GetInstance(listeningAddress, 0, new StringCodec()))
             {
                 var observer = Observer.Create<string>(queue.Add);
                 IPEndPoint endpoint1 = new IPEndPoint(listeningAddress, 0);

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/lang/cs/Org.Apache.REEF.Wake/Impl/DefaultRemoteManagerFactory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake/Impl/DefaultRemoteManagerFactory.cs b/lang/cs/Org.Apache.REEF.Wake/Impl/DefaultRemoteManagerFactory.cs
new file mode 100644
index 0000000..38a020f
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Wake/Impl/DefaultRemoteManagerFactory.cs
@@ -0,0 +1,53 @@
+/*
+ * 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.Net;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Wake.Remote;
+using Org.Apache.REEF.Wake.Remote.Impl;
+
+namespace Org.Apache.REEF.Wake.Impl
+{
+    /// <summary>
+    /// An implementation of IRemoteManagerFactory for DefaultRemoteManager.
+    /// </summary>
+    internal sealed class DefaultRemoteManagerFactory : IRemoteManagerFactory
+    {
+        [Inject]
+        private DefaultRemoteManagerFactory()
+        {
+        }
+
+        public IRemoteManager<T> GetInstance<T>(IPAddress localAddress, int port, ICodec<T> codec)
+        {
+#pragma warning disable 618
+            // This is the one place allowed to call this constructor. Hence, disabling the warning is OK.
+            return new DefaultRemoteManager<T>(localAddress, port, codec);
+#pragma warning restore 618
+        }
+
+        public IRemoteManager<T> GetInstance<T>(ICodec<T> codec)
+        {
+#pragma warning disable 618
+            // This is the one place allowed to call this constructor. Hence, disabling the warning is OK.
+            return new DefaultRemoteManager<T>(codec);
+#pragma warning restore 618
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/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 cc1ec2e..26f2fbf 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
@@ -48,6 +48,7 @@ under the License.
     <Compile Include="IEventHandler.cs" />
     <Compile Include="IIdentifier.cs" />
     <Compile Include="IIdentifierFactory.cs" />
+    <Compile Include="Impl\DefaultRemoteManagerFactory.cs" />
     <Compile Include="Impl\LoggingEventHandler.cs" />
     <Compile Include="Impl\MissingStartHandlerHandler.cs" />
     <Compile Include="Impl\MultiEventHandler.cs" />
@@ -67,6 +68,7 @@ under the License.
     <Compile Include="Remote\Impl\WritableLink.cs" />
     <Compile Include="Remote\Impl\WritableTransportClient.cs" />
     <Compile Include="Remote\Impl\WritableTransportServer.cs" />
+    <Compile Include="Remote\IRemoteManagerFactory.cs" />
     <Compile Include="Remote\IWritable.cs" />
     <Compile Include="Remote\Proto\WakeRemoteProtosGen.cs" />
     <Compile Include="Remote\ICodec.cs" />

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/lang/cs/Org.Apache.REEF.Wake/Remote/IRemoteManagerFactory.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/IRemoteManagerFactory.cs b/lang/cs/Org.Apache.REEF.Wake/Remote/IRemoteManagerFactory.cs
new file mode 100644
index 0000000..7d8041b
--- /dev/null
+++ b/lang/cs/Org.Apache.REEF.Wake/Remote/IRemoteManagerFactory.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.Net;
+using Org.Apache.REEF.Tang.Annotations;
+using Org.Apache.REEF.Wake.Impl;
+
+namespace Org.Apache.REEF.Wake.Remote
+{
+    /// <summary>
+    /// Creates new intsances of IRemoteManager.
+    /// </summary>
+    [DefaultImplementation(typeof(DefaultRemoteManagerFactory))]
+    public interface IRemoteManagerFactory
+    {
+        /// <summary>
+        /// Constructs a DefaultRemoteManager listening on the specified address and any
+        /// available port.
+        /// </summary>
+        /// <param name="localAddress">The address to listen on</param>
+        /// <param name="port">The port to listen on</param>
+        /// <param name="codec">The codec used for serializing messages</param>
+        IRemoteManager<T> GetInstance<T>(IPAddress localAddress, int port, ICodec<T> codec);
+
+        /// <summary>
+        /// Constructs a DefaultRemoteManager. Does not listen for incoming messages.
+        /// </summary>
+        /// <param name="codec">The codec used for serializing messages</param>
+        IRemoteManager<T> GetInstance<T>(ICodec<T> codec);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/cc30a358/lang/cs/Org.Apache.REEF.Wake/Remote/Impl/DefaultRemoteManager.cs
----------------------------------------------------------------------
diff --git a/lang/cs/Org.Apache.REEF.Wake/Remote/Impl/DefaultRemoteManager.cs b/lang/cs/Org.Apache.REEF.Wake/Remote/Impl/DefaultRemoteManager.cs
index 4f92a1f..210ebcf 100644
--- a/lang/cs/Org.Apache.REEF.Wake/Remote/Impl/DefaultRemoteManager.cs
+++ b/lang/cs/Org.Apache.REEF.Wake/Remote/Impl/DefaultRemoteManager.cs
@@ -28,7 +28,7 @@ namespace Org.Apache.REEF.Wake.Remote.Impl
     /// <summary>
     /// Manages incoming and outgoing messages between remote hosts.
     /// </summary>
-    public class DefaultRemoteManager<T> : IRemoteManager<T>
+    public sealed class DefaultRemoteManager<T> : IRemoteManager<T>
     {
         private static readonly Logger LOGGER = Logger.GetLogger(typeof(DefaultRemoteManager<T>));
 
@@ -43,6 +43,7 @@ namespace Org.Apache.REEF.Wake.Remote.Impl
         /// </summary>
         /// <param name="localAddress">The address to listen on</param>
         /// <param name="codec">The codec used for serializing messages</param>
+        [Obsolete("Use IRemoteManagerFactory.GetInstance() instead.", false)]
         public DefaultRemoteManager(IPAddress localAddress, ICodec<T> codec) : this(localAddress, 0, codec)
         {
         }
@@ -52,6 +53,7 @@ namespace Org.Apache.REEF.Wake.Remote.Impl
         /// </summary>
         /// <param name="localEndpoint">The endpoint to listen on</param>
         /// <param name="codec">The codec used for serializing messages</param>
+        [Obsolete("Use IRemoteManagerFactory.GetInstance() instead.", false)]
         public DefaultRemoteManager(IPEndPoint localEndpoint, ICodec<T> codec)
         {
             if (localEndpoint == null)
@@ -86,6 +88,7 @@ namespace Org.Apache.REEF.Wake.Remote.Impl
         /// <param name="localAddress">The address to listen on</param>
         /// <param name="port">The port to listen on</param>
         /// <param name="codec">The codec used for serializing messages</param>
+        [Obsolete("Use IRemoteManagerFactory.GetInstance() instead.", false)]
         public DefaultRemoteManager(IPAddress localAddress, int port, ICodec<T> codec)
         {
             if (localAddress == null)
@@ -119,6 +122,7 @@ namespace Org.Apache.REEF.Wake.Remote.Impl
         /// Constructs a DefaultRemoteManager. Does not listen for incoming messages.
         /// </summary>
         /// <param name="codec">The codec used for serializing messages</param>
+        [Obsolete("Use IRemoteManagerFactory.GetInstance() instead.", false)]
         public DefaultRemoteManager(ICodec<T> codec)
         {
             using (LOGGER.LogFunction("DefaultRemoteManager::DefaultRemoteManager"))