You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by sr...@apache.org on 2015/04/26 19:46:29 UTC

[1/6] storm git commit: STORM-721. Storm UI server should support SSL.

Repository: storm
Updated Branches:
  refs/heads/master 0cbc89c19 -> 168fa2f81


STORM-721. Storm UI server should support SSL.


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

Branch: refs/heads/master
Commit: e86b924e860bfe7722785b81f9647c5697a9461f
Parents: dcee1e2
Author: Sriharsha Chintalapani <ma...@harsha.io>
Authored: Tue Mar 24 15:48:03 2015 -0700
Committer: Sriharsha Chintalapani <ma...@harsha.io>
Committed: Tue Mar 24 15:48:03 2015 -0700

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/drpc.clj      | 10 +++--
 storm-core/src/clj/backtype/storm/ui/core.clj   | 14 ++++++-
 .../src/clj/backtype/storm/ui/helpers.clj       | 44 +++++++++++++-------
 storm-core/src/jvm/backtype/storm/Config.java   | 39 +++++++++++++++++
 4 files changed, 86 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e86b924e/storm-core/src/clj/backtype/storm/daemon/drpc.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
index cb9b1d2..2ab3cce 100644
--- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
@@ -200,7 +200,7 @@
           drpc-port (int (conf DRPC-PORT))
           drpc-service-handler (service-handler conf)
           ;; requests and returns need to be on separate thread pools, since calls to
-          ;; "execute" don't unblock until other thrift methods are called. So if 
+          ;; "execute" don't unblock until other thrift methods are called. So if
           ;; 64 threads are calling execute, the server won't accept the result
           ;; invocations that will unblock those threads
           handler-server (when (> drpc-port 0)
@@ -210,7 +210,7 @@
           invoke-server (ThriftServer. conf
                           (DistributedRPCInvocations$Processor. drpc-service-handler)
                           ThriftConnectionType/DRPC_INVOCATIONS)
-          http-creds-handler (AuthUtils/GetDrpcHttpCredentialsPlugin conf)] 
+          http-creds-handler (AuthUtils/GetDrpcHttpCredentialsPlugin conf)]
       (add-shutdown-hook-with-force-kill-in-1-sec (fn []
                                                     (if handler-server (.stop handler-server))
                                                     (.stop invoke-server)))
@@ -225,7 +225,8 @@
               https-port (int (conf DRPC-HTTPS-PORT))
               https-ks-path (conf DRPC-HTTPS-KEYSTORE-PATH)
               https-ks-password (conf DRPC-HTTPS-KEYSTORE-PASSWORD)
-              https-ks-type (conf DRPC-HTTPS-KEYSTORE-TYPE)]
+              https-ks-type (conf DRPC-HTTPS-KEYSTORE-TYPE)
+              https-key-password (conf DRPC-HTTPS-KEY-PASSWORD)]
 
           (storm-run-jetty
            {:port drpc-http-port
@@ -234,7 +235,8 @@
                                         https-port
                                         https-ks-path
                                         https-ks-password
-                                        https-ks-type)
+                                        https-ks-type
+                                        https-key-password)
                             (config-filter server app filters-confs))})))
       (when handler-server
         (.serve handler-server)))))

http://git-wip-us.apache.org/repos/asf/storm/blob/e86b924e/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index c64f35d..fa39ad1 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -1022,10 +1022,22 @@
     (let [conf *STORM-CONF*
           header-buffer-size (int (.get conf UI-HEADER-BUFFER-BYTES))
           filters-confs [{:filter-class (conf UI-FILTER)
-                          :filter-params (conf UI-FILTER-PARAMS)}]]
+                          :filter-params (conf UI-FILTER-PARAMS)}]
+          https-port (if (not-nil? (conf UI-HTTPS-PORT)) (conf UI-HTTPS-PORT) 0)
+          https-ks-path (conf UI-HTTPS-KEYSTORE-PATH)
+          https-ks-password (conf UI-HTTPS-KEYSTORE-PASSWORD)
+          https-ks-type (conf UI-HTTPS-KEYSTORE-TYPE)
+          https-key-password (conf UI-HTTPS-KEY-PASSWORD)]
       (storm-run-jetty {:port (conf UI-PORT)
                         :host (conf UI-HOST)
+                        :https-port https-port
                         :configurator (fn [server]
+                                        (config-ssl server
+                                                    https-port
+                                                    https-ks-path
+                                                    https-ks-password
+                                                    https-ks-type
+                                                    https-key-password)
                                         (doseq [connector (.getConnectors server)]
                                           (.setRequestHeaderSize connector header-buffer-size))
                                         (config-filter server app filters-confs))}))

http://git-wip-us.apache.org/repos/asf/storm/blob/e86b924e/storm-core/src/clj/backtype/storm/ui/helpers.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj
index 98d8e4b..e6cbb8c 100644
--- a/storm-core/src/clj/backtype/storm/ui/helpers.clj
+++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj
@@ -20,13 +20,14 @@
          [string :only [blank? join]]
          [walk :only [keywordize-keys]]])
   (:use [backtype.storm config log])
-  (:use [backtype.storm.util :only [clojurify-structure uuid defnk url-encode]])
+  (:use [backtype.storm.util :only [clojurify-structure uuid defnk url-encode not-nil?]])
   (:use [clj-time coerce format])
   (:import [backtype.storm.generated ExecutorInfo ExecutorSummary])
   (:import [org.eclipse.jetty.server Server]
            [org.eclipse.jetty.server.nio SelectChannelConnector]
            [org.eclipse.jetty.server.ssl SslSocketConnector]
-           [org.eclipse.jetty.servlet ServletHolder FilterMapping])
+           [org.eclipse.jetty.servlet ServletHolder FilterMapping]
+           [org.eclipse.jetty.util.ssl SslContextFactory])
   (:require [ring.util servlet])
   (:require [compojure.route :as route]
             [compojure.handler :as handler]))
@@ -132,20 +133,22 @@
 (defn unauthorized-user-html [user]
   [[:h2 "User '" (escape-html user) "' is not authorized."]])
 
-(defn- mk-ssl-connector [port ks-path ks-password ks-type]
-  (doto (SslSocketConnector.)
-    (.setExcludeCipherSuites (into-array String ["SSL_RSA_WITH_RC4_128_MD5" "SSL_RSA_WITH_RC4_128_SHA"]))
-    (.setExcludeProtocols (into-array String ["SSLv3"]))
-    (.setAllowRenegotiate false)
-    (.setKeystore ks-path)
-    (.setKeystoreType ks-type)
-    (.setKeyPassword ks-password)
-    (.setPassword ks-password)
-    (.setPort port)))
-
-(defn config-ssl [server port ks-path ks-password ks-type]
+(defn- mk-ssl-connector [port ks-path ks-password ks-type key-password]
+  (let [sslContextFactory (doto (SslContextFactory.)
+                            (.setExcludeCipherSuites (into-array String ["SSL_RSA_WITH_RC4_128_MD5" "SSL_RSA_WITH_RC4_128_SHA"]))
+                            (.setExcludeProtocols (into-array String ["SSLv3"]))
+                            (.setAllowRenegotiate false)
+                            (.setKeyStorePath ks-path)
+                            (.setKeyStoreType ks-type)
+                            (.setKeyStorePassword ks-password)
+                            (.setKeyManagerPassword key-password))]
+    (doto (SslSocketConnector. sslContextFactory)
+      (.setPort port))))
+
+
+(defn config-ssl [server port ks-path ks-password ks-type key-password]
   (when (> port 0)
-    (.addConnector server (mk-ssl-connector port ks-path ks-password ks-type))))
+    (.addConnector server (mk-ssl-connector port ks-path ks-password ks-type key-password))))
 
 (defn config-filter [server handler filters-confs]
   (if filters-confs
@@ -167,6 +170,13 @@
    :status 400
    :body (.getMessage ex)})
 
+(defn- remove-non-ssl-connectors [server]
+  (doseq [c (.getConnectors server)]
+    (when-not (or (nil? c) (instance? SslSocketConnector c))
+      (.removeConnector server c)
+      ))
+  server)
+
 ;; Modified from ring.adapter.jetty 1.3.0
 (defn- jetty-create-server
   "Construct a Jetty Server instance."
@@ -177,7 +187,9 @@
                     (.setMaxIdleTime (options :max-idle-time 200000)))
         server    (doto (Server.)
                     (.addConnector connector)
-                    (.setSendDateHeader true))]
+                    (.setSendDateHeader true))
+        https-port (options :https-port)]
+    (if (and (not-nil? https-port) (> https-port 0)) (remove-non-ssl-connectors server))
     server))
 
 (defn storm-run-jetty

http://git-wip-us.apache.org/repos/asf/storm/blob/e86b924e/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 9cddb2e..bc884d5 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -567,6 +567,39 @@ public class Config extends HashMap<String, Object> {
     public static final Object UI_HEADER_BUFFER_BYTES_SCHEMA = Number.class;
 
     /**
+     * This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients.
+     */
+    public static final String UI_HTTPS_PORT = "ui.https.port";
+    public static final Object UI_HTTPS_PORT_SCHEMA = Number.class;
+
+    /**
+     * Path to the keystore used by Storm UI for setting up HTTPS (SSL).
+     */
+    public static final String UI_HTTPS_KEYSTORE_PATH = "ui.https.keystore.path";
+    public static final Object UI_HTTPS_KEYSTORE_PATH_SCHEMA = String.class;
+
+    /**
+     * Password to the keystore used by Storm UI for setting up HTTPS (SSL).
+     */
+    public static final String UI_HTTPS_KEYSTORE_PASSWORD = "ui.https.keystore.password";
+    public static final Object UI_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class;
+
+    /**
+     * Type of keystore used by Storm UI for setting up HTTPS (SSL).
+     * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details.
+     */
+    public static final String UI_HTTPS_KEYSTORE_TYPE = "ui.https.keystore.type";
+    public static final Object UI_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class;
+
+    /**
+     * Password to the private key in the keystore for settting up HTTPS (SSL).
+     */
+    public static final String UI_HTTPS_KEY_PASSWORD = "ui.https.key.password";
+    public static final Object UI_HTTPS_KEY_PASSWORD_SCHEMA = String.class;
+
+
+
+    /**
      * List of DRPC servers so that the DRPCSpout knows who to talk to.
      */
     public static final String DRPC_SERVERS = "drpc.servers";
@@ -604,6 +637,12 @@ public class Config extends HashMap<String, Object> {
     public static final Object DRPC_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class;
 
     /**
+     * Password to the private key in the keystore for settting up HTTPS (SSL).
+     */
+    public static final String DRPC_HTTPS_KEY_PASSWORD = "drpc.https.key.password";
+    public static final Object DRPC_HTTPS_KEY_PASSWORD_SCHEMA = String.class;
+
+    /**
      * The DRPC transport plug-in for Thrift client/server communication
      */
     public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport";


[4/6] storm git commit: STORM-721.Storm UI server should support SSL. Added client side auth configs. update to SECURITY.md

Posted by sr...@apache.org.
STORM-721.Storm UI server should support SSL. Added client side auth
configs. update to SECURITY.md


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/00be569c
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/00be569c
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/00be569c

Branch: refs/heads/master
Commit: 00be569c799a955d7f82a7270fbf37a6178629fd
Parents: cc2520a
Author: Sriharsha Chintalapani <ma...@harsha.io>
Authored: Tue Apr 7 15:45:02 2015 -0700
Committer: Sriharsha Chintalapani <ma...@harsha.io>
Committed: Tue Apr 7 15:45:02 2015 -0700

----------------------------------------------------------------------
 SECURITY.md | 25 ++++++++++++++++++++++++-
 1 file changed, 24 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/00be569c/SECURITY.md
----------------------------------------------------------------------
diff --git a/SECURITY.md b/SECURITY.md
index 7133003..c231547 100644
--- a/SECURITY.md
+++ b/SECURITY.md
@@ -94,8 +94,20 @@ For UI users needs to set following config in storm.yaml. Generating keystores w
 4. ui.https.keystore.password (keystore password)
 5. ui.https.key.password (private key password)
 
+optional config 
+6. ui.https.truststore.path (example "/etc/ssl/storm_truststore.jks")
+7. ui.https.truststore.password (truststore password)
+8. ui.https.truststore.type (example "jks")
+
+If users want to setup 2-way auth
+9. ui.https.want.client.auth (If this set to true server requests for client certifcate authentication, but keeps the connection if no authentication provided)
+10. ui.https.need.client.auth (If this set to true server requires client to provide authentication)
+
+
+
+
 ### DRPC
-similarly to UI users need to configure following 
+similarly to UI , users need to configure following for DRPC
 
 1. drpc.https.port 
 2. drpc.https.keystore.type (example "jks")
@@ -103,6 +115,17 @@ similarly to UI users need to configure following
 4. drpc.https.keystore.password (keystore password)
 5. drpc.https.key.password (private key password)
 
+optional config 
+6. drpc.https.truststore.path (example "/etc/ssl/storm_truststore.jks")
+7. drpc.https.truststore.password (truststore password)
+8. drpc.https.truststore.type (example "jks")
+
+If users want to setup 2-way auth
+9. drpc.https.want.client.auth (If this set to true server requests for client certifcate authentication, but keeps the connection if no authentication provided)
+10. drpc.https.need.client.auth (If this set to true server requires client to provide authentication)
+
+
+
 
 
 ## Authentication (Kerberos)


[6/6] storm git commit: Added STORM-721 to CHANGELOG.

Posted by sr...@apache.org.
Added STORM-721 to CHANGELOG.


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/168fa2f8
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/168fa2f8
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/168fa2f8

Branch: refs/heads/master
Commit: 168fa2f813c4a37784f4cc28c10297eb666d05c9
Parents: 20d5476
Author: Sriharsha Chintalapani <ha...@hortonworks.com>
Authored: Sun Apr 26 10:39:36 2015 -0700
Committer: Sriharsha Chintalapani <ha...@hortonworks.com>
Committed: Sun Apr 26 10:39:36 2015 -0700

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/168fa2f8/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 23920e3..572dedb 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-721: Storm UI server should support SSL.
  * STORM-583: Add Microsoft Azure Event Hub spout implementations
  * STORM-712: Storm daemons shutdown if OutOfMemoryError occurs in any thread
  * STORM-730: remove extra curly brace


[5/6] storm git commit: Merge branch 'STORM-721' of https://github.com/harshach/incubator-storm into STORM-721

Posted by sr...@apache.org.
Merge branch 'STORM-721' of https://github.com/harshach/incubator-storm into STORM-721


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/20d5476b
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/20d5476b
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/20d5476b

Branch: refs/heads/master
Commit: 20d5476bdb769090230c375098d4fa8772b93d1c
Parents: 0cbc89c 00be569
Author: Sriharsha Chintalapani <ha...@hortonworks.com>
Authored: Sun Apr 26 10:33:27 2015 -0700
Committer: Sriharsha Chintalapani <ha...@hortonworks.com>
Committed: Sun Apr 26 10:33:27 2015 -0700

----------------------------------------------------------------------
 SECURITY.md                                     | 50 +++++++++++
 .../src/clj/backtype/storm/daemon/drpc.clj      | 20 ++++-
 storm-core/src/clj/backtype/storm/ui/core.clj   | 24 ++++-
 .../src/clj/backtype/storm/ui/helpers.clj       | 51 +++++++----
 storm-core/src/jvm/backtype/storm/Config.java   | 94 ++++++++++++++++++++
 5 files changed, 219 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/20d5476b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/20d5476b/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/20d5476b/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------


[2/6] storm git commit: Merge remote-tracking branch 'upstream/master' into STORM-721

Posted by sr...@apache.org.
Merge remote-tracking branch 'upstream/master' into STORM-721

Conflicts:
	storm-core/src/clj/backtype/storm/ui/helpers.clj


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/604026f3
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/604026f3
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/604026f3

Branch: refs/heads/master
Commit: 604026f38b25fdc796b4162b8d6919ecf3eeca68
Parents: e86b924 2aaa718
Author: Sriharsha Chintalapani <ma...@harsha.io>
Authored: Wed Apr 1 10:06:05 2015 -0700
Committer: Sriharsha Chintalapani <ma...@harsha.io>
Committed: Wed Apr 1 10:06:05 2015 -0700

----------------------------------------------------------------------
 CHANGELOG.md                                    |  13 ++
 README.markdown                                 |   2 +
 STORM-UI-REST-API.md                            |  30 +++
 docs/_posts/2015-03-25-storm094-released.md     |  24 ++
 docs/downloads.html                             |  64 +++--
 examples/storm-starter/pom.xml                  |   2 +-
 external/storm-hbase/pom.xml                    |   2 +-
 .../org/apache/storm/hbase/bolt/HBaseBolt.java  |   2 +-
 .../storm/hbase/bolt/HBaseLookupBolt.java       |   4 +-
 .../hbase/trident/state/HBaseMapState.java      |   2 +
 .../storm/hbase/trident/state/HBaseState.java   |   2 +-
 external/storm-hdfs/pom.xml                     |   2 +-
 .../org/apache/storm/hdfs/bolt/HdfsBolt.java    |   2 +-
 .../storm/hdfs/bolt/SequenceFileBolt.java       |   2 +-
 external/storm-hive/pom.xml                     |   2 +-
 external/storm-jdbc/pom.xml                     |   2 +-
 external/storm-kafka/pom.xml                    |   2 +-
 .../src/jvm/storm/kafka/KafkaUtils.java         |  16 +-
 external/storm-redis/README.md                  | 108 ++++++++-
 external/storm-redis/pom.xml                    |   2 +-
 .../storm/redis/bolt/AbstractRedisBolt.java     |   8 +-
 .../storm/redis/bolt/RedisLookupBolt.java       | 112 +++++++++
 .../apache/storm/redis/bolt/RedisStoreBolt.java | 100 ++++++++
 .../redis/common/config/JedisClusterConfig.java |  82 +++++++
 .../redis/common/config/JedisPoolConfig.java    |  97 ++++++++
 .../common/container/JedisClusterContainer.java |  47 ++++
 .../JedisCommandsContainerBuilder.java          |  38 +++
 .../JedisCommandsInstanceContainer.java         |  25 ++
 .../redis/common/container/JedisContainer.java  |  65 ++++++
 .../common/mapper/RedisDataTypeDescription.java |  50 ++++
 .../redis/common/mapper/RedisLookupMapper.java  |  40 ++++
 .../storm/redis/common/mapper/RedisMapper.java  |  22 ++
 .../redis/common/mapper/RedisStoreMapper.java   |  21 ++
 .../storm/redis/common/mapper/TupleMapper.java  |  27 +++
 .../trident/mapper/TridentTupleMapper.java      |  27 ---
 .../trident/state/RedisClusterMapState.java     |   2 +-
 .../redis/trident/state/RedisClusterState.java  |   2 +-
 .../trident/state/RedisClusterStateQuerier.java |  10 +-
 .../trident/state/RedisClusterStateUpdater.java |  10 +-
 .../redis/trident/state/RedisMapState.java      |  21 +-
 .../storm/redis/trident/state/RedisState.java   |   2 +-
 .../redis/trident/state/RedisStateQuerier.java  |  10 +-
 .../state/RedisStateSetCountQuerier.java        |  74 ------
 .../trident/state/RedisStateSetUpdater.java     |  80 -------
 .../redis/trident/state/RedisStateUpdater.java  |  10 +-
 .../redis/util/config/JedisClusterConfig.java   |  82 -------
 .../redis/util/config/JedisPoolConfig.java      |  97 --------
 .../util/container/JedisClusterContainer.java   |  47 ----
 .../JedisCommandsContainerBuilder.java          |  38 ---
 .../JedisCommandsInstanceContainer.java         |  25 --
 .../redis/util/container/JedisContainer.java    |  65 ------
 .../storm/redis/topology/LookupWordCount.java   | 115 ++++++---
 .../redis/topology/PersistentWordCount.java     |  81 ++++---
 .../storm/redis/topology/WordCounter.java       |  19 +-
 .../redis/trident/WordCountTridentRedis.java    |   7 +-
 .../trident/WordCountTridentRedisCluster.java   |   6 +-
 .../WordCountTridentRedisClusterMap.java        |   8 +-
 .../redis/trident/WordCountTridentRedisMap.java |   9 +-
 .../redis/trident/WordCountTupleMapper.java     |  10 +-
 pom.xml                                         |   2 +-
 .../maven-shade-clojure-transformer/pom.xml     |   2 +-
 storm-buildtools/storm-maven-plugins/pom.xml    |   2 +-
 storm-core/pom.xml                              |   2 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  62 ++++-
 .../src/clj/backtype/storm/ui/helpers.clj       |  15 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   6 +
 .../jvm/backtype/storm/utils/NimbusClient.java  |   7 +
 storm-core/src/ui/public/css/style.css          |  62 +++++
 .../storm/security/auth/nimbus_auth_test.clj    | 231 ++++++++++---------
 storm-dist/binary/pom.xml                       |   2 +-
 storm-dist/source/pom.xml                       |   2 +-
 71 files changed, 1420 insertions(+), 851 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/604026f3/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/604026f3/storm-core/src/clj/backtype/storm/ui/helpers.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/helpers.clj
index e6cbb8c,6f5ca86..1eae3ac
--- a/storm-core/src/clj/backtype/storm/ui/helpers.clj
+++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj
@@@ -20,14 -20,16 +20,17 @@@
           [string :only [blank? join]]
           [walk :only [keywordize-keys]]])
    (:use [backtype.storm config log])
 -  (:use [backtype.storm.util :only [clojurify-structure uuid defnk url-encode]])
 +  (:use [backtype.storm.util :only [clojurify-structure uuid defnk url-encode not-nil?]])
    (:use [clj-time coerce format])
    (:import [backtype.storm.generated ExecutorInfo ExecutorSummary])
+   (:import [java.util EnumSet])
    (:import [org.eclipse.jetty.server Server]
             [org.eclipse.jetty.server.nio SelectChannelConnector]
             [org.eclipse.jetty.server.ssl SslSocketConnector]
             [org.eclipse.jetty.servlet ServletHolder FilterMapping]
-            [org.eclipse.jetty.util.ssl SslContextFactory])
++	   [org.eclipse.jetty.util.ssl SslContextFactory]
+            [org.eclipse.jetty.server DispatcherType]
+            [org.eclipse.jetty.servlets CrossOriginFilter])
    (:require [ring.util servlet])
    (:require [compojure.route :as route]
              [compojure.handler :as handler]))
@@@ -133,23 -135,30 +136,32 @@@
  (defn unauthorized-user-html [user]
    [[:h2 "User '" (escape-html user) "' is not authorized."]])
  
 -(defn- mk-ssl-connector [port ks-path ks-password ks-type]
 -  (doto (SslSocketConnector.)
 -    (.setExcludeCipherSuites (into-array String ["SSL_RSA_WITH_RC4_128_MD5" "SSL_RSA_WITH_RC4_128_SHA"]))
 -    (.setExcludeProtocols (into-array String ["SSLv3"]))
 -    (.setAllowRenegotiate false)
 -    (.setKeystore ks-path)
 -    (.setKeystoreType ks-type)
 -    (.setKeyPassword ks-password)
 -    (.setPassword ks-password)
 -    (.setPort port)))
 -
 -(defn config-ssl [server port ks-path ks-password ks-type]
 +(defn- mk-ssl-connector [port ks-path ks-password ks-type key-password]
 +  (let [sslContextFactory (doto (SslContextFactory.)
 +                            (.setExcludeCipherSuites (into-array String ["SSL_RSA_WITH_RC4_128_MD5" "SSL_RSA_WITH_RC4_128_SHA"]))
 +                            (.setExcludeProtocols (into-array String ["SSLv3"]))
 +                            (.setAllowRenegotiate false)
 +                            (.setKeyStorePath ks-path)
 +                            (.setKeyStoreType ks-type)
 +                            (.setKeyStorePassword ks-password)
 +                            (.setKeyManagerPassword key-password))]
 +    (doto (SslSocketConnector. sslContextFactory)
 +      (.setPort port))))
 +
 +
 +(defn config-ssl [server port ks-path ks-password ks-type key-password]
    (when (> port 0)
 -    (.addConnector server (mk-ssl-connector port ks-path ks-password ks-type))))
 +    (.addConnector server (mk-ssl-connector port ks-path ks-password ks-type key-password))))
  
