Prechádzať zdrojové kódy

HADOOP-16152. Upgrade Eclipse Jetty version to 9.4.x. Contributed by Yuming Wang, Siyao Meng.

Co-authored-By: Siyao Meng <smeng@cloudera.com>
Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org>
Yuming Wang 5 rokov pred
rodič
commit
3d41f33018

+ 16 - 0
hadoop-client-modules/hadoop-client-minicluster/pom.xml

@@ -811,6 +811,15 @@
                         <exclude>*/**</exclude>
                       </excludes>
                     </filter>
+                    <!-- Jetty 9.4.x: jetty-client and jetty-xml are depended by org.eclipse.jetty.websocket:websocket-client.
+                         But we are only excluding jetty-client not jetty-xml because HttpServer2 implicitly uses the shaded package name.
+                    -->
+                    <filter>
+                      <artifact>org.eclipse.jetty:jetty-client</artifact>
+                      <excludes>
+                        <exclude>*/**</exclude>
+                      </excludes>
+                    </filter>
                   </filters>
 
                   <!-- relocate classes from mssql-jdbc -->
@@ -939,6 +948,13 @@
                         <exclude>**/pom.xml</exclude>
                       </excludes>
                     </relocation>
+                    <relocation>
+                      <pattern>javax/websocket/</pattern>
+                      <shadedPattern>${shaded.dependency.prefix}.javax.websocket.</shadedPattern>
+                      <excludes>
+                        <exclude>**/pom.xml</exclude>
+                      </excludes>
+                    </relocation>
                     <relocation>
                       <pattern>jersey/</pattern>
                       <shadedPattern>${shaded.dependency.prefix}.jersey.</shadedPattern>

+ 3 - 0
hadoop-client-modules/hadoop-client-runtime/pom.xml

@@ -161,6 +161,9 @@
                       <exclude>org.eclipse.jetty.websocket:*</exclude>
                       <exclude>org.eclipse.jetty:jetty-servlet</exclude>
                       <exclude>org.eclipse.jetty:jetty-security</exclude>
+                      <exclude>org.eclipse.jetty:jetty-client</exclude>
+                      <exclude>org.eclipse.jetty:jetty-http</exclude>
+                      <exclude>org.eclipse.jetty:jetty-xml</exclude>
                       <exclude>org.ow2.asm:*</exclude>
                       <!-- Leave bouncycastle unshaded because it's signed with a special Oracle certificate so it can be a custom JCE security provider -->
                       <exclude>org.bouncycastle:*</exclude>

+ 7 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java

@@ -24,7 +24,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogConfigurationException;
 import org.apache.commons.logging.LogFactory;
 import org.apache.log4j.Appender;
-import org.eclipse.jetty.server.NCSARequestLog;
+import org.eclipse.jetty.server.AsyncRequestLogWriter;
+import org.eclipse.jetty.server.CustomRequestLog;
 import org.eclipse.jetty.server.RequestLog;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -85,10 +86,11 @@ public class HttpRequestLog {
       if (appender instanceof HttpRequestLogAppender) {
         HttpRequestLogAppender requestLogAppender
           = (HttpRequestLogAppender)appender;
-        NCSARequestLog requestLog = new NCSARequestLog();
-        requestLog.setFilename(requestLogAppender.getFilename());
-        requestLog.setRetainDays(requestLogAppender.getRetainDays());
-        return requestLog;
+        AsyncRequestLogWriter logWriter = new AsyncRequestLogWriter();
+        logWriter.setFilename(requestLogAppender.getFilename());
+        logWriter.setRetainDays(requestLogAppender.getRetainDays());
+        return new CustomRequestLog(logWriter,
+            CustomRequestLog.EXTENDED_NCSA_FORMAT);
       } else {
         LOG.warn("Jetty request log for {} was of the wrong class", loggerName);
         return null;

+ 16 - 28
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java

@@ -88,13 +88,11 @@ import org.eclipse.jetty.server.RequestLog;
 import org.eclipse.jetty.server.SecureRequestCustomizer;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.server.SessionManager;
 import org.eclipse.jetty.server.SslConnectionFactory;
 import org.eclipse.jetty.server.handler.AllowSymLinkAliasChecker;
 import org.eclipse.jetty.server.handler.ContextHandlerCollection;
 import org.eclipse.jetty.server.handler.HandlerCollection;
 import org.eclipse.jetty.server.handler.RequestLogHandler;
-import org.eclipse.jetty.server.session.AbstractSessionManager;
 import org.eclipse.jetty.server.session.SessionHandler;
 import org.eclipse.jetty.servlet.FilterHolder;
 import org.eclipse.jetty.servlet.FilterMapping;
@@ -519,7 +517,8 @@ public final class HttpServer2 implements FilterContainer {
       httpConfig.addCustomizer(new SecureRequestCustomizer());
       ServerConnector conn = createHttpChannelConnector(server, httpConfig);
 
-      SslContextFactory sslContextFactory = new SslContextFactory();
+      SslContextFactory.Server sslContextFactory =
+          new SslContextFactory.Server();
       sslContextFactory.setNeedClientAuth(needsClientAuth);
       sslContextFactory.setKeyManagerPassword(keyPassword);
       if (keyStore != null) {
@@ -621,12 +620,9 @@ public final class HttpServer2 implements FilterContainer {
       threadPool.setMaxThreads(maxThreads);
     }
 
-    SessionManager sm = webAppContext.getSessionHandler().getSessionManager();
-    if (sm instanceof AbstractSessionManager) {
-      AbstractSessionManager asm = (AbstractSessionManager)sm;
-      asm.setHttpOnly(true);
-      asm.getSessionCookieConfig().setSecure(true);
-    }
+    SessionHandler handler = webAppContext.getSessionHandler();
+    handler.setHttpOnly(true);
+    handler.getSessionCookieConfig().setSecure(true);
 
     ContextHandlerCollection contexts = new ContextHandlerCollection();
     RequestLog requestLog = HttpRequestLog.getRequestLog(name);
@@ -777,12 +773,8 @@ public final class HttpServer2 implements FilterContainer {
       }
       logContext.setDisplayName("logs");
       SessionHandler handler = new SessionHandler();
-      SessionManager sm = handler.getSessionManager();
-      if (sm instanceof AbstractSessionManager) {
-        AbstractSessionManager asm = (AbstractSessionManager) sm;
-        asm.setHttpOnly(true);
-        asm.getSessionCookieConfig().setSecure(true);
-      }
+      handler.setHttpOnly(true);
+      handler.getSessionCookieConfig().setSecure(true);
       logContext.setSessionHandler(handler);
       logContext.addAliasCheck(new AllowSymLinkAliasChecker());
       setContextAttributes(logContext, conf);
@@ -800,12 +792,8 @@ public final class HttpServer2 implements FilterContainer {
     params.put("org.eclipse.jetty.servlet.Default.dirAllowed", "false");
     params.put("org.eclipse.jetty.servlet.Default.gzip", "true");
     SessionHandler handler = new SessionHandler();
-    SessionManager sm = handler.getSessionManager();
-    if (sm instanceof AbstractSessionManager) {
-      AbstractSessionManager asm = (AbstractSessionManager) sm;
-      asm.setHttpOnly(true);
-      asm.getSessionCookieConfig().setSecure(true);
-    }
+    handler.setHttpOnly(true);
+    handler.getSessionCookieConfig().setSecure(true);
     staticContext.setSessionHandler(handler);
     staticContext.addAliasCheck(new AllowSymLinkAliasChecker());
     setContextAttributes(staticContext, conf);
@@ -1268,7 +1256,7 @@ public final class HttpServer2 implements FilterContainer {
    * @return
    */
   private static BindException constructBindException(ServerConnector listener,
-      BindException ex) {
+      IOException ex) {
     BindException be = new BindException("Port in use: "
         + listener.getHost() + ":" + listener.getPort());
     if (ex != null) {
@@ -1290,7 +1278,7 @@ public final class HttpServer2 implements FilterContainer {
       try {
         bindListener(listener);
         break;
-      } catch (BindException ex) {
+      } catch (IOException ex) {
         if (port == 0 || !findPort) {
           throw constructBindException(listener, ex);
         }
@@ -1310,13 +1298,13 @@ public final class HttpServer2 implements FilterContainer {
    */
   private void bindForPortRange(ServerConnector listener, int startPort)
       throws Exception {
-    BindException bindException = null;
+    IOException ioException = null;
     try {
       bindListener(listener);
       return;
-    } catch (BindException ex) {
+    } catch (IOException ex) {
       // Ignore exception.
-      bindException = ex;
+      ioException = ex;
     }
     for(Integer port : portRanges) {
       if (port == startPort) {
@@ -1329,10 +1317,10 @@ public final class HttpServer2 implements FilterContainer {
         return;
       } catch (BindException ex) {
         // Ignore exception. Move to next port.
-        bindException = ex;
+        ioException = ex;
       }
     }
-    throw constructBindException(listener, bindException);
+    throw constructBindException(listener, ioException);
   }
 
   /**

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java

@@ -18,7 +18,7 @@
 package org.apache.hadoop.http;
 
 import org.apache.log4j.Logger;
-import org.eclipse.jetty.server.NCSARequestLog;
+import org.eclipse.jetty.server.CustomRequestLog;
 import org.eclipse.jetty.server.RequestLog;
 import org.junit.Test;
 
@@ -42,6 +42,7 @@ public class TestHttpRequestLog {
     RequestLog requestLog = HttpRequestLog.getRequestLog("test");
     Logger.getLogger("http.requests.test").removeAppender(requestLogAppender);
     assertNotNull("RequestLog should not be null", requestLog);
-    assertEquals("Class mismatch", NCSARequestLog.class, requestLog.getClass());
+    assertEquals("Class mismatch",
+        CustomRequestLog.class, requestLog.getClass());
   }
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java

@@ -105,7 +105,8 @@ public class TestJettyHelper implements MethodRule {
       conn.setHost(host);
       conn.setPort(port);
       if (ssl) {
-        SslContextFactory sslContextFactory = new SslContextFactory();
+        SslContextFactory.Server sslContextFactory =
+            new SslContextFactory.Server();
         sslContextFactory.setNeedClientAuth(false);
         sslContextFactory.setKeyStorePath(keyStore);
         sslContextFactory.setKeyStoreType(keyStoreType);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java

@@ -83,8 +83,9 @@ public class DatanodeHttpServer implements Closeable {
   // set them to the minimum possible
   private static final int HTTP_SELECTOR_THREADS = 1;
   private static final int HTTP_ACCEPTOR_THREADS = 1;
+  // Jetty 9.4.x: Adding one more thread to HTTP_MAX_THREADS.
   private static final int HTTP_MAX_THREADS =
-      HTTP_SELECTOR_THREADS + HTTP_ACCEPTOR_THREADS + 1;
+      HTTP_SELECTOR_THREADS + HTTP_ACCEPTOR_THREADS + 2;
   private final HttpServer2 infoServer;
   private final EventLoopGroup bossGroup;
   private final EventLoopGroup workerGroup;

+ 1 - 1
hadoop-project/pom.xml

@@ -35,7 +35,7 @@
 
     <failIfNoTests>false</failIfNoTests>
     <maven.test.redirectTestOutputToFile>true</maven.test.redirectTestOutputToFile>
-    <jetty.version>9.3.27.v20190418</jetty.version>
+    <jetty.version>9.4.20.v20190813</jetty.version>
     <test.exclude>_</test.exclude>
     <test.exclude.pattern>_</test.exclude.pattern>
 

+ 1 - 2
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java

@@ -97,7 +97,6 @@ import org.apache.hadoop.yarn.sls.utils.SLSUtils;
 import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.eclipse.jetty.util.ConcurrentHashSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -371,7 +370,7 @@ public class SLSRunner extends Configured implements Tool {
 
     // create NM simulators
     Random random = new Random();
-    Set<String> rackSet = new ConcurrentHashSet<>();
+    Set<String> rackSet = ConcurrentHashMap.newKeySet();
     int threadPoolSize = Math.max(poolSize,
         SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
     ExecutorService executorService = Executors.

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/pom.xml

@@ -126,6 +126,14 @@
                     <groupId>org.eclipse.jetty</groupId>
                     <artifactId>jetty-io</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>org.eclipse.jetty</groupId>
+                    <artifactId>jetty-xml</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.eclipse.jetty</groupId>
+                    <artifactId>jetty-http</artifactId>
+                </exclusion>
             </exclusions>
             <scope>test</scope>
         </dependency>
@@ -143,6 +151,14 @@
                     <groupId>org.eclipse.jetty</groupId>
                     <artifactId>jetty-io</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>org.eclipse.jetty</groupId>
+                    <artifactId>jetty-xml</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.eclipse.jetty</groupId>
+                    <artifactId>jetty-http</artifactId>
+                </exclusion>
             </exclusions>
             <scope>test</scope>
         </dependency>