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 2015/10/16 15:21:04 UTC

incubator-reef git commit: [REEF-592] Remove deprecated classes in REEF-591

Repository: incubator-reef
Updated Branches:
  refs/heads/master 1e598fa38 -> 4c072712b


[REEF-592] Remove deprecated classes in REEF-591

This patch:
  * Removes the legacy port assigning path in HttpServerImpl
  * Removes deprecated PortNumber, MinPort, MaxPort and MaxAttempts classes
  * Removes relevant injectable parameters from the HttpServerImpl constructor
  * Removes httpServerPortRetryTest and httpServerPortRangeTest from TestHttpServer
Relevant tests cases are covered by the tests using TcpPortProvider in the same class

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

Pull request:
  This closes #570


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

Branch: refs/heads/master
Commit: 4c072712b24e680481b065d887b749c66840d34a
Parents: 1e598fa
Author: sergey.dudoladov@tu-berlin.de <se...@tu-berlin.de>
Authored: Tue Oct 13 14:05:59 2015 +0200
Committer: Dongjoon Hyun <do...@apache.org>
Committed: Fri Oct 16 15:19:30 2015 +0900

----------------------------------------------------------------------
 .../apache/reef/webserver/HttpServerImpl.java   | 27 +++------
 .../apache/reef/webserver/MaxPortNumber.java    | 31 ----------
 .../apache/reef/webserver/MaxRetryAttempts.java | 31 ----------
 .../apache/reef/webserver/MinPortNumber.java    | 31 ----------
 .../org/apache/reef/webserver/PortNumber.java   | 35 ------------
 .../apache/reef/webserver/TestHttpServer.java   | 60 +-------------------
 .../apache/reef/webserver/TestTrackingUri.java  |  3 +-
 7 files changed, 10 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/4c072712/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/HttpServerImpl.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/HttpServerImpl.java b/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/HttpServerImpl.java
index b76ea77..581cc93 100644
--- a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/HttpServerImpl.java
+++ b/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/HttpServerImpl.java
@@ -22,6 +22,7 @@ import org.apache.reef.tang.annotations.Parameter;
 import org.apache.reef.util.logging.LoggingScope;
 import org.apache.reef.util.logging.LoggingScopeFactory;
 import org.apache.reef.wake.remote.ports.TcpPortProvider;
+import org.apache.reef.wake.remote.ports.parameters.TcpPortRangeBegin;
 import org.mortbay.jetty.Server;
 
 import javax.inject.Inject;
@@ -68,10 +69,7 @@ public final class HttpServerImpl implements HttpServer {
    */
   @Inject
   HttpServerImpl(final JettyHandler jettyHandler,
-                 @Parameter(PortNumber.class) final int portNumber,
-                 @Parameter(MaxPortNumber.class) final int maxPortNumber,
-                 @Parameter(MinPortNumber.class) final int minPortNumber,
-                 @Parameter(MaxRetryAttempts.class) final int maxRetryAttempts,
+                 @Parameter(TcpPortRangeBegin.class) final int portNumber,
                  final TcpPortProvider tcpPortProvider,
                  final LoggingScopeFactory loggingScopeFactory) throws Exception {
 
@@ -81,22 +79,11 @@ public final class HttpServerImpl implements HttpServer {
     Server srv = null;
 
     try (final LoggingScope ls = this.loggingScopeFactory.httpServer()) {
-      if (portNumber != Integer.parseInt(PortNumber.DEFAULT_VALUE)) {
-        // legacy path to be removed
-        for (int attempt = 0; attempt < maxRetryAttempts && srv  == null; ++attempt) {
-          if (attempt > 0) {
-            // first attempt should be portNumber passed in
-            availablePort = getNextPort(maxPortNumber, minPortNumber);
-          }
-          srv = tryPort(availablePort);
-        }
-      } else {
-        // new TcpPortProvider path
-        final Iterator<Integer> ports = tcpPortProvider.iterator();
-        while (ports.hasNext() && srv  == null) {
-          availablePort = ports.next();
-          srv = tryPort(availablePort);
-        }
+
+      final Iterator<Integer> ports = tcpPortProvider.iterator();
+      while (ports.hasNext() && srv  == null) {
+        availablePort = ports.next();
+        srv = tryPort(availablePort);
       }
 
       if (srv  != null) {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/4c072712/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MaxPortNumber.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MaxPortNumber.java b/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MaxPortNumber.java
deleted file mode 100644
index 2c05c76..0000000
--- a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MaxPortNumber.java
+++ /dev/null
@@ -1,31 +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.
- */
-package org.apache.reef.webserver;
-
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-
-/**
- * max port number range when generating a port number for the Http Server.
- * @deprecated in 0.13, use TcpPortRangeCount and TcpPortProvider instead.
- */
-@NamedParameter(doc = "Max port number for Jetty Server", default_value = "49151")
-@Deprecated
-public class MaxPortNumber implements Name<Integer> {
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/4c072712/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MaxRetryAttempts.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MaxRetryAttempts.java b/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MaxRetryAttempts.java
deleted file mode 100644
index c523187..0000000
--- a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MaxRetryAttempts.java
+++ /dev/null
@@ -1,31 +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.
- */
-package org.apache.reef.webserver;
-
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-
-/**
- * Max retry times when generating a port number for the Http Server.
- * @deprecated in 0.13, use TcpPortRangeTryCount and TcpPortProvider instead.
- */
-@NamedParameter(doc = "Maximum retry attempts for port number of Jetty Server", default_value = "100")
-@Deprecated
-public class MaxRetryAttempts implements Name<Integer> {
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/4c072712/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MinPortNumber.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MinPortNumber.java b/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MinPortNumber.java
deleted file mode 100644
index ecc76bb..0000000
--- a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/MinPortNumber.java
+++ /dev/null
@@ -1,31 +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.
- */
-package org.apache.reef.webserver;
-
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-
-/**
- * minimum port number range when generating a port number for the Http Server.
- * @deprecated in 0.13, use TcpPortRangeStart and TcpPortProvider instead.
- */
-@NamedParameter(doc = "Minimum port number for Jetty Server", default_value = "1024")
-@Deprecated
-public class MinPortNumber implements Name<Integer> {
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/4c072712/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/PortNumber.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/PortNumber.java b/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/PortNumber.java
deleted file mode 100644
index 92bad3b..0000000
--- a/lang/java/reef-webserver/src/main/java/org/apache/reef/webserver/PortNumber.java
+++ /dev/null
@@ -1,35 +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.
- */
-package org.apache.reef.webserver;
-
-import org.apache.reef.tang.annotations.Name;
-import org.apache.reef.tang.annotations.NamedParameter;
-
-/**
- * port number for the Http Server
- * Clients can set a preferred port number. However, Reef will detect if the given port number has been used
- * in the machine. If yes, it will generate another available port number.
- * @deprecated in 0.13, use TcpPortRangeStart and TcpPortProvider instead.
- */
-@NamedParameter(doc = "Port number for Jetty Server", default_value = PortNumber.DEFAULT_VALUE)
-@Deprecated
-public final class PortNumber implements Name<Integer> {
-  private PortNumber(){}
-  public static final String DEFAULT_VALUE = "8080";
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/4c072712/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestHttpServer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestHttpServer.java b/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestHttpServer.java
index cf05e6f..28905c7 100644
--- a/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestHttpServer.java
+++ b/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestHttpServer.java
@@ -56,7 +56,7 @@ public class TestHttpServer {
     final Configuration httpRuntimeConfiguration = HttpRuntimeConfiguration.CONF.build();
 
     final Configuration httpServerConfiguration = Tang.Factory.getTang().newConfigurationBuilder()
-        .bindNamedParameter(PortNumber.class, "9000")
+        .bindNamedParameter(TcpPortRangeBegin.class, "9000")
         .build();
 
     final Configuration configuration =
@@ -86,33 +86,6 @@ public class TestHttpServer {
   }
 
   @Test
-  public void httpServerPortRangeTest() throws Exception {
-    final Configuration httpRuntimeConfiguration = HttpRuntimeConfiguration.CONF.build();
-
-    final Configuration httpServerConfiguration = Tang.Factory.getTang().newConfigurationBuilder()
-        .bindNamedParameter(PortNumber.class, "6000")
-        .bindNamedParameter(MaxPortNumber.class, "9900")
-        .bindNamedParameter(MinPortNumber.class, "1000")
-        .bindNamedParameter(MaxRetryAttempts.class, "3")
-        .build();
-
-    final Configuration configuration =
-        Configurations.merge(httpRuntimeConfiguration, httpServerConfiguration);
-
-    final Injector injector1 = Tang.Factory.getTang().newInjector(configuration);
-    final HttpServer httpServer1 = injector1.getInstance(HttpServer.class);
-
-    final Injector injector2 = Tang.Factory.getTang().newInjector(configuration);
-    final HttpServer httpServer2 = injector2.getInstance(HttpServer.class);
-
-    Assert.assertTrue("port number is out of specified range",
-        httpServer2.getPort() >= 1000 && httpServer2.getPort() <= 9900);
-
-    httpServer1.stop();
-    httpServer2.stop();
-  }
-
-  @Test
   public void httpServerPortRangeTestWithTcpPortProvider() throws Exception {
     final Configuration httpRuntimeConfiguration = HttpRuntimeConfiguration.CONF.build();
 
@@ -169,37 +142,6 @@ public class TestHttpServer {
   }
 
   @Test
-  public void httpServerPortRetryTest() throws Exception {
-
-    final Configuration httpRuntimeConfiguration = HttpRuntimeConfiguration.CONF.build();
-    final Injector injector1 = Tang.Factory.getTang().newInjector(httpRuntimeConfiguration);
-    final HttpServer httpServer1 = injector1.getInstance(HttpServer.class);
-    final String portUsed = "" + httpServer1.getPort();
-
-    final Configuration httpServerConfiguration = Tang.Factory.getTang().newConfigurationBuilder()
-        .bindNamedParameter(PortNumber.class, portUsed)
-        .bindNamedParameter(MaxPortNumber.class, portUsed)
-        .bindNamedParameter(MinPortNumber.class, portUsed)
-        .bindNamedParameter(MaxRetryAttempts.class, "3")
-        .build();
-
-    final Configuration configuration =
-        Configurations.merge(httpRuntimeConfiguration, httpServerConfiguration);
-
-    final Injector injector2 = Tang.Factory.getTang().newInjector(configuration);
-
-    try {
-      injector2.getInstance(HttpServer.class);
-      Assert.fail("Created two web servers on the same port: " + portUsed);
-    } catch (final InjectionException ex) {
-      Assert.assertEquals("Could not find available port for http", ex.getCause().getMessage());
-    }
-
-    httpServer1.stop();
-  }
-
-
-  @Test
   public void httpServerAddHandlerTest() throws Exception {
 
     final Configuration httpRuntimeConfiguration = HttpRuntimeConfiguration.CONF.build();

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/4c072712/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestTrackingUri.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestTrackingUri.java b/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestTrackingUri.java
index a53ffc9..a2ac62e 100644
--- a/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestTrackingUri.java
+++ b/lang/java/reef-webserver/src/test/java/org/apache/reef/webserver/TestTrackingUri.java
@@ -24,6 +24,7 @@ import org.apache.reef.tang.JavaConfigurationBuilder;
 import org.apache.reef.tang.Tang;
 import org.apache.reef.tang.exceptions.BindException;
 import org.apache.reef.tang.exceptions.InjectionException;
+import org.apache.reef.wake.remote.ports.parameters.TcpPortRangeBegin;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -55,7 +56,7 @@ public class TestTrackingUri {
   @Test
   public void testHttpTrackingUri() throws InjectionException, UnknownHostException, BindException {
     final JavaConfigurationBuilder cb = Tang.Factory.getTang().newConfigurationBuilder()
-        .bindNamedParameter(PortNumber.class, "8888")
+        .bindNamedParameter(TcpPortRangeBegin.class, "8888")
         .bindImplementation(TrackingURLProvider.class, HttpTrackingURLProvider.class)
         .bindImplementation(HttpServer.class, HttpServerImpl.class);