+ (defn cors-filter-handler
+   []
+   (doto (org.eclipse.jetty.servlet.FilterHolder. (CrossOriginFilter.))
+     (.setInitParameter CrossOriginFilter/ALLOWED_ORIGINS_PARAM "*")
+     (.setInitParameter CrossOriginFilter/ALLOWED_METHODS_PARAM "GET, POST, PUT")
+     (.setInitParameter CrossOriginFilter/ALLOWED_HEADERS_PARAM "X-Requested-With, X-Requested-By, Access-Control-Allow-Origin, Content-Type, Content-Length, Accept, Origin")
+     (.setInitParameter CrossOriginFilter/ACCESS_CONTROL_ALLOW_ORIGIN_HEADER "*")
+     ))
+ 
  (defn config-filter [server handler filters-confs]
    (if filters-confs
      (let [servlet-holder (ServletHolder.

http://git-wip-us.apache.org/repos/asf/storm/blob/604026f3/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------


[3/6] storm git commit: STORM-721.Storm UI server should support SSL. Added client side auth configs.

Posted by sr...@apache.org.
STORM-721.Storm UI server should support SSL. Added client side auth configs.


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

Branch: refs/heads/master
Commit: cc2520a07c2c5005e14dddd21663d14dc3867bc2
Parents: 604026f
Author: Sriharsha Chintalapani <ma...@harsha.io>
Authored: Wed Apr 1 13:54:44 2015 -0700
Committer: Sriharsha Chintalapani <ma...@harsha.io>
Committed: Wed Apr 1 13:54:44 2015 -0700

----------------------------------------------------------------------
 SECURITY.md                                     | 27 ++++++++++
 .../src/clj/backtype/storm/daemon/drpc.clj      | 14 ++++-
 storm-core/src/clj/backtype/storm/ui/core.clj   | 14 ++++-
 .../src/clj/backtype/storm/ui/helpers.clj       | 15 ++++--
 storm-core/src/jvm/backtype/storm/Config.java   | 55 ++++++++++++++++++++
 5 files changed, 118 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/cc2520a0/SECURITY.md
----------------------------------------------------------------------
diff --git a/SECURITY.md b/SECURITY.md
index 6b73254..7133003 100644
--- a/SECURITY.md
+++ b/SECURITY.md
@@ -34,6 +34,7 @@ IPsec to encrypt all traffic being sent between the hosts in the cluster.
 | 3774 | `drpc.http.port` | External HTTP DRPC Clients | DRPC |
 | 670{0,1,2,3} | `supervisor.slots.ports` | Worker Processes | Worker Processes |
 
+
 ### UI/Logviewer
 
 The UI and logviewer processes provide a way to not only see what a cluster is
@@ -78,6 +79,32 @@ curl  -i --negotiate -u:anyUser  -b ~/cookiejar.txt -c ~/cookiejar.txt  http://s
 
 **Caution**: In AD MIT Keberos setup the key size is bigger than the default UI jetty server request header size. Make sure you set ui.header.buffer.bytes to 65536 in storm.yaml. More details are on [STORM-633](https://issues.apache.org/jira/browse/STORM-633)
 
+
+## UI / DRPC SSL 
+
+Both UI and DRPC allows users to configure ssl .
+
+### UI
+
+For UI users needs to set following config in storm.yaml. Generating keystores with proper keys and certs should be taken care by the user before this step.
+
+1. ui.https.port 
+2. ui.https.keystore.type (example "jks")
+3. ui.https.keystore.path (example "/etc/ssl/storm_keystore.jks")
+4. ui.https.keystore.password (keystore password)
+5. ui.https.key.password (private key password)
+
+### DRPC
+similarly to UI users need to configure following 
+
+1. drpc.https.port 
+2. drpc.https.keystore.type (example "jks")
+3. drpc.https.keystore.path (example "/etc/ssl/storm_keystore.jks")
+4. drpc.https.keystore.password (keystore password)
+5. drpc.https.key.password (private key password)
+
+
+
 ## Authentication (Kerberos)
 
 Storm offers pluggable authentication support through thrift and SASL.  This

http://git-wip-us.apache.org/repos/asf/storm/blob/cc2520a0/storm-core/src/clj/backtype/storm/daemon/drpc.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
index 2ab3cce..eae16a1 100644
--- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
@@ -226,7 +226,12 @@
               https-ks-path (conf DRPC-HTTPS-KEYSTORE-PATH)
               https-ks-password (conf DRPC-HTTPS-KEYSTORE-PASSWORD)
               https-ks-type (conf DRPC-HTTPS-KEYSTORE-TYPE)
-              https-key-password (conf DRPC-HTTPS-KEY-PASSWORD)]
+              https-key-password (conf DRPC-HTTPS-KEY-PASSWORD)
+              https-ts-path (conf DRPC-HTTPS-TRUSTSTORE-PATH)
+              https-ts-password (conf DRPC-HTTPS-TRUSTSTORE-PASSWORD)
+              https-ts-type (conf DRPC-HTTPS-TRUSTSTORE-TYPE)
+              https-want-client-auth (conf DRPC-HTTPS-WANT-CLIENT-AUTH)
+              https-need-client-auth (conf DRPC-HTTPS-NEED-CLIENT-AUTH)]
 
           (storm-run-jetty
            {:port drpc-http-port
@@ -236,7 +241,12 @@
                                         https-ks-path
                                         https-ks-password
                                         https-ks-type
-                                        https-key-password)
+                                        https-key-password
+                                        https-ts-path
+                                        https-ts-password
+                                        https-ts-type
+                                        https-need-client-auth
+                                        https-want-client-auth)
                             (config-filter server app filters-confs))})))
       (when handler-server
         (.serve handler-server)))))

http://git-wip-us.apache.org/repos/asf/storm/blob/cc2520a0/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index 42cb920..23cde65 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -1083,7 +1083,12 @@
           https-ks-path (conf UI-HTTPS-KEYSTORE-PATH)
           https-ks-password (conf UI-HTTPS-KEYSTORE-PASSWORD)
           https-ks-type (conf UI-HTTPS-KEYSTORE-TYPE)
-          https-key-password (conf UI-HTTPS-KEY-PASSWORD)]
+          https-key-password (conf UI-HTTPS-KEY-PASSWORD)
+          https-ts-path (conf UI-HTTPS-TRUSTSTORE-PATH)
+          https-ts-password (conf UI-HTTPS-TRUSTSTORE-PASSWORD)
+          https-ts-type (conf UI-HTTPS-TRUSTSTORE-TYPE)
+          https-want-client-auth (conf UI-HTTPS-WANT-CLIENT-AUTH)
+          https-need-client-auth (conf UI-HTTPS-NEED-CLIENT-AUTH)]
       (storm-run-jetty {:port (conf UI-PORT)
                         :host (conf UI-HOST)
                         :https-port https-port
@@ -1093,7 +1098,12 @@
                                                     https-ks-path
                                                     https-ks-password
                                                     https-ks-type
-                                                    https-key-password)
+                                                    https-key-password
+                                                    https-ts-path
+                                                    https-ts-password
+                                                    https-ts-type
+                                                    https-need-client-auth
+                                                    https-want-client-auth)
                                         (doseq [connector (.getConnectors server)]
                                           (.setRequestHeaderSize connector header-buffer-size))
                                         (config-filter server app filters-confs))}))

http://git-wip-us.apache.org/repos/asf/storm/blob/cc2520a0/storm-core/src/clj/backtype/storm/ui/helpers.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj
index 1eae3ac..70ef179 100644
--- a/storm-core/src/clj/backtype/storm/ui/helpers.clj
+++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj
@@ -136,7 +136,8 @@
 (defn unauthorized-user-html [user]
   [[:h2 "User '" (escape-html user) "' is not authorized."]])
 
-(defn- mk-ssl-connector [port ks-path ks-password ks-type key-password]
+(defn- mk-ssl-connector [port ks-path ks-password ks-type key-password
+                         ts-path ts-password ts-type need-client-auth want-client-auth]
   (let [sslContextFactory (doto (SslContextFactory.)
                             (.setExcludeCipherSuites (into-array String ["SSL_RSA_WITH_RC4_128_MD5" "SSL_RSA_WITH_RC4_128_SHA"]))
                             (.setExcludeProtocols (into-array String ["SSLv3"]))
@@ -145,13 +146,21 @@
                             (.setKeyStoreType ks-type)
                             (.setKeyStorePassword ks-password)
                             (.setKeyManagerPassword key-password))]
+    (if (and (not-nil? ts-path) (not-nil? ts-password) (not-nil? ts-type))
+      ((.setTrustStore sslContextFactory ts-path)
+       (.setTrustStoreType sslContextFactory ts-type)
+       (.setTrustStoreType sslContextFactory ts-password)))
+    (if (need-client-auth) (.setNeedClientAuth sslContextFactory true)
+        (if (want-client-auth) (.setWantClientAuth sslContextFactory true)))
     (doto (SslSocketConnector. sslContextFactory)
       (.setPort port))))
 
 
-(defn config-ssl [server port ks-path ks-password ks-type key-password]
+(defn config-ssl [server port ks-path ks-password ks-type key-password
+                  ts-path ts-password ts-type need-client-auth want-client-auth]
   (when (> port 0)
-    (.addConnector server (mk-ssl-connector port ks-path ks-password ks-type key-password))))
+    (.addConnector server (mk-ssl-connector port ks-path ks-password ks-type key-password
+                                            ts-path ts-password ts-type need-client-auth want-client-auth))))
 
 (defn cors-filter-handler
   []

http://git-wip-us.apache.org/repos/asf/storm/blob/cc2520a0/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index a1bc419..d7f715b 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -597,6 +597,33 @@ public class Config extends HashMap<String, Object> {
     public static final String UI_HTTPS_KEY_PASSWORD = "ui.https.key.password";
     public static final Object UI_HTTPS_KEY_PASSWORD_SCHEMA = String.class;
 
+    /**
+     * Path to the truststore used by Storm UI settting up HTTPS (SSL).
+     */
+    public static final String UI_HTTPS_TRUSTSTORE_PATH = "ui.https.truststore.path";
+    public static final Object UI_HTTPS_TRUSTSTORE_PATH_SCHEMA = String.class;
+
+    /**
+     * Password to the truststore used by Storm UI settting up HTTPS (SSL).
+     */
+    public static final String UI_HTTPS_TRUSTSTORE_PASSWORD = "ui.https.truststore.password";
+    public static final Object UI_HTTPS_TRUSTSTORE_PASSWORD_SCHEMA = String.class;
+
+    /**
+     * Type of truststore used by Storm UI for setting up HTTPS (SSL).
+     * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details.
+     */
+    public static final String UI_HTTPS_TRUSTSTORE_TYPE = "ui.https.truststore.type";
+    public static final Object UI_HTTPS_TRUSTSTORE_TYPE_SCHEMA = String.class;
+
+    /**
+     * Password to the truststore used by Storm DRPC settting up HTTPS (SSL).
+     */
+    public static final String UI_HTTPS_WANT_CLIENT_AUTH = "ui.https.want.client.auth";
+    public static final Object UI_HTTPS_WANT_CLIENT_AUTH_SCHEMA = Boolean.class;
+
+    public static final String UI_HTTPS_NEED_CLIENT_AUTH = "ui.https.need.client.auth";
+    public static final Object UI_HTTPS_NEED_CLIENT_AUTH_SCHEMA = Boolean.class;
 
 
     /**
@@ -643,6 +670,34 @@ public class Config extends HashMap<String, Object> {
     public static final Object DRPC_HTTPS_KEY_PASSWORD_SCHEMA = String.class;
 
     /**
+     * Path to the truststore used by Storm DRPC settting up HTTPS (SSL).
+     */
+    public static final String DRPC_HTTPS_TRUSTSTORE_PATH = "drpc.https.truststore.path";
+    public static final Object DRPC_HTTPS_TRUSTSTORE_PATH_SCHEMA = String.class;
+
+    /**
+     * Password to the truststore used by Storm DRPC settting up HTTPS (SSL).
+     */
+    public static final String DRPC_HTTPS_TRUSTSTORE_PASSWORD = "drpc.https.truststore.password";
+    public static final Object DRPC_HTTPS_TRUSTSTORE_PASSWORD_SCHEMA = String.class;
+
+    /**
+     * Type of truststore used by Storm DRPC for setting up HTTPS (SSL).
+     * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details.
+     */
+    public static final String DRPC_HTTPS_TRUSTSTORE_TYPE = "drpc.https.truststore.type";
+    public static final Object DRPC_HTTPS_TRUSTSTORE_TYPE_SCHEMA = String.class;
+
+    /**
+     * Password to the truststore used by Storm DRPC settting up HTTPS (SSL).
+     */
+    public static final String DRPC_HTTPS_WANT_CLIENT_AUTH = "drpc.https.want.client.auth";
+    public static final Object DRPC_HTTPS_WANT_CLIENT_AUTH_SCHEMA = Boolean.class;
+
+    public static final String DRPC_HTTPS_NEED_CLIENT_AUTH = "drpc.https.need.client.auth";
+    public static final Object DRPC_HTTPS_NEED_CLIENT_AUTH_SCHEMA = Boolean.class;
+
+    /**
      * The DRPC transport plug-in for Thrift client/server communication
      */
     public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport";