You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flume.apache.org by hs...@apache.org on 2014/11/12 20:51:54 UTC

[1/2] flume git commit: FLUME-2548: Enable SSLv2Hello for Avro Source and NettyAvroRpcClient

Repository: flume
Updated Branches:
  refs/heads/flume-1.5 451fad027 -> 3aceda6eb


FLUME-2548: Enable SSLv2Hello for Avro Source and NettyAvroRpcClient

(Hari Shreedharan via Jarek Jarcec Cecho)


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

Branch: refs/heads/flume-1.5
Commit: 2a59111090433a49ef2bad56219c4838277238dc
Parents: 451fad0
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Wed Nov 12 06:16:47 2014 -0800
Committer: Hari Shreedharan <hs...@apache.org>
Committed: Wed Nov 12 11:50:30 2014 -0800

----------------------------------------------------------------------
 .../java/org/apache/flume/source/AvroSource.java | 19 +++++++++++--------
 flume-ng-doc/sphinx/FlumeUserGuide.rst           |  2 +-
 .../org/apache/flume/api/NettyAvroRpcClient.java | 15 +++++++++++----
 3 files changed, 23 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flume/blob/2a591110/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java
----------------------------------------------------------------------
diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java
index 59ee43a..6eb6a0a 100644
--- a/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java
+++ b/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java
@@ -25,11 +25,7 @@ import java.io.FileInputStream;
 import java.net.InetSocketAddress;
 import java.security.KeyStore;
 import java.security.Security;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -149,7 +145,7 @@ public class AvroSource extends AbstractSource implements EventDrivenSource,
   private String keystore;
   private String keystorePassword;
   private String keystoreType;
-  private List<String> excludeProtocols;
+  private final List<String> excludeProtocols = new LinkedList<String>();
   private boolean enableSsl = false;
   private boolean enableIpFilter;
   private String patternRuleConfigDefinition;
@@ -181,8 +177,15 @@ public class AvroSource extends AbstractSource implements EventDrivenSource,
     keystore = context.getString(KEYSTORE_KEY);
     keystorePassword = context.getString(KEYSTORE_PASSWORD_KEY);
     keystoreType = context.getString(KEYSTORE_TYPE_KEY, "JKS");
-    excludeProtocols = Arrays.asList(
-        context.getString(EXCLUDE_PROTOCOLS, "SSLv2Hello SSLv3").split(" "));
+    String excludeProtocolsStr = context.getString(EXCLUDE_PROTOCOLS);
+    if (excludeProtocolsStr == null) {
+      excludeProtocols.add("SSLv3");
+    } else {
+      excludeProtocols.addAll(Arrays.asList(excludeProtocolsStr.split(" ")));
+      if (!excludeProtocols.contains("SSLv3")) {
+        excludeProtocols.add("SSLv3");
+      }
+    }
 
     if (enableSsl) {
       Preconditions.checkNotNull(keystore,

http://git-wip-us.apache.org/repos/asf/flume/blob/2a591110/flume-ng-doc/sphinx/FlumeUserGuide.rst
----------------------------------------------------------------------
diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst
index c70f6de..2da8092 100644
--- a/flume-ng-doc/sphinx/FlumeUserGuide.rst
+++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst
@@ -705,7 +705,7 @@ ssl                  false             Set this to true to enable SSL encryption
 keystore             --                This is the path to a Java keystore file. Required for SSL.
 keystore-password    --                The password for the Java keystore. Required for SSL.
 keystore-type        JKS               The type of the Java keystore. This can be "JKS" or "PKCS12".
-exclude-protocols    SSLv2Hello SSLv3  Space-separated list of SSL/TLS protocols to exclude
+exclude-protocols    SSLv3             Space-separated list of SSL/TLS protocols to exclude. SSLv3 will always be excluded in addition to the protocols specified.
 ipFilter             false             Set this to true to enable ipFiltering for netty
 ipFilter.rules       --                Define N netty ipFilter pattern rules with this config.
 ==================   ================  ===================================================

http://git-wip-us.apache.org/repos/asf/flume/blob/2a591110/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java
----------------------------------------------------------------------
diff --git a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java
index ad9b580..3661672 100644
--- a/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java
+++ b/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java
@@ -96,7 +96,7 @@ implements RpcClient {
   private String truststore;
   private String truststorePassword;
   private String truststoreType;
-  private List<String> excludeProtocols;
+  private final List<String> excludeProtocols = new LinkedList<String>();
 
   private Transceiver transceiver;
   private AvroSourceProtocol.Callback avroClient;
@@ -607,9 +607,16 @@ implements RpcClient {
         RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_PASSWORD);
     truststoreType = properties.getProperty(
         RpcClientConfigurationConstants.CONFIG_TRUSTSTORE_TYPE, "JKS");
-    excludeProtocols = Arrays.asList(properties.getProperty(
-        RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS, "SSLv2Hello SSLv3")
-        .split(" "));
+    String excludeProtocolsStr = properties.getProperty(
+      RpcClientConfigurationConstants.CONFIG_EXCLUDE_PROTOCOLS);
+    if (excludeProtocolsStr == null) {
+      excludeProtocols.add("SSLv3");
+    } else {
+      excludeProtocols.addAll(Arrays.asList(excludeProtocolsStr.split(" ")));
+      if (!excludeProtocols.contains("SSLv3")) {
+        excludeProtocols.add("SSLv3");
+      }
+    }
 
     String maxIoWorkersStr = properties.getProperty(
       RpcClientConfigurationConstants.MAX_IO_WORKERS);


[2/2] flume git commit: FLUME-2549: Enable SSLv2Hello for HttpSource

Posted by hs...@apache.org.
FLUME-2549: Enable SSLv2Hello for HttpSource

(Hari Shreedharan via Jarek Jarcec Cecho)


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

Branch: refs/heads/flume-1.5
Commit: 3aceda6eb8f677e457e3322ac2fce9e2719a8c6d
Parents: 2a59111
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Wed Nov 12 07:35:06 2014 -0800
Committer: Hari Shreedharan <hs...@apache.org>
Committed: Wed Nov 12 11:50:41 2014 -0800

----------------------------------------------------------------------
 .../apache/flume/source/http/HTTPSource.java    | 23 +++++-
 .../http/HTTPSourceConfigurationConstants.java  |  1 +
 .../flume/source/http/TestHTTPSource.java       | 79 +-------------------
 flume-ng-doc/sphinx/FlumeUserGuide.rst          | 31 ++++----
 4 files changed, 39 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flume/blob/3aceda6e/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java
----------------------------------------------------------------------
diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java
index 4b2717c..b520b03 100644
--- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java
+++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSource.java
@@ -93,6 +93,7 @@ public class HTTPSource extends AbstractSource implements
   private volatile String keyStorePath;
   private volatile String keyStorePassword;
   private volatile Boolean sslEnabled;
+  private final List<String> excludedProtocols = new LinkedList<String>();
 
 
   @Override
@@ -120,7 +121,18 @@ public class HTTPSource extends AbstractSource implements
         Preconditions.checkArgument(keyStorePath != null && !keyStorePath.isEmpty(),
                                         "Keystore is required for SSL Conifguration" );
         keyStorePassword = context.getString(HTTPSourceConfigurationConstants.SSL_KEYSTORE_PASSWORD);
-        Preconditions.checkArgument(keyStorePassword != null, "Keystore password is required for SSL Configuration");
+        Preconditions.checkArgument(keyStorePassword != null,
+          "Keystore password is required for SSL Configuration");
+        String excludeProtocolsStr = context.getString(HTTPSourceConfigurationConstants
+          .EXCLUDE_PROTOCOLS);
+        if (excludeProtocolsStr == null) {
+          excludedProtocols.add("SSLv3");
+        } else {
+          excludedProtocols.addAll(Arrays.asList(excludeProtocolsStr.split(" ")));
+          if (!excludedProtocols.contains("SSLv3")) {
+            excludedProtocols.add("SSLv3");
+          }
+        }
       }
 
 
@@ -172,7 +184,7 @@ public class HTTPSource extends AbstractSource implements
 
 
     if (sslEnabled) {
-      SslSocketConnector sslSocketConnector = new HTTPSourceSocketConnector();
+      SslSocketConnector sslSocketConnector = new HTTPSourceSocketConnector(excludedProtocols);
       sslSocketConnector.setKeystore(keyStorePath);
       sslSocketConnector.setKeyPassword(keyStorePassword);
       sslSocketConnector.setReuseAddress(true);
@@ -274,6 +286,11 @@ public class HTTPSource extends AbstractSource implements
 
   private static class HTTPSourceSocketConnector extends SslSocketConnector {
 
+    private final List<String> excludedProtocols;
+    HTTPSourceSocketConnector(List<String> excludedProtocols) {
+      this.excludedProtocols = excludedProtocols;
+    }
+
     @Override
     public ServerSocket newServerSocket(String host, int port,
       int backlog) throws IOException {
@@ -282,7 +299,7 @@ public class HTTPSource extends AbstractSource implements
       String[] protocols = socket.getEnabledProtocols();
       List<String> newProtocols = new ArrayList<String>(protocols.length);
       for(String protocol: protocols) {
-        if (!(protocol.equals("SSLv3") || protocol.equals("SSLv2Hello"))) {
+        if (!excludedProtocols.contains(protocol)) {
           newProtocols.add(protocol);
         }
       }

http://git-wip-us.apache.org/repos/asf/flume/blob/3aceda6e/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java
----------------------------------------------------------------------
diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java
index ed52827..86caf7d 100644
--- a/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java
+++ b/flume-ng-core/src/main/java/org/apache/flume/source/http/HTTPSourceConfigurationConstants.java
@@ -37,5 +37,6 @@ public class HTTPSourceConfigurationConstants {
   public static final String SSL_KEYSTORE = "keystore";
   public static final String SSL_KEYSTORE_PASSWORD = "keystorePassword";
   public static final String SSL_ENABLED = "enableSSL";
+  public static final String EXCLUDE_PROTOCOLS = "excludeProtocols";
 
 }

http://git-wip-us.apache.org/repos/asf/flume/blob/3aceda6e/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java
----------------------------------------------------------------------
diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java
index 64111be..c59fdd4 100644
--- a/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java
+++ b/flume-ng-core/src/test/java/org/apache/flume/source/http/TestHTTPSource.java
@@ -321,11 +321,6 @@ public class TestHTTPSource {
     doTestHttps("SSLv3");
   }
 
-  @Test (expected = javax.net.ssl.SSLHandshakeException.class)
-  public void testHttpsSSLv2Hello() throws Exception {
-    doTestHttps("SSLv2Hello");
-  }
-
   public void doTestHttps(String protocol) throws Exception {
     Type listType = new TypeToken<List<JSONEvent>>() {
     }.getType();
@@ -384,7 +379,7 @@ public class TestHTTPSource {
       if(protocol != null) {
         factory = new DisabledProtocolsSocketFactory(sc.getSocketFactory(), protocol);
       } else {
-        factory = new EnabledProtocolsSocketFactory(sc.getSocketFactory());
+        factory = sc.getSocketFactory();
       }
       HttpsURLConnection.setDefaultSSLSocketFactory(factory);
       HttpsURLConnection.setDefaultHostnameVerifier(
@@ -498,78 +493,8 @@ public class TestHTTPSource {
 
     DisabledProtocolsSocketFactory(javax.net.ssl.SSLSocketFactory factory, String protocol) {
       this.socketFactory = factory;
-      if(protocol.equals("SSLv2Hello")) {
-        protocols = new String[2];
-        protocols[0] = "TLSv1";
-        protocols[1] = protocol;
-      } else {
-        protocols = new String[1];
-        protocols[0] = protocol;
-      }
-    }
-
-    @Override
-    public String[] getDefaultCipherSuites() {
-      return socketFactory.getDefaultCipherSuites();
-    }
-
-    @Override
-    public String[] getSupportedCipherSuites() {
-      return socketFactory.getSupportedCipherSuites();
-    }
-
-    @Override
-    public Socket createSocket(Socket socket, String s, int i, boolean b)
-      throws IOException {
-      SSLSocket sc = (SSLSocket) socketFactory.createSocket(socket, s, i, b);
-      sc.setEnabledProtocols(protocols);
-      return sc;
-    }
-
-    @Override
-    public Socket createSocket(String s, int i)
-      throws IOException, UnknownHostException {
-      SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i);
-      sc.setEnabledProtocols(protocols);
-      return sc;
-    }
-
-    @Override
-    public Socket createSocket(String s, int i, InetAddress inetAddress, int i2)
-      throws IOException, UnknownHostException {
-      SSLSocket sc = (SSLSocket)socketFactory.createSocket(s, i, inetAddress,
-        i2);
-      sc.setEnabledProtocols(protocols);
-      return sc;
-    }
-
-    @Override
-    public Socket createSocket(InetAddress inetAddress, int i)
-      throws IOException {
-      SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i);
-      sc.setEnabledProtocols(protocols);
-      return sc;
-    }
-
-    @Override
-    public Socket createSocket(InetAddress inetAddress, int i,
-      InetAddress inetAddress2, int i2) throws IOException {
-      SSLSocket sc = (SSLSocket)socketFactory.createSocket(inetAddress, i,
-        inetAddress2, i2);
-      sc.setEnabledProtocols(protocols);
-      return sc;
-    }
-  }
-
-  private class EnabledProtocolsSocketFactory extends javax.net.ssl.SSLSocketFactory {
-
-    private final javax.net.ssl.SSLSocketFactory socketFactory;
-    private final String[] protocols;
-
-    EnabledProtocolsSocketFactory(javax.net.ssl.SSLSocketFactory factory) {
-      this.socketFactory = factory;
       protocols = new String[1];
-      protocols[0] = "TLSv1";
+      protocols[0] = protocol;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/flume/blob/3aceda6e/flume-ng-doc/sphinx/FlumeUserGuide.rst
----------------------------------------------------------------------
diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst
index 2da8092..68bac8b 100644
--- a/flume-ng-doc/sphinx/FlumeUserGuide.rst
+++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst
@@ -1292,22 +1292,23 @@ unavailable status.
 All events sent in one post request are considered to be one batch and
 inserted into the channel in one transaction.
 
-==============  ============================================  ====================================================================
-Property Name   Default                                       Description
-==============  ============================================  ====================================================================
-**type**                                                      The component type name, needs to be ``http``
-**port**        --                                            The port the source should bind to.
-bind            0.0.0.0                                       The hostname or IP address to listen on
-handler         ``org.apache.flume.source.http.JSONHandler``  The FQCN of the handler class.
-handler.*       --                                            Config parameters for the handler
-selector.type   replicating                                   replicating or multiplexing
-selector.*                                                    Depends on the selector.type value
-interceptors    --                                            Space-separated list of interceptors
+=================  ============================================  =====================================================================================
+Property Name      Default                                       Description
+=================  ============================================  =====================================================================================
+**type**                                                         The component type name, needs to be ``http``
+**port**           --                                            The port the source should bind to.
+bind               0.0.0.0                                       The hostname or IP address to listen on
+handler            ``org.apache.flume.source.http.JSONHandler``  The FQCN of the handler class.
+handler.*          --                                            Config parameters for the handler
+selector.type      replicating                                   replicating or multiplexing
+selector.*                                                       Depends on the selector.type value
+interceptors       --                                            Space-separated list of interceptors
 interceptors.*
-enableSSL       false                                         Set the property true, to enable SSL
-keystore                                                      Location of the keystore includng keystore file name
-keystorePassword                                              Keystore password
-==================================================================================================================================
+enableSSL          false                                         Set the property true, to enable SSL. *HTTP Source does not support SSLv3.*
+excludeProtocols   SSLv3                                         Space-separated list of SSL/TLS protocols to exclude. SSLv3 is always excluded.
+keystore                                                         Location of the keystore includng keystore file name
+keystorePassword                                                 Keystore password
+======================================================================================================================================================
 
 For example, a http source for agent named a1: