From fe5f41aa8332700d3f98422cb7d986fc47289bcd Mon Sep 17 00:00:00 2001
From: "Robert (Bobby) Evans"
Date: Wed, 21 May 2014 11:03:11 -0500
Subject: [PATCH 01/61] STORM-216: Added Authentication and Authorization.
This is an upmerged version of https://github.com/yahoo/incubator-storm/tree/security
---
.gitignore | 2 +-
SECURITY.md | 327 +-
bin/storm | 15 +-
conf/defaults.yaml | 36 +
conf/jaas_digest.conf | 8 +-
conf/jaas_kerberos.conf | 15 +
pom.xml | 20 +
storm-core/pom.xml | 229 +-
.../src/clj/backtype/storm/LocalCluster.clj | 3 +
.../src/clj/backtype/storm/LocalDRPC.clj | 4 +-
.../src/clj/backtype/storm/bootstrap.clj | 5 +-
storm-core/src/clj/backtype/storm/cluster.clj | 96 +-
.../storm/command/upload_credentials.clj | 35 +
storm-core/src/clj/backtype/storm/config.clj | 41 +-
.../src/clj/backtype/storm/daemon/common.clj | 18 +-
.../src/clj/backtype/storm/daemon/drpc.clj | 164 +-
.../clj/backtype/storm/daemon/executor.clj | 116 +-
.../clj/backtype/storm/daemon/logviewer.clj | 377 ++-
.../src/clj/backtype/storm/daemon/nimbus.clj | 237 +-
.../clj/backtype/storm/daemon/supervisor.clj | 190 +-
.../src/clj/backtype/storm/daemon/task.clj | 7 +-
.../src/clj/backtype/storm/daemon/worker.clj | 135 +-
storm-core/src/clj/backtype/storm/testing.clj | 27 +-
.../src/clj/backtype/storm/testing4j.clj | 4 +-
storm-core/src/clj/backtype/storm/thrift.clj | 13 +-
storm-core/src/clj/backtype/storm/timer.clj | 62 +-
storm-core/src/clj/backtype/storm/ui/core.clj | 338 +-
.../src/clj/backtype/storm/ui/helpers.clj | 45 +-
storm-core/src/clj/backtype/storm/util.clj | 84 +-
.../src/clj/backtype/storm/zookeeper.clj | 19 +-
storm-core/src/clj/storm/trident/testing.clj | 1 +
.../dev/drpc-simple-acl-test-scenario.yaml | 11 +
storm-core/src/jvm/backtype/storm/Config.java | 368 +-
.../jvm/backtype/storm/ConfigValidation.java | 156 +-
.../src/jvm/backtype/storm/Constants.java | 3 +-
.../backtype/storm/ICredentialsListener.java | 32 +
.../src/jvm/backtype/storm/ILocalCluster.java | 2 +
.../jvm/backtype/storm/StormSubmitter.java | 161 +-
.../storm/drpc/DRPCInvocationsClient.java | 91 +-
.../jvm/backtype/storm/drpc/DRPCSpout.java | 100 +-
.../backtype/storm/drpc/ReturnResults.java | 35 +-
.../generated/AuthorizationException.java | 328 ++
.../backtype/storm/generated/Credentials.java | 373 ++
.../storm/generated/DistributedRPC.java | 110 +-
.../generated/DistributedRPCInvocations.java | 352 +-
.../jvm/backtype/storm/generated/Nimbus.java | 3006 ++++++++++++++++-
.../storm/generated/SubmitOptions.java | 98 +-
.../storm/generated/TopologyInfo.java | 192 +-
.../storm/generated/TopologySummary.java | 192 +-
.../jvm/backtype/storm/scheduler/Cluster.java | 13 +
.../scheduler/multitenant/DefaultPool.java | 219 ++
.../storm/scheduler/multitenant/FreePool.java | 125 +
.../scheduler/multitenant/IsolatedPool.java | 346 ++
.../multitenant/MultitenantScheduler.java | 98 +
.../storm/scheduler/multitenant/Node.java | 322 ++
.../storm/scheduler/multitenant/NodePool.java | 296 ++
.../storm/security/auth/AuthUtils.java | 165 +-
.../auth/DefaultHttpCredentialsPlugin.java | 83 +
.../auth/DefaultPrincipalToLocal.java | 43 +
.../storm/security/auth/IAutoCredentials.java | 55 +
.../security/auth/ICredentialsRenewer.java | 40 +
.../security/auth/IHttpCredentialsPlugin.java | 50 +
.../security/auth/IPrincipalToLocal.java | 41 +
.../storm/security/auth/ITransportPlugin.java | 14 +-
.../auth/KerberosPrincipalToLocal.java | 45 +
.../storm/security/auth/ReqContext.java | 10 +-
.../security/auth/SaslTransportPlugin.java | 45 +-
.../security/auth/SimpleTransportPlugin.java | 61 +-
.../security/auth/SingleUserPrincipal.java | 56 +
.../storm/security/auth/TBackoffConnect.java | 77 +
.../storm/security/auth/ThriftClient.java | 85 +-
.../security/auth/ThriftConnectionType.java | 77 +
.../storm/security/auth/ThriftServer.java | 19 +-
.../auth/authorizer/DRPCAuthorizerBase.java | 46 +
.../authorizer/DRPCSimpleACLAuthorizer.java | 157 +
.../auth/authorizer/DenyAuthorizer.java | 4 +-
.../auth/authorizer/NoopAuthorizer.java | 6 +-
.../auth/authorizer/SimpleACLAuthorizer.java | 113 +
.../authorizer/SimpleWhitelistAuthorizer.java | 70 +
.../digest/DigestSaslTransportPlugin.java | 1 +
.../storm/security/auth/kerberos/AutoTGT.java | 267 ++
.../auth/kerberos/AutoTGTKrb5LoginModule.java | 108 +
.../kerberos/AutoTGTKrb5LoginModuleTest.java | 44 +
.../auth/kerberos/ClientCallbackHandler.java | 104 +
.../kerberos/KerberosSaslTransportPlugin.java | 206 ++
.../auth/kerberos/ServerCallbackHandler.java | 86 +
.../auth/kerberos/jaas_kerberos_cluster.conf | 31 +
.../auth/kerberos/jaas_kerberos_launcher.conf | 12 +
.../testing/SingleUserSimpleTransport.java | 37 +
.../state/TestTransactionalState.java | 47 +
.../state/TransactionalState.java | 56 +-
.../storm/ui/InvalidRequestException.java | 20 +
.../jvm/backtype/storm/utils/DRPCClient.java | 63 +-
.../jvm/backtype/storm/utils/LocalState.java | 15 +-
.../backtype/storm/utils/NimbusClient.java | 11 +-
.../jvm/backtype/storm/utils/TestUtils.java | 34 +
.../src/jvm/backtype/storm/utils/Utils.java | 126 +-
.../storm/utils/ZookeeperAuthInfo.java | 9 +-
.../utils/ZookeeperServerCnxnFactory.java | 84 +
.../trident/drpc/ReturnResultsReducer.java | 13 +-
.../state/TestTransactionalState.java | 47 +
.../topology/state/TransactionalState.java | 58 +-
.../src/native/worker-launcher/.autom4te.cfg | 42 +
.../worker-launcher/.deps/worker-launcher.Po | 1 +
.../src/native/worker-launcher/Makefile.am | 32 +
.../src/native/worker-launcher/configure.ac | 50 +
.../worker-launcher/impl/configuration.c | 340 ++
.../worker-launcher/impl/configuration.h | 45 +
.../src/native/worker-launcher/impl/main.c | 210 ++
.../worker-launcher/impl/worker-launcher.c | 779 +++++
.../worker-launcher/impl/worker-launcher.h | 129 +
.../test/test-worker-launcher.c | 340 ++
storm-core/src/py/storm/DistributedRPC.py | 21 +-
.../src/py/storm/DistributedRPCInvocations.py | 80 +-
storm-core/src/py/storm/Nimbus-remote | 7 +
storm-core/src/py/storm/Nimbus.py | 636 +++-
storm-core/src/py/storm/ttypes.py | 1227 ++++++-
storm-core/src/storm.thrift | 58 +-
storm-core/src/ui/public/css/style.css | 6 +
.../test/clj/backtype/storm/cluster_test.clj | 91 +-
.../test/clj/backtype/storm/config_test.clj | 11 +
.../clj/backtype/storm/logviewer_test.clj | 187 +
.../messaging/netty_integration_test.clj | 3 +-
.../test/clj/backtype/storm/nimbus_test.clj | 340 +-
.../scheduler/multitenant_scheduler_test.clj | 737 ++++
.../storm/security/auth/AuthUtils_test.clj | 16 +-
.../DefaultHttpCredentialsPlugin_test.clj | 40 +
.../storm/security/auth/ThriftClient_test.clj | 28 +-
.../storm/security/auth/ThriftServer_test.clj | 8 +-
.../storm/security/auth/auth_test.clj | 363 +-
.../DRPCSimpleAclAuthorizer_test.clj | 226 ++
.../security/auth/auto_login_module_test.clj | 91 +
.../storm/security/auth/drpc-auth-alice.jaas | 5 +
.../storm/security/auth/drpc-auth-bob.jaas | 5 +
.../security/auth/drpc-auth-charlie.jaas | 5 +
.../storm/security/auth/drpc-auth-server.jaas | 6 +
.../storm/security/auth/drpc_auth_test.clj | 315 ++
.../storm/security/auth/nimbus_auth_test.clj | 181 +
.../clj/backtype/storm/submitter_test.clj | 75 +
.../clj/backtype/storm/supervisor_test.clj | 248 +-
.../clj/backtype/storm/transactional_test.clj | 27 +-
.../test/clj/backtype/storm/ui_test.clj | 49 +
.../utils/ZookeeperServerCnxnFactory_test.clj | 35 +
.../test/clj/backtype/storm/utils_test.clj | 58 +-
.../test/clj/storm/trident/state_test.clj | 25 +-
145 files changed, 18509 insertions(+), 1212 deletions(-)
create mode 100644 conf/jaas_kerberos.conf
create mode 100644 storm-core/src/clj/backtype/storm/command/upload_credentials.clj
create mode 100644 storm-core/src/dev/drpc-simple-acl-test-scenario.yaml
create mode 100644 storm-core/src/jvm/backtype/storm/ICredentialsListener.java
create mode 100644 storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
create mode 100644 storm-core/src/jvm/backtype/storm/generated/Credentials.java
create mode 100644 storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java
create mode 100644 storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java
create mode 100644 storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java
create mode 100644 storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java
create mode 100644 storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java
create mode 100644 storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/IAutoCredentials.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/ThriftConnectionType.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf
create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf
create mode 100644 storm-core/src/jvm/backtype/storm/testing/SingleUserSimpleTransport.java
create mode 100644 storm-core/src/jvm/backtype/storm/transactional/state/TestTransactionalState.java
create mode 100644 storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java
create mode 100644 storm-core/src/jvm/backtype/storm/utils/TestUtils.java
create mode 100644 storm-core/src/jvm/backtype/storm/utils/ZookeeperServerCnxnFactory.java
create mode 100644 storm-core/src/jvm/storm/trident/topology/state/TestTransactionalState.java
create mode 100644 storm-core/src/native/worker-launcher/.autom4te.cfg
create mode 100644 storm-core/src/native/worker-launcher/.deps/worker-launcher.Po
create mode 100644 storm-core/src/native/worker-launcher/Makefile.am
create mode 100644 storm-core/src/native/worker-launcher/configure.ac
create mode 100644 storm-core/src/native/worker-launcher/impl/configuration.c
create mode 100644 storm-core/src/native/worker-launcher/impl/configuration.h
create mode 100644 storm-core/src/native/worker-launcher/impl/main.c
create mode 100644 storm-core/src/native/worker-launcher/impl/worker-launcher.c
create mode 100644 storm-core/src/native/worker-launcher/impl/worker-launcher.h
create mode 100644 storm-core/src/native/worker-launcher/test/test-worker-launcher.c
create mode 100644 storm-core/test/clj/backtype/storm/logviewer_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/auto_login_module_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/drpc-auth-server.jaas
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/drpc_auth_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/submitter_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/ui_test.clj
create mode 100644 storm-core/test/clj/backtype/storm/utils/ZookeeperServerCnxnFactory_test.clj
diff --git a/.gitignore b/.gitignore
index b2a37f9c999..f7448d996ba 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,4 +27,4 @@ target
/.lein-plugins/
*.ipr
*.iws
-.idea
\ No newline at end of file
+.idea
diff --git a/SECURITY.md b/SECURITY.md
index 93036b2595b..882954a768f 100644
--- a/SECURITY.md
+++ b/SECURITY.md
@@ -1,16 +1,19 @@
# Running Apache Storm Securely
-The current release of Apache Storm offers no authentication or authorization.
-It does not encrypt any data being sent across the network, and does not
-attempt to restrict access to data stored on the local file system or in
-Apache Zookeeper. As such there are a number of different precautions you may
-want to enact outside of storm itself to be sure storm is running securely.
+Apache Storm offers a range of configuration options when trying to secure
+your cluster. By default all authentication and authorization is disabled but
+can be turned on as needed.
+
+## Firewall/OS level Security
+
+You can still have a secure storm cluster without turning on formal
+Authentication and Authorization. But to do so usually requires
+configuring your Operating System to ristrict the operations that can be done.
+This is generally a good idea even if you plan on running your cluster with Auth.
The exact detail of how to setup these precautions varies a lot and is beyond
the scope of this document.
-## Network Security
-
It is generally a good idea to enable a firewall and restrict incoming network
connections to only those originating from the cluster itself and from trusted
hosts and services, a complete list of ports storm uses are below.
@@ -34,41 +37,305 @@ IPsec to encrypt all traffic being sent between the hosts in the cluster.
The UI and logviewer processes provide a way to not only see what a cluster is
doing, but also manipulate running topologies. In general these processes should
-not be exposed except to users of the cluster. It is often simplest to restrict
-these ports to only accept connections from local hosts, and then front them with another web server,
-like Apache httpd, that can authenticate/authorize incoming connections and
+not be exposed except to users of the cluster.
+
+Some form of Authentication is typically required, with using java servlet filters
+
+```yaml
+ui.filter: "filter.class"
+ui.filter.params: "param1":"value1"
+```
+or by restricting the UI/log viewers ports to only accept connections from local
+hosts, and then front them with another web server, like Apache httpd, that can
+authenticate/authorize incoming connections and
proxy the connection to the storm process. To make this work the ui process must have
logviewer.port set to the port of the proxy in its storm.yaml, while the logviewers
must have it set to the actual port that they are going to bind to.
-### Nimbus
+The servlet filters are prefered because it allows indavidual topologies to
+specificy who is and who is not allowed to access the pages associated with
+them.
-Nimbus's Thrift port should be locked down as it can be used to control the entire
-cluster including running arbitrary user code on different nodes in the cluster.
-Ideally access to it is restricted to nodes within the cluster and possibly some gateway
-nodes that allow authorized users to log into them and run storm client commands.
+## Authentication (Kerberos)
-### DRPC
+Storm offers pluggable authentication support through thrift and SASL. This
+example only goes off of Kerberos as it is a common setup for most big data
+projects.
+
+Setting up a KDC and configuring kerberos on each node is beyond the scope of
+this document and it is assumed that you have done that already.
+
+### Create Headless Principals and keytabs
+
+Each Zookeeper Server, Nimbus, and DRPC server will need a service principal, which, by convention, includes the FQDN of the host it will run on. Be aware that the zookeeper user *MUST* be zookeeper.
+The supervisors and UI also need a principal to run as, but because they are outgoing connections they do not need to be service principals.
+The following is an example of how to setup kerberos principals, but the
+details may varry depending on your KDC and OS.
+
+
+```bash
+# Zookeeper (Will need one of these for each box in teh Zk ensamble)
+sudo kadmin.local -q 'addprinc zookeeper/zk1.example.com@STORM.EXAMPLE.COM'
+sudo kadmin.local -q "ktadd -k /tmp/zk.keytab zookeeper/zk1.example.com@STORM.EXAMPLE.COM"
+# Nimbus and DRPC
+sudo kadmin.local -q 'addprinc storm/storm.example.com@STORM.EXAMPLE.COM'
+sudo kadmin.local -q "ktadd -k /tmp/storm.keytab storm/storm.example.com@STORM.EXAMPLE.COM"
+# All UI logviewer and Supervisors
+sudo kadmin.local -q 'addprinc storm@STORM.EXAMPLE.COM'
+sudo kadmin.local -q "ktadd -k /tmp/storm.keytab storm@STORM.EXAMPLE.COM"
+```
+
+be sure to distribute the keytab(s) to the appropriate boxes and set the FS permissions so that only the headless user running ZK, or storm has access to them.
+
+#### Storm Kerberos Configuration
+
+Both storm and Zookeeper use jaas configuration files to log the user in.
+Each jaas file may have multiple sections for different interfaces being used.
+
+To enable Kerberos authentication in storm you need to set the following storm.yaml configs
+```yaml
+storm.thrift.transport: "backtype.storm.security.auth.kerberos.KerberosSaslTransportPlugin"
+java.security.auth.login.config: "/path/to/jaas.conf"
+```
+
+Nimbus and the supervisor processes will also connect to ZooKeeper(ZK) and we want to configure them to use Kerberos for authentication with ZK. To do this append
+```
+-Djava.security.auth.login.config=/path/to/jaas.conf
+```
+
+to the childopts of nimbus, ui, and supervisor. Here is an example given the default childopts settings at the time of writing:
+
+```yaml
+nimbus.childopts: "-Xmx1024m -Djava.security.auth.login.config=/path/to/jaas.conf"
+ui.childopts: "-Xmx768m -Djava.security.auth.login.config=/path/to/jaas.conf"
+supervisor.childopts: "-Xmx256m -Djava.security.auth.login.config=/path/to/jaas.conf"
+```
+
+The jaas.conf file should look something like the following for the storm nodes.
+The StormServer section is used by nimbus and the DRPC Nodes. It does not need to be included on supervisor nodes.
+The StormClient section is used by all storm clients that want to talk to nimbus, including the ui, logviewer, and supervisor. We will use this section on the gateways as well but the structure of that will be a bit different.
+The Client section is used by processes wanting to talk to zookeeper and really only needs to be included with nimbus and the supervisors.
+The Server section is used by the zookeeper servers.
+Having unused sections in the jaas is not a problem.
+
+```
+StormServer {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="$keytab"
+ storeKey=true
+ useTicketCache=false
+ principal="$principal";
+};
+StormClient {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="$keytab"
+ storeKey=true
+ useTicketCache=false
+ serviceName="$nimbus_user"
+ principal="$principal";
+};
+Client {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="$keytab"
+ storeKey=true
+ useTicketCache=false
+ serviceName="zookeeper"
+ principal="$principal";
+};
+Server {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="$keytab"
+ storeKey=true
+ useTicketCache=false
+ principal="$principal";
+};
+```
+
+The following is an example based off of the keytabs generated
+```
+StormServer {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="/keytabs/storm.keytab"
+ storeKey=true
+ useTicketCache=false
+ principal="storm/storm.example.com@STORM.EXAMPLE.COM";
+};
+StormClient {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="/keytabs/storm.keytab"
+ storeKey=true
+ useTicketCache=false
+ serviceName="storm"
+ principal="storm@STORM.EXAMPLE.COM";
+};
+Client {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="/keytabs/storm.keytab"
+ storeKey=true
+ useTicketCache=false
+ serviceName="zookeeper"
+ principal="storm@STORM.EXAMPLE.COM";
+};
+Server {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="/keytabs/zk.keytab"
+ storeKey=true
+ useTicketCache=false
+ serviceName="zookeeper"
+ principal="zookeeper/zk1.example.com@STORM.EXAMPLE.COM";
+};
+```
+
+Nimbus also will translate the principal into a local user name, so that other services can use this name. To configure this for Kerberos authentication set
+
+```
+storm.principal.tolocal: "backtype.storm.security.auth.KerberosPrincipalToLocal"
+```
+
+This only needs to be done on nimbus, but it will not hurt on any node.
+We also need to inform the topology who the supervisor daemon and the nimbus daemon are running as from a ZooKeeper perspective.
+
+```
+storm.zookeeper.superACL: "sasl:${nimbus-user}"
+```
+
+Here *nimbus-user* is the Kerberos user that nimbus uses to authenticate with ZooKeeper. If ZooKeeeper is stripping host and realm then this needs to have host and realm stripped too.
+
+#### ZooKeeper Ensemble
+
+Complete details of how to setup a secure ZK are beyond the scope of this document. But in general you want to enable SASL authentication on each server, and optionally strip off host and realm
+
+```
+authProvider.1 = org.apache.zookeeper.server.auth.SASLAuthenticationProvider
+kerberos.removeHostFromPrincipal = true
+kerberos.removeRealmFromPrincipal = true
+```
+
+And you want to include the jaas.conf on the command line when launching the server so it can use it can find the keytab.
+```
+-Djava.security.auth.login.config=/jaas/zk_jaas.conf
+```
+
+#### Gateways
-Each DRPC server has two different ports. The invocations port is accessed by worker
-processes within the cluster. The other port is accessed by external clients that
-want to query the topology. The external port should be restricted to hosts that you
-want to be able to do queries.
+Ideally the end user will only need to run kinit before interacting with storm. To make this happen seamlessly we need the default jaas.conf on the gateways to be something like
-### Supervisors
+```
+StormClient {
+ com.sun.security.auth.module.Krb5LoginModule required
+ doNotPrompt=false
+ useTicketCache=true
+ serviceName="$nimbus_user";
+};
+```
-Supervisors are only clients they are not servers, and as such don't need special restrictions.
+The end user can override this if they have a headless user that has a keytab.
-### Workers
+### Authorization Setup
-Worker processes receive data from each other. There is the option to encrypt this data using
-Blowfish by setting `topology.tuple.serializer` to `backtype.storm.security.serialization.BlowfishTupleSerializer`
-and setting `topology.tuple.serializer.blowfish.key` to a secret key you want your topology to use.
+*Authentication* does the job of verifying who the user is, but we also need *authorization* to do the job of enforcing what each user can do.
+
+The preferred authorization plug-in for nimbus is The *SimpleACLAuthorizer*. To use the *SimpleACLAuthorizer*, set the following:
+
+```yaml
+nimbus.authorizer: "backtype.storm.security.auth.authorizer.SimpleACLAuthorizer"
+```
+
+DRPC has a separate authorizer configuration for it. Do not use SimpleACLAuthorizer for DRPC.
+
+The *SimpleACLAuthorizer* plug-in needs to know who the supervisor users are, and it needs to know about all of the administrator users, including the user running the ui daemon.
+
+These are set through *nimbus.supervisor.users* and *nimbus.admins* respectively. Each can either be a full Kerberos principal name, or the name of the user with host and realm stripped off.
+
+The UI and Log servers have their own authorization configurations. These are set through *logs.users* and *ui.users*. These should be set to the admin users for all of the nodes in the cluster.
+
+When a topology is sumbitted, the sumbitting user can specify users in this list as well. The users specified-in addition to the users in the cluster-wide setting-will be granted access to the submitted topology's details in the ui and/or to the topology's worker logs in the logviewers.
+
+### Supervisors headless User and group Setup
+
+To ensure isolation of users in multi-tenancy, there is need to run supervisors and headless user and group unique to execution on the supervisor nodes. To enable this follow below steps.
+1. Add headlessuser to all supervisor hosts.
+2. Create unique group and make it the primary group for the headless user on the supervisor nodes.
+3. The set following properties on storm for these supervisor nodes.
+
+### Multi-tenant Scheduler
+
+To support multi-tenancy better we have written a new scheduler. To enable this scheduler set.
+```yaml
+storm.scheduler: "backtype.storm.scheduler.multitenant.MultitenantScheduler"
+```
+Be aware that many of the features of this scheduler rely on storm authentication. Without them the scheduler will not know what the user is and will not isolate topologies properly.
+
+The goal of the multi-tenant scheduler is to provide a way to isolate topologies from one another, but to also limit the resources that an individual user can have in the cluster.
+
+The scheduler currently has one config that can be set either through =storm.yaml= or through a separate config file called =multitenant-scheduler.yaml= that should be placed in the same directory as =storm.yaml=. It is preferable to use =multitenant-scheduler.yaml= because it can be updated without needing to restart nimbus.
+
+There is currently only one config in =multitenant-scheduler.yaml=, =multitenant.scheduler.user.pools= is a map from the user name, to the maximum number of nodes that user is guaranteed to be able to use for their topologies.
+
+For example:
+
+```yaml
+multitenant.scheduler.user.pools:
+ "evans": 10
+ "derek": 10
+```
+
+### Run as User
+By default storm runs workers as the user that is running the supervisor. This is not ideal for security. To make storm run the topologies as the user that launched them set.
+
+```yaml
+supervisor.run.worker.as.user: true
+```
+
+There are several files that go along with this that are needed to be configured properly to make storm secure.
+
+The worker-launcher executable is a special program that allows the supervisor to launch workers as different users. For this to work it needs to be owned by root, but with the group set to be a group that only teh supervisor headless user is a part of.
+It also needs to have 6550 permissions.
+There is also a worker-launcher.cfg file, usually located under /etc/ that should look somethign like the following
+
+```
+storm.worker-launcher.group=$(worker_launcher_group)
+min.user.id=$(min_user_id)
+```
+where worker_launcher_group is the same group the supervisor is a part of, and min.user.id is set to the first real user id on the system.
+This config file also needs to be owned by root and not have world or group write permissions.
+
+### Automatic Credentials Push and Renewal
+Individual topologies have the ability to push credentials (tickets and tokens) to workers so that they can access secure services. Exposing this to all of the users can be a pain for them.
+To hide this from them in the common case plugins can be used to populate the credentials, unpack them on the other side into a java Subject, and also allow Nimbus to renew the credentials if needed.
+These are controlled by the following configs. topology.auto-credentials is a list of java plugins that populate the credentials and unpack them on the worker side.
+On a kerberos secure cluster they should be set by default to point to backtype.storm.security.auth.kerberos.AutoTGT. nimbus.credential.renewers.classes should also be set to this value so that nimbus can periodically renew the TGT on behalf of the user.
+
+nimbus.credential.renewers.freq.secs controls how often the renewer will poll to see if anything needs to be renewed, but the default should be fine.
+
+### Limits
+By default storm allows any sized topology to be submitted. But ZK and others have limitations on how big a topology can actually be. The following configs allow you to limit the maximum size a topology can be.
+
+| YAML Setting | Description |
+|------------|----------------------|
+| nimbus.slots.perTopology | The maximum number of slots/workers a topology can use. |
+| nimbus.executors.perTopology | The maximum number of executors/threads a topology can use. |
+
+### Log Cleanup
+The Logviewer deamon now is also responsible for cleaning up old log files for dead topologies.
+
+| YAML Setting | Description |
+|--------------|-------------------------------------|
+| logviewer.cleanup.age.mins | How old (by last modification time) must a worker's log be before that log is considered for clean-up. (Living workers' logs are never cleaned up by the logviewer: Their logs are rolled via logback.) |
+| logviewer.cleanup.interval.secs | Interval of time in seconds that the logviewer cleans up worker logs. |
+
+
+### DRPC
+Hopefully more on this soon
-### Zookeeper
-Zookeeper uses other ports for communications within the ensemble the details of which
-are beyond the scope of this document. You should look at restricting Zookeeper access
-as well, because storm does not set up any ACLs for the data it write to Zookeeper.
diff --git a/bin/storm b/bin/storm
index c9885a29e28..d657aa78b4d 100755
--- a/bin/storm
+++ b/bin/storm
@@ -190,6 +190,18 @@ def kill(*args):
jvmtype="-client",
extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])
+
+def upload_credentials(*args):
+ """Syntax: [storm upload_credentials topology-name [credkey credvalue]*]
+
+ Uploads a new set of credentials to a running topology
+ """
+ exec_storm_class(
+ "backtype.storm.command.upload_credentials",
+ args=args,
+ jvmtype="-client",
+ extrajars=[USER_CONF_DIR, STORM_DIR + "/bin"])
+
def activate(*args):
"""Syntax: [storm activate topology-name]
@@ -431,7 +443,8 @@ COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui
"drpc": drpc, "supervisor": supervisor, "localconfvalue": print_localconfvalue,
"remoteconfvalue": print_remoteconfvalue, "repl": repl, "classpath": print_classpath,
"activate": activate, "deactivate": deactivate, "rebalance": rebalance, "help": print_usage,
- "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version}
+ "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version,
+ "upload-credentials": upload_credentials}
def parse_config(config_list):
global CONFIG_OPTS
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index bb1cb040bed..d482065d36f 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -32,14 +32,25 @@ storm.zookeeper.connection.timeout: 15000
storm.zookeeper.retry.times: 5
storm.zookeeper.retry.interval: 1000
storm.zookeeper.retry.intervalceiling.millis: 30000
+storm.zookeeper.auth.user: null
+storm.zookeeper.auth.password: null
storm.cluster.mode: "distributed" # can be distributed or local
storm.local.mode.zmq: false
storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin"
+storm.principal.tolocal: "backtype.storm.security.auth.DefaultPrincipalToLocal"
storm.messaging.transport: "backtype.storm.messaging.netty.Context"
+storm.nimbus.retry.times: 5
+storm.nimbus.retry.interval.millis: 2000
+storm.nimbus.retry.intervalceiling.millis: 60000
+storm.auth.simple-white-list.users: []
+storm.auth.simple-acl.users: []
+storm.auth.simple-acl.users.commands: []
+storm.auth.simple-acl.admins: []
### nimbus.* configs are for the master
nimbus.host: "localhost"
nimbus.thrift.port: 6627
+nimbus.thrift.threads: 64
nimbus.thrift.max_buffer_size: 1048576
nimbus.childopts: "-Xmx1024m"
nimbus.task.timeout.secs: 30
@@ -51,22 +62,40 @@ nimbus.task.launch.secs: 120
nimbus.reassign: true
nimbus.file.copy.expiration.secs: 600
nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator"
+nimbus.credential.renewers.freq.secs: 600
### ui.* configs are for the master
ui.port: 8080
ui.childopts: "-Xmx768m"
+ui.actions.enabled: true
+ui.filter: null
+ui.filter.params: null
+ui.users: null
+ui.header.buffer.bytes: 4096
+ui.http.creds.plugin: backtype.storm.security.auth.DefaultHttpCredentialsPlugin
logviewer.port: 8000
logviewer.childopts: "-Xmx128m"
+logviewer.cleanup.age.mins: 10080
logviewer.appender.name: "A1"
+logs.users: null
drpc.port: 3772
drpc.worker.threads: 64
+drpc.max_buffer_size: 1048576
drpc.queue.size: 128
drpc.invocations.port: 3773
+drpc.invocations.threads: 64
drpc.request.timeout.secs: 600
drpc.childopts: "-Xmx768m"
+drpc.http.port: 3774
+drpc.https.port: -1
+drpc.https.keystore.password: ""
+drpc.https.keystore.type: "JKS"
+drpc.http.creds.plugin: backtype.storm.security.auth.DefaultHttpCredentialsPlugin
+drpc.authorizer.acl.filename: "drpc-auth-acl.yaml"
+drpc.authorizer.acl.strict: false
transactional.zookeeper.root: "/transactional"
transactional.zookeeper.servers: null
@@ -80,6 +109,7 @@ supervisor.slots.ports:
- 6702
- 6703
supervisor.childopts: "-Xmx256m"
+supervisor.run.worker.as.user: false
#how long supervisor will wait to ensure that a worker process is started
supervisor.worker.start.timeout.secs: 120
#how long between heartbeats until supervisor considers that worker dead and tries to restart it
@@ -89,13 +119,18 @@ supervisor.monitor.frequency.secs: 3
#how frequently the supervisor heartbeats to the cluster state (for nimbus)
supervisor.heartbeat.frequency.secs: 5
supervisor.enable: true
+supervisor.supervisors: []
+supervisor.supervisors.commands: []
+
### worker.* configs are for task workers
worker.childopts: "-Xmx768m"
+worker.gc.childopts: ""
worker.heartbeat.frequency.secs: 1
task.heartbeat.frequency.secs: 3
task.refresh.poll.secs: 10
+task.credentials.poll.secs: 30
zmq.threads: 1
zmq.linger.millis: 5000
@@ -140,5 +175,6 @@ topology.max.error.report.per.interval: 5
topology.kryo.factory: "backtype.storm.serialization.DefaultKryoFactory"
topology.tuple.serializer: "backtype.storm.serialization.types.ListDelegateSerializer"
topology.trident.batch.emit.interval.millis: 500
+topology.testing.always.try.serialize: false
dev.zookeeper.path: "/tmp/dev-storm-zookeeper"
diff --git a/conf/jaas_digest.conf b/conf/jaas_digest.conf
index 06dd7aa1ac5..301f3e0c73a 100644
--- a/conf/jaas_digest.conf
+++ b/conf/jaas_digest.conf
@@ -17,21 +17,17 @@
*/
/* This is example of JAAS Login configuration for digest authentication
-*/
-/*
StormServer section should contain a list of authorized users and their passwords.
+StormClient section contains one user name and his/her password.
*/
StormServer {
org.apache.zookeeper.server.auth.DigestLoginModule required
user_super="adminsecret"
- user_bob="bobsecret";
+ user_bob="bobsecret"
user_john="johnsecret";
};
-/*
-StormClient section contains one user name and his/her password.
-*/
StormClient {
org.apache.zookeeper.server.auth.DigestLoginModule required
username="bob"
diff --git a/conf/jaas_kerberos.conf b/conf/jaas_kerberos.conf
new file mode 100644
index 00000000000..5861df279d7
--- /dev/null
+++ b/conf/jaas_kerberos.conf
@@ -0,0 +1,15 @@
+StormServer {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="/etc/nimbus_server.keytab"
+ storeKey=true
+ useTicketCache=false
+ principal="nimbus_server/carcloth.corp.yahoo.com@STORM.CORP.YAHOO.COM";
+};
+StormClient {
+ com.sun.security.auth.module.Krb5LoginModule required
+ doNotPrompt=true
+ useTicketCache=true
+ serviceName="nimbus_server";
+};
+
diff --git a/pom.xml b/pom.xml
index 2c0cbc11118..8f1fe73b1c8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -179,6 +179,9 @@
0.3.02.1.33.4.5
+ 2.1.3
+ 0.1.0
+ 1.2.0
@@ -415,6 +418,23 @@
netty${netty.version}
+
+ org.clojars.runa
+ conjure
+ ${conjure.version}
+ test
+
+
+ org.clojure
+ clojure-contrib
+ ${clojure-contrib.version}
+ test
+
+
+ org.clojure
+ data.codec
+ ${clojure-data-codec.version}
+ org.clojuretools.nrepl
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index fec62180da8..c6022fdea4d 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -28,6 +28,11 @@
Storm CoreStorm Core Java API and Clojure implementation.
+
+ /etc/storm
+
+
+
@@ -76,7 +81,22 @@
clojure-completetest
-
+
+ org.clojars.runa
+ conjure
+ test
+
+
+ org.clojure
+ clojure-contrib
+ test
+
+
+ org.clojure
+ data.codec
+ test
+
+
commons-io
@@ -171,11 +191,6 @@
mockito-alltest
-
- org.clojars.runa
- conjure
- test
- src/jvm
@@ -238,7 +253,8 @@
test-with-junit
- ${test.extra.args}
+
+ ${argLine} ${test.extra.args}
@@ -303,7 +319,204 @@
-
+
+ maven-dependency-plugin
+ 2.6
+
+
+ copy-dependencies
+ package
+
+ copy-dependencies
+
+
+ false
+ false
+ true
+ runtime
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-source-plugin
+ 2.2.1
+
+
+ attach-sources
+
+
+ jar-no-fork
+
+
+
+
+
+
+
+ coverage
+ true
+
+
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+ 1.7
+
+
+ add-source
+ generate-sources
+
+ add-source
+
+
+
+ src/clj
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-antrun-plugin
+ 1.7
+
+
+ pre-test-jacoco-clean
+ process-test-classes
+
+
+
+
+
+
+ run
+
+
+
+
+
+ org.jacoco
+ jacoco-maven-plugin
+ 0.6.1.201212231917
+
+
+ prepare-agent
+
+ prepare-agent
+
+
+ true
+
+ backtype/storm/metric/api/IMetricsConsumer$DataPointFieldAccess
+ backtype/storm/metric/api/IMetricsConsumer$TaskInfoFieldAccess
+ backtype/storm/testing/TestSerObjectFieldAccess
+
+
+
+
+ report
+ prepare-package
+
+ report
+
+
+
+ backtype/storm/generated/*
+
+
+ backtype/*/*/*/*
+ backtype/*/*/*
+ backtype/*/*
+ backtype/*
+ zilch/*
+ storm/*/*/*/*
+ storm/*/*/*
+ storm/*/*
+ storm/*
+
+
+
+
+
+
+
+
+
+ native
+
+
+
+ org.codehaus.mojo
+ exec-maven-plugin
+ 1.2.1
+
+
+ generate-sources
+ exec
+
+
+
+ sh
+
+ -c
+ mkdir -p ${project.build.directory}/; cp -rufv ${basedir}/src/native/ ${project.build.directory}/
+
+
+
+
+ org.codehaus.mojo
+ make-maven-plugin
+
+
+ compile
+ compile
+
+ autoreconf
+ configure
+ make-install
+
+
+
+ test
+ test
+
+ test
+
+
+
+
+
+ ${project.build.directory}/native/worker-launcher
+
+ -i
+
+
+
+
+
+ CFLAGS
+ -DEXEC_CONF_DIR=${worker-launcher.conf.dir} ${worker-launcher.additional_cflags}
+
+
+ ${project.build.directory}/native/worker-launcher
+ /usr/local
+
+
+ ${project.build.directory}/native/target
+
+
+
+
+
+
+ false
+
+
+
+
diff --git a/storm-core/src/clj/backtype/storm/LocalCluster.clj b/storm-core/src/clj/backtype/storm/LocalCluster.clj
index 77f3b3f8816..db6e60ccdda 100644
--- a/storm-core/src/clj/backtype/storm/LocalCluster.clj
+++ b/storm-core/src/clj/backtype/storm/LocalCluster.clj
@@ -43,6 +43,9 @@
topology
submit-opts))
+(defn -uploadNewCredentials [this name creds]
+ (.uploadNewCredentials (:nimbus (. this state)) name creds))
+
(defn -shutdown [this]
(kill-local-storm-cluster (. this state)))
diff --git a/storm-core/src/clj/backtype/storm/LocalDRPC.clj b/storm-core/src/clj/backtype/storm/LocalDRPC.clj
index a6dab95c329..d4884e48c60 100644
--- a/storm-core/src/clj/backtype/storm/LocalDRPC.clj
+++ b/storm-core/src/clj/backtype/storm/LocalDRPC.clj
@@ -15,7 +15,7 @@
;; limitations under the License.
(ns backtype.storm.LocalDRPC
(:require [backtype.storm.daemon [drpc :as drpc]])
- (:use [backtype.storm util])
+ (:use [backtype.storm config util])
(:import [backtype.storm.utils InprocMessaging ServiceRegistry])
(:gen-class
:init init
@@ -24,7 +24,7 @@
:state state ))
(defn -init []
- (let [handler (drpc/service-handler)
+ (let [handler (drpc/service-handler (read-storm-config))
id (ServiceRegistry/registerService handler)
]
[[] {:service-id id :handler handler}]
diff --git a/storm-core/src/clj/backtype/storm/bootstrap.clj b/storm-core/src/clj/backtype/storm/bootstrap.clj
index 5f34ff1022f..0b41cdd87ed 100644
--- a/storm-core/src/clj/backtype/storm/bootstrap.clj
+++ b/storm-core/src/clj/backtype/storm/bootstrap.clj
@@ -23,10 +23,11 @@
(import (quote [backtype.storm.utils Utils LocalState Time TimeCacheMap
TimeCacheMap$ExpiredCallback
RotatingMap RotatingMap$ExpiredCallback
- BufferFileInputStream
+ BufferFileInputStream ZookeeperServerCnxnFactory
RegisteredGlobalState ThriftTopologyUtils DisruptorQueue
MutableObject MutableLong]))
(import (quote [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer]))
+ (import (quote [backtype.storm.security.auth ThriftServer ThriftClient ReqContext]))
(import (quote [backtype.storm.spout ISpout SpoutOutputCollector ISpoutOutputCollector ShellSpout]))
(import (quote [backtype.storm.tuple Tuple TupleImpl Fields MessageId]))
(import (quote [backtype.storm.task IBolt IOutputCollector
@@ -52,7 +53,7 @@
TopologySummary ExecutorSummary ExecutorStats ExecutorSpecificStats
SpoutStats BoltStats ErrorInfo SupervisorSummary ExecutorInfo
KillOptions SubmitOptions RebalanceOptions JavaObject JavaObjectArg
- TopologyInitialStatus]))
+ TopologyInitialStatus AuthorizationException]))
(import (quote [backtype.storm.daemon.common StormBase Assignment
SupervisorInfo WorkerHeartbeat]))
(import (quote [backtype.storm.grouping CustomStreamGrouping]))
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index b5c1e3b6f1b..7c43aea5f88 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -14,38 +14,45 @@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.cluster
- (:import [org.apache.zookeeper.data Stat])
- (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException])
+ (:import [org.apache.zookeeper.data Stat ACL Id])
+ (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
(:import [backtype.storm.utils Utils])
+ (:import [java.security MessageDigest])
+ (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
(:use [backtype.storm util log config])
(:require [backtype.storm [zookeeper :as zk]])
(:require [backtype.storm.daemon [common :as common]])
-
)
(defprotocol ClusterState
- (set-ephemeral-node [this path data])
+ (set-ephemeral-node [this path data acls])
(delete-node [this path])
- (create-sequential [this path data])
- (set-data [this path data]) ;; if node does not exist, create persistent with this data
+ (create-sequential [this path data acls])
+ (set-data [this path data acls]) ;; if node does not exist, create persistent with this data
(get-data [this path watch?])
(get-children [this path watch?])
- (mkdirs [this path])
+ (mkdirs [this path acls])
(close [this])
(register [this callback])
(unregister [this id])
)
-(defn mk-distributed-cluster-state [conf]
- (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf conf)]
- (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT))
+(defn mk-topo-only-acls [topo-conf]
+ (let [payload (.get topo-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)]
+ (when (Utils/isZkAuthenticationConfiguredTopology topo-conf)
+ [(first ZooDefs$Ids/CREATOR_ALL_ACL)
+ (ACL. ZooDefs$Perms/READ (Id. "digest" (DigestAuthenticationProvider/generateDigest payload)))])))
+
+(defnk mk-distributed-cluster-state [conf :auth-conf nil :acls nil]
+ (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf auth-conf)]
+ (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT) acls)
(.close zk))
(let [callbacks (atom {})
active (atom true)
zk (zk/mk-client conf
(conf STORM-ZOOKEEPER-SERVERS)
(conf STORM-ZOOKEEPER-PORT)
- :auth-conf conf
+ :auth-conf auth-conf
:root (conf STORM-ZOOKEEPER-ROOT)
:watcher (fn [state type path]
(when @active
@@ -65,28 +72,28 @@
(unregister [this id]
(swap! callbacks dissoc id))
- (set-ephemeral-node [this path data]
- (zk/mkdirs zk (parent-path path))
+ (set-ephemeral-node [this path data acls]
+ (zk/mkdirs zk (parent-path path) acls)
(if (zk/exists zk path false)
(try-cause
(zk/set-data zk path data) ; should verify that it's ephemeral
(catch KeeperException$NoNodeException e
(log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
- (zk/create-node zk path data :ephemeral)
+ (zk/create-node zk path data :ephemeral acls)
))
- (zk/create-node zk path data :ephemeral)
+ (zk/create-node zk path data :ephemeral acls)
))
- (create-sequential [this path data]
- (zk/create-node zk path data :sequential))
+ (create-sequential [this path data acls]
+ (zk/create-node zk path data :sequential acls))
- (set-data [this path data]
+ (set-data [this path data acls]
;; note: this does not turn off any existing watches
(if (zk/exists zk path false)
(zk/set-data zk path data)
(do
- (zk/mkdirs zk (parent-path path))
- (zk/create-node zk path data :persistent)
+ (zk/mkdirs zk (parent-path path) acls)
+ (zk/create-node zk path data :persistent acls)
)))
(delete-node [this path]
@@ -100,8 +107,8 @@
(get-children [this path watch?]
(zk/get-children zk path watch?))
- (mkdirs [this path]
- (zk/mkdirs zk path))
+ (mkdirs [this path acls]
+ (zk/mkdirs zk path acls))
(close [this]
(reset! active false)
@@ -135,6 +142,8 @@
(remove-storm! [this storm-id])
(report-error [this storm-id task-id error])
(errors [this storm-id task-id])
+ (set-credentials! [this storm-id creds topo-conf])
+ (credentials [this storm-id callback])
(disconnect [this])
)
@@ -146,12 +155,14 @@
(def SUPERVISORS-ROOT "supervisors")
(def WORKERBEATS-ROOT "workerbeats")
(def ERRORS-ROOT "errors")
+(def CREDENTIALS-ROOT "credentials")
(def ASSIGNMENTS-SUBTREE (str "/" ASSIGNMENTS-ROOT))
(def STORMS-SUBTREE (str "/" STORMS-ROOT))
(def SUPERVISORS-SUBTREE (str "/" SUPERVISORS-ROOT))
(def WORKERBEATS-SUBTREE (str "/" WORKERBEATS-ROOT))
(def ERRORS-SUBTREE (str "/" ERRORS-ROOT))
+(def CREDENTIALS-SUBTREE (str "/" CREDENTIALS-ROOT))
(defn supervisor-path [id]
(str SUPERVISORS-SUBTREE "/" id))
@@ -174,6 +185,9 @@
(defn error-path [storm-id component-id]
(str (error-storm-root storm-id) "/" (url-encode component-id)))
+(defn credentials-path [storm-id]
+ (str CREDENTIALS-SUBTREE "/" storm-id))
+
(defn- issue-callback! [cb-atom]
(let [cb @cb-atom]
(reset! cb-atom nil)
@@ -210,14 +224,15 @@
(into {}))))
;; Watches should be used for optimization. When ZK is reconnecting, they're not guaranteed to be called.
-(defn mk-storm-cluster-state [cluster-state-spec]
+(defnk mk-storm-cluster-state [cluster-state-spec :acls nil]
(let [[solo? cluster-state] (if (satisfies? ClusterState cluster-state-spec)
[false cluster-state-spec]
- [true (mk-distributed-cluster-state cluster-state-spec)])
+ [true (mk-distributed-cluster-state cluster-state-spec :auth-conf cluster-state-spec :acls acls)])
assignment-info-callback (atom {})
supervisors-callback (atom nil)
assignments-callback (atom nil)
storm-base-callback (atom {})
+ credentials-callback (atom {})
state-id (register
cluster-state
(fn [type path]
@@ -228,12 +243,13 @@
(issue-map-callback! assignment-info-callback (first args)))
SUPERVISORS-ROOT (issue-callback! supervisors-callback)
STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
+ CREDENTIALS-ROOT (issue-map-callback! credentials-callback (first args))
;; this should never happen
(halt-process! 30 "Unknown callback for subtree " subtree args)
)
)))]
(doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE]]
- (mkdirs cluster-state p))
+ (mkdirs cluster-state p acls))
(reify
StormClusterState
@@ -288,14 +304,14 @@
)
(worker-heartbeat! [this storm-id node port info]
- (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize info)))
+ (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize info) acls))
(remove-worker-heartbeat! [this storm-id node port]
(delete-node cluster-state (workerbeat-path storm-id node port))
)
(setup-heartbeats! [this storm-id]
- (mkdirs cluster-state (workerbeat-storm-root storm-id)))
+ (mkdirs cluster-state (workerbeat-storm-root storm-id) acls))
(teardown-heartbeats! [this storm-id]
(try-cause
@@ -312,11 +328,11 @@
)))
(supervisor-heartbeat! [this supervisor-id info]
- (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info))
+ (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info) acls)
)
(activate-storm! [this storm-id storm-base]
- (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base))
+ (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base) acls)
)
(update-storm! [this storm-id new-elems]
@@ -326,7 +342,8 @@
(set-data cluster-state (storm-path storm-id)
(-> base
(merge new-elems)
- Utils/serialize))))
+ Utils/serialize)
+ acls)))
(storm-base [this storm-id callback]
(when callback
@@ -339,18 +356,29 @@
)
(set-assignment! [this storm-id info]
- (set-data cluster-state (assignment-path storm-id) (Utils/serialize info))
+ (set-data cluster-state (assignment-path storm-id) (Utils/serialize info) acls)
)
(remove-storm! [this storm-id]
(delete-node cluster-state (assignment-path storm-id))
+ (delete-node cluster-state (credentials-path storm-id))
(remove-storm-base! this storm-id))
+ (set-credentials! [this storm-id creds topo-conf]
+ (let [topo-acls (mk-topo-only-acls topo-conf)
+ path (credentials-path storm-id)]
+ (set-data cluster-state path (Utils/serialize creds) topo-acls)))
+
+ (credentials [this storm-id callback]
+ (when callback
+ (swap! credentials-callback assoc storm-id callback))
+ (maybe-deserialize (get-data cluster-state (credentials-path storm-id) (not-nil? callback))))
+
(report-error [this storm-id component-id error]
(let [path (error-path storm-id component-id)
data {:time-secs (current-time-secs) :error (stringify-error error)}
- _ (mkdirs cluster-state path)
- _ (create-sequential cluster-state (str path "/e") (Utils/serialize data))
+ _ (mkdirs cluster-state path acls)
+ _ (create-sequential cluster-state (str path "/e") (Utils/serialize data) acls)
to-kill (->> (get-children cluster-state path false)
(sort-by parse-error-path)
reverse
@@ -360,7 +388,7 @@
(errors [this storm-id component-id]
(let [path (error-path storm-id component-id)
- _ (mkdirs cluster-state path)
+ _ (mkdirs cluster-state path acls)
children (get-children cluster-state path false)
errors (dofor [c children]
(let [data (-> (get-data cluster-state (str path "/" c) false)
diff --git a/storm-core/src/clj/backtype/storm/command/upload_credentials.clj b/storm-core/src/clj/backtype/storm/command/upload_credentials.clj
new file mode 100644
index 00000000000..05a82cb0131
--- /dev/null
+++ b/storm-core/src/clj/backtype/storm/command/upload_credentials.clj
@@ -0,0 +1,35 @@
+;; 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.
+(ns backtype.storm.command.upload-credentials
+ (:use [clojure.tools.cli :only [cli]])
+ (:use [backtype.storm log util])
+ (:import [backtype.storm StormSubmitter])
+ (:import [java.util Properties])
+ (:import [java.io FileReader])
+ (:gen-class))
+
+(defn read-map [file-name]
+ (let [props (Properties. )
+ _ (.load props (FileReader. file-name))]
+ (clojurify-structure props)))
+
+(defn -main [& args]
+ (let [[{cred-file :file} [name & rawCreds]] (cli args ["-f" "--file" :default nil])
+ _ (when (and rawCreds (not (even? (.size rawCreds)))) (throw (RuntimeException. "Need an even number of arguments to make a map")))
+ mapping (if rawCreds (apply assoc {} rawCreds) {})
+ file-mapping (if (nil? cred-file) {} (read-map cred-file))]
+ (StormSubmitter/pushCredentials name {} (merge file-mapping mapping))
+ (log-message "Uploaded new creds to topology: " name)))
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index 15be94dd36d..dcdca63b9bc 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -14,12 +14,12 @@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.config
- (:import [java.io FileReader File])
+ (:import [java.io FileReader File IOException])
(:import [backtype.storm Config ConfigValidation$FieldValidator])
(:import [backtype.storm.utils Utils LocalState])
(:import [org.apache.commons.io FileUtils])
(:require [clojure [string :as str]])
- (:use [backtype.storm util])
+ (:use [backtype.storm log util])
)
(def RESOURCES-SUBDIR "resources")
@@ -117,10 +117,13 @@
(validate-configs-with-schemas conf)
conf))
-(defn read-yaml-config [name]
- (let [conf (clojurify-structure (Utils/findAndReadConfigFile name true))]
- (validate-configs-with-schemas conf)
- conf))
+(defn read-yaml-config
+ ([name must-exist]
+ (let [conf (clojurify-structure (Utils/findAndReadConfigFile name must-exist))]
+ (validate-configs-with-schemas conf)
+ conf))
+ ([name]
+ (read-yaml-config true)))
(defn master-local-dir [conf]
(let [ret (str (conf STORM-LOCAL-DIR) file-path-separator "nimbus")]
@@ -198,6 +201,32 @@
(Utils/deserialize (FileUtils/readFileToByteArray (File. topology-path)))
))
+(defn worker-user-root [conf]
+ (str (conf STORM-LOCAL-DIR) "/workers-users"))
+
+(defn worker-user-file [conf worker-id]
+ (str (worker-user-root conf) "/" worker-id))
+
+(defn get-worker-user [conf worker-id]
+ (log-message "GET worker-user " worker-id)
+ (try
+ (str/trim (slurp (worker-user-file conf worker-id)))
+ (catch IOException e
+ (log-warn-error e "Failed to get worker user for " worker-id ".")
+ nil
+ )))
+
+
+(defn set-worker-user! [conf worker-id user]
+ (log-message "SET worker-user " worker-id " " user)
+ (let [file (worker-user-file conf worker-id)]
+ (.mkdirs (.getParentFile (File. file)))
+ (spit (worker-user-file conf worker-id) user)))
+
+(defn remove-worker-user! [conf worker-id]
+ (log-message "REMOVE worker-user " worker-id)
+ (.delete (File. (worker-user-file conf worker-id))))
+
(defn worker-root
([conf]
(str (conf STORM-LOCAL-DIR) file-path-separator "workers"))
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index e6bf81ae66c..6a996022e58 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -21,6 +21,7 @@
(:import [backtype.storm.task WorkerTopologyContext])
(:import [backtype.storm Constants])
(:import [backtype.storm.metric SystemBolt])
+ (:import [backtype.storm.security.auth IAuthorizer])
(:require [clojure.set :as set])
(:require [backtype.storm.daemon.acker :as acker])
(:require [backtype.storm.thrift :as thrift])
@@ -40,6 +41,7 @@
(def SYSTEM-TICK-STREAM-ID Constants/SYSTEM_TICK_STREAM_ID)
(def METRICS-STREAM-ID Constants/METRICS_STREAM_ID)
(def METRICS-TICK-STREAM-ID Constants/METRICS_TICK_STREAM_ID)
+(def CREDENTIALS-CHANGED-STREAM-ID Constants/CREDENTIALS_CHANGED_STREAM_ID)
;; the task id is the virtual port
;; node->host is here so that tasks know who to talk to just from assignment
@@ -48,7 +50,7 @@
;; component->executors is a map from spout/bolt id to number of executors for that component
-(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors])
+(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors owner])
(defrecord SupervisorInfo [time-secs hostname assignment-id used-ports meta scheduler-meta uptime-secs])
@@ -288,7 +290,8 @@
{}
(SystemBolt.)
{SYSTEM-TICK-STREAM-ID (thrift/output-fields ["rate_secs"])
- METRICS-TICK-STREAM-ID (thrift/output-fields ["interval"])}
+ METRICS-TICK-STREAM-ID (thrift/output-fields ["interval"])
+ CREDENTIALS-CHANGED-STREAM-ID (thrift/output-fields ["creds"])}
:p 0
:conf {TOPOLOGY-TASKS 0})]
(.put_to_bolts topology SYSTEM-COMPONENT-ID system-bolt-spec)))
@@ -349,3 +352,14 @@
(->> executor->node+port
(mapcat (fn [[e node+port]] (for [t (executor-id->tasks e)] [t node+port])))
(into {})))
+
+(defn mk-authorization-handler [klassname conf]
+ (let [aznClass (if klassname (Class/forName klassname))
+ aznHandler (if aznClass (.newInstance aznClass))]
+ (if aznHandler (.prepare ^IAuthorizer aznHandler conf))
+ (log-debug "authorization class name:" klassname
+ " class:" aznClass
+ " handler:" aznHandler)
+ aznHandler
+ ))
+
diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
index df07343b046..a7d3c32569d 100644
--- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
@@ -14,17 +14,22 @@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.daemon.drpc
- (:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
- (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
- (:import [org.apache.thrift.exception])
- (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
+ (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftConnectionType])
+ (:import [backtype.storm.security.auth.authorizer DRPCAuthorizerBase])
(:import [backtype.storm.generated DistributedRPC DistributedRPC$Iface DistributedRPC$Processor
DRPCRequest DRPCExecutionException DistributedRPCInvocations DistributedRPCInvocations$Iface
DistributedRPCInvocations$Processor])
(:import [java.util.concurrent Semaphore ConcurrentLinkedQueue ThreadPoolExecutor ArrayBlockingQueue TimeUnit])
(:import [backtype.storm.daemon Shutdownable])
(:import [java.net InetAddress])
+ (:import [backtype.storm.generated AuthorizationException])
(:use [backtype.storm bootstrap config log])
+ (:use [backtype.storm.daemon common])
+ (:use [backtype.storm.ui helpers])
+ (:use compojure.core)
+ (:use ring.middleware.reload)
+ (:use [ring.adapter.jetty :only [run-jetty]])
+ (:require [compojure.handler :as handler])
(:gen-class))
(bootstrap)
@@ -40,17 +45,32 @@
))
(@queues-atom function))
+(defn check-authorization
+ ([aclHandler mapping operation context]
+ (if aclHandler
+ (let [context (or context (ReqContext/context))]
+ (if-not (.permit aclHandler context operation mapping)
+ (let [principal (.principal context)
+ user (if principal (.getName principal) "unknown")]
+ (throw (AuthorizationException.
+ (str "DRPC request '" operation "' for '"
+ user "' user is not authorized"))))))))
+ ([aclHandler mapping operation]
+ (check-authorization aclHandler mapping operation (ReqContext/context))))
+
;; TODO: change this to use TimeCacheMap
-(defn service-handler []
- (let [conf (read-storm-config)
+(defn service-handler [conf]
+ (let [drpc-acl-handler (mk-authorization-handler (conf DRPC-AUTHORIZER) conf)
ctr (atom 0)
id->sem (atom {})
id->result (atom {})
id->start (atom {})
+ id->func (atom {})
request-queues (atom {})
cleanup (fn [id] (swap! id->sem dissoc id)
(swap! id->result dissoc id)
- (swap! id->start dissoc id))
+ (swap! id->start dissoc id)
+ (swap! id->func dissoc id))
my-ip (.getHostAddress (InetAddress/getLocalHost))
clear-thread (async-loop
(fn []
@@ -66,12 +86,16 @@
]
(reify DistributedRPC$Iface
(^String execute [this ^String function ^String args]
- (log-debug "Received DRPC request for " function " " args " at " (System/currentTimeMillis))
+ (log-debug "Received DRPC request for " function " (" args ") at " (System/currentTimeMillis))
+ (check-authorization drpc-acl-handler
+ {DRPCAuthorizerBase/FUNCTION_NAME function}
+ "execute")
(let [id (str (swap! ctr (fn [v] (mod (inc v) 1000000000))))
^Semaphore sem (Semaphore. 0)
req (DRPCRequest. args id)
^ConcurrentLinkedQueue queue (acquire-queue request-queues function)
]
+ (swap! id->func assoc id function)
(swap! id->start assoc id (current-time-secs))
(swap! id->sem assoc id sem)
(.add queue req)
@@ -87,19 +111,30 @@
))))
DistributedRPCInvocations$Iface
(^void result [this ^String id ^String result]
- (let [^Semaphore sem (@id->sem id)]
- (log-debug "Received result " result " for " id " at " (System/currentTimeMillis))
- (when sem
- (swap! id->result assoc id result)
- (.release sem)
- )))
+ (when-let [func (@id->func id)]
+ (check-authorization drpc-acl-handler
+ {DRPCAuthorizerBase/FUNCTION_NAME func}
+ "result")
+ (let [^Semaphore sem (@id->sem id)]
+ (log-debug "Received result " result " for " id " at " (System/currentTimeMillis))
+ (when sem
+ (swap! id->result assoc id result)
+ (.release sem)
+ ))))
(^void failRequest [this ^String id]
- (let [^Semaphore sem (@id->sem id)]
- (when sem
- (swap! id->result assoc id (DRPCExecutionException. "Request failed"))
- (.release sem)
- )))
+ (when-let [func (@id->func id)]
+ (check-authorization drpc-acl-handler
+ {DRPCAuthorizerBase/FUNCTION_NAME func}
+ "failRequest")
+ (let [^Semaphore sem (@id->sem id)]
+ (when sem
+ (swap! id->result assoc id (DRPCExecutionException. "Request failed"))
+ (.release sem)
+ ))))
(^DRPCRequest fetchRequest [this ^String func]
+ (check-authorization drpc-acl-handler
+ {DRPCAuthorizerBase/FUNCTION_NAME func}
+ "fetchRequest")
(let [^ConcurrentLinkedQueue queue (acquire-queue request-queues func)
ret (.poll queue)]
(if ret
@@ -112,35 +147,90 @@
(.interrupt clear-thread))
)))
+(defn handle-request [handler]
+ (fn [request]
+ (handler request)))
+
+(defn webapp [handler http-creds-handler]
+ (->
+ (routes
+ (POST "/drpc/:func" [:as {:keys [body servlet-request]} func & m]
+ (let [args (slurp body)]
+ (if http-creds-handler
+ (.populateContext http-creds-handler (ReqContext/context)
+ servlet-request))
+ (.execute handler func args)))
+ (POST "/drpc/:func/" [:as {:keys [body servlet-request]} func & m]
+ (let [args (slurp body)]
+ (if http-creds-handler
+ (.populateContext http-creds-handler (ReqContext/context)
+ servlet-request))
+ (.execute handler func args)))
+ (GET "/drpc/:func/:args" [:as {:keys [servlet-request]} func args & m]
+ (if http-creds-handler
+ (.populateContext http-creds-handler (ReqContext/context)
+ servlet-request))
+ (.execute handler func args))
+ (GET "/drpc/:func/" [:as {:keys [servlet-request]} func & m]
+ (if http-creds-handler
+ (.populateContext http-creds-handler (ReqContext/context)
+ servlet-request))
+ (.execute handler func ""))
+ (GET "/drpc/:func" [:as {:keys [servlet-request]} func & m]
+ (if http-creds-handler
+ (.populateContext http-creds-handler (ReqContext/context)
+ servlet-request))
+ (.execute handler func "")))
+ (wrap-reload '[backtype.storm.daemon.drpc])
+ handle-request))
+
(defn launch-server!
([]
(let [conf (read-storm-config)
worker-threads (int (conf DRPC-WORKER-THREADS))
queue-size (int (conf DRPC-QUEUE-SIZE))
- service-handler (service-handler)
+ drpc-http-port (int (conf DRPC-HTTP-PORT))
+ 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
;; 64 threads are calling execute, the server won't accept the result
;; invocations that will unblock those threads
- handler-server (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-PORT)))
- (THsHaServer$Args.)
- (.workerThreads 64)
- (.executorService (ThreadPoolExecutor. worker-threads worker-threads
- 60 TimeUnit/SECONDS (ArrayBlockingQueue. queue-size)))
- (.protocolFactory (TBinaryProtocol$Factory.))
- (.processor (DistributedRPC$Processor. service-handler))
- ))
- invoke-server (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-INVOCATIONS-PORT)))
- (THsHaServer$Args.)
- (.workerThreads 64)
- (.protocolFactory (TBinaryProtocol$Factory.))
- (.processor (DistributedRPCInvocations$Processor. service-handler))
- ))]
-
- (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop handler-server) (.stop invoke-server))))
+ handler-server (when (> drpc-port 0)
+ (ThriftServer. conf
+ (DistributedRPC$Processor. service-handler)
+ ThriftConnectionType/DRPC))
+ invoke-server (ThriftServer. conf
+ (DistributedRPCInvocations$Processor. service-handler)
+ ThriftConnectionType/DRPC_INVOCATIONS)
+ http-creds-handler (AuthUtils/GetDrpcHttpCredentialsPlugin conf)]
+ (.addShutdownHook (Runtime/getRuntime) (Thread. (fn []
+ (if handler-server (.stop handler-server))
+ (.stop invoke-server))))
(log-message "Starting Distributed RPC servers...")
(future (.serve invoke-server))
- (.serve handler-server))))
+ (when (> drpc-http-port 0)
+ (let [app (webapp drpc-service-handler http-creds-handler)
+ filter-class (conf DRPC-HTTP-FILTER)
+ filter-params (conf DRPC-HTTP-FILTER-PARAMS)
+ filters-confs [{:filter-class filter-class
+ :filter-params filter-params}]
+ 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)]
+
+ (run-jetty app
+ {:port drpc-http-port :join? false
+ :configurator (fn [server]
+ (config-ssl server
+ https-port
+ https-ks-path
+ https-ks-password
+ https-ks-type)
+ (config-filter server app filters-confs))})))
+ (when handler-server
+ (.serve handler-server)))))
(defn -main []
(launch-server!))
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index f133a1b4c61..d193a9fc0ac 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -16,13 +16,15 @@
(ns backtype.storm.daemon.executor
(:use [backtype.storm.daemon common])
(:use [backtype.storm bootstrap])
+ (:import [backtype.storm ICredentialsListener])
(:import [backtype.storm.hooks ITaskHook])
(:import [backtype.storm.tuple Tuple])
(:import [backtype.storm.spout ISpoutWaitStrategy])
(:import [backtype.storm.hooks.info SpoutAckInfo SpoutFailInfo
EmitInfo BoltFailInfo BoltAckInfo BoltExecuteInfo])
- (:import [backtype.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
+ (:import [backtype.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint])
(:import [backtype.storm Config])
+ (:import [java.util.concurrent ConcurrentLinkedQueue])
(:require [backtype.storm [tuple :as tuple]])
(:require [backtype.storm.daemon [task :as task]])
(:require [backtype.storm.daemon.builtin-metrics :as builtin-metrics]))
@@ -157,7 +159,8 @@
(defprotocol RunningExecutor
(render-stats [this])
- (get-executor-id [this]))
+ (get-executor-id [this])
+ (credentials-changed [this creds]))
(defn throttled-report-error-fn [executor]
(let [storm-conf (:storm-conf executor)
@@ -179,9 +182,11 @@
))))
;; in its own function so that it can be mocked out by tracked topologies
-(defn mk-executor-transfer-fn [batch-transfer->worker]
+(defn mk-executor-transfer-fn [batch-transfer->worker storm-conf]
(fn this
- ([task tuple block? ^List overflow-buffer]
+ ([task tuple block? ^ConcurrentLinkedQueue overflow-buffer]
+ (when (= true (storm-conf TOPOLOGY-DEBUG))
+ (log-message "TRANSFERING tuple TASK: " task " TUPLE: " tuple))
(if (and overflow-buffer (not (.isEmpty overflow-buffer)))
(.add overflow-buffer [task tuple])
(try-cause
@@ -222,7 +227,7 @@
:shared-executor-data (HashMap.)
:storm-active-atom (:storm-active-atom worker)
:batch-transfer-queue batch-transfer->worker
- :transfer-fn (mk-executor-transfer-fn batch-transfer->worker)
+ :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
:suicide-fn (:suicide-fn worker)
:storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker))
:type executor-type
@@ -312,7 +317,7 @@
[[nil (TupleImpl. context [tick-time-secs] Constants/SYSTEM_TASK_ID Constants/SYSTEM_TICK_STREAM_ID)]]
)))))))
-(defn mk-executor [worker executor-id]
+(defn mk-executor [worker executor-id initial-credentials]
(let [executor-data (mk-executor-data worker executor-id)
_ (log-message "Loading executor " (:component-id executor-data) ":" (pr-str executor-id))
task-datas (->> executor-data
@@ -329,7 +334,7 @@
;; trick isn't thread-safe)
system-threads [(start-batch-transfer->worker-handler! worker executor-data)]
handlers (with-error-reaction report-error-and-die
- (mk-threads executor-data task-datas))
+ (mk-threads executor-data task-datas initial-credentials))
threads (concat handlers system-threads)]
(setup-ticks! worker executor-data)
@@ -341,6 +346,13 @@
(stats/render-stats! (:stats executor-data)))
(get-executor-id [this]
executor-id )
+ (credentials-changed [this creds]
+ (let [receive-queue (:receive-queue executor-data)
+ context (:worker-context executor-data)]
+ (disruptor/publish
+ receive-queue
+ [[nil (TupleImpl. context [creds] Constants/SYSTEM_TASK_ID Constants/CREDENTIALS_CHANGED_STREAM_ID)]]
+ )))
Shutdownable
(shutdown
[this]
@@ -361,23 +373,25 @@
(log-message "Shut down executor " component-id ":" (pr-str executor-id)))
)))
-(defn- fail-spout-msg [executor-data task-data msg-id tuple-info time-delta]
+(defn- fail-spout-msg [executor-data task-data msg-id tuple-info time-delta reason id]
(let [^ISpout spout (:object task-data)
+ storm-conf (:storm-conf executor-data)
task-id (:task-id task-data)]
;;TODO: need to throttle these when there's lots of failures
- (log-debug "Failing message " msg-id ": " tuple-info)
+ (when (= true (storm-conf TOPOLOGY-DEBUG))
+ (log-message "SPOUT Failing " id ": " tuple-info " REASON: " reason " MSG-ID: " msg-id))
(.fail spout msg-id)
(task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
(when time-delta
(builtin-metrics/spout-failed-tuple! (:builtin-metrics task-data) (:stats executor-data) (:stream tuple-info))
(stats/spout-failed-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
-(defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta]
+(defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id]
(let [storm-conf (:storm-conf executor-data)
^ISpout spout (:object task-data)
task-id (:task-id task-data)]
(when (= true (storm-conf TOPOLOGY-DEBUG))
- (log-message "Acking message " msg-id))
+ (log-message "SPOUT Acking message " id " " msg-id))
(.ack spout msg-id)
(task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
(when time-delta
@@ -393,7 +407,7 @@
(fn [tuple-batch sequence-id end-of-batch?]
(fast-list-iter [[task-id msg] tuple-batch]
(let [^TupleImpl tuple (if (instance? Tuple msg) msg (.deserialize deserializer msg))]
- (when debug? (log-message "Processing received message " tuple))
+ (when debug? (log-message "Processing received message FOR " task-id " TUPLE: " tuple))
(if task-id
(tuple-action-fn task-id tuple)
;; null task ids are broadcast tuples
@@ -412,7 +426,7 @@
ret
))
-(defmethod mk-threads :spout [executor-data task-datas]
+(defmethod mk-threads :spout [executor-data task-datas initial-credentials]
(let [{:keys [storm-conf component-id worker-context transfer-fn report-error sampler open-or-prepare-was-called?]} executor-data
^ISpoutWaitStrategy spout-wait-strategy (init-spout-wait-strategy storm-conf)
max-spout-pending (executor-max-spout-pending storm-conf (count task-datas))
@@ -424,15 +438,20 @@
pending (RotatingMap.
2 ;; microoptimize for performance of .size method
(reify RotatingMap$ExpiredCallback
- (expire [this msg-id [task-id spout-id tuple-info start-time-ms]]
+ (expire [this id [task-id spout-id tuple-info start-time-ms]]
(let [time-delta (if start-time-ms (time-delta-ms start-time-ms))]
- (fail-spout-msg executor-data (get task-datas task-id) spout-id tuple-info time-delta)
+ (fail-spout-msg executor-data (get task-datas task-id) spout-id tuple-info time-delta "TIMEOUT" id)
))))
tuple-action-fn (fn [task-id ^TupleImpl tuple]
(let [stream-id (.getSourceStreamId tuple)]
(condp = stream-id
Constants/SYSTEM_TICK_STREAM_ID (.rotate pending)
Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data task-datas tuple)
+ Constants/CREDENTIALS_CHANGED_STREAM_ID
+ (let [task-data (get task-datas task-id)
+ spout-obj (:object task-data)]
+ (when (instance? ICredentialsListener spout-obj)
+ (.setCredentials spout-obj (.getValue tuple 0))))
(let [id (.getValue tuple 0)
[stored-task-id spout-id tuple-finished-info start-time-ms] (.remove pending id)]
(when spout-id
@@ -441,9 +460,9 @@
(let [time-delta (if start-time-ms (time-delta-ms start-time-ms))]
(condp = stream-id
ACKER-ACK-STREAM-ID (ack-spout-msg executor-data (get task-datas task-id)
- spout-id tuple-finished-info time-delta)
+ spout-id tuple-finished-info time-delta id)
ACKER-FAIL-STREAM-ID (fail-spout-msg executor-data (get task-datas task-id)
- spout-id tuple-finished-info time-delta)
+ spout-id tuple-finished-info time-delta "FAIL-STREAM" id)
)))
;; TODO: on failure, emit tuple to failure stream
))))
@@ -460,7 +479,7 @@
;; when the overflow buffer is full, spouts stop calling nextTuple until it's able to clear the overflow buffer
;; this limits the size of the overflow buffer to however many tuples a spout emits in one call of nextTuple,
;; preventing memory issues
- overflow-buffer (LinkedList.)]
+ overflow-buffer (ConcurrentLinkedQueue.)]
[(async-loop
(fn []
@@ -471,8 +490,8 @@
(log-message "Opening spout " component-id ":" (keys task-datas))
(doseq [[task-id task-data] task-datas
:let [^ISpout spout-obj (:object task-data)
- tasks-fn (:tasks-fn task-data)
- send-spout-msg (fn [out-stream-id values message-id out-task-id]
+ tasks-fn (:tasks-fn task-data)
+ send-spout-msg (fn [out-stream-id values message-id out-task-id]
(.increment emitted-count)
(let [out-tasks (if out-task-id
(tasks-fn out-task-id out-stream-id values)
@@ -506,13 +525,14 @@
(when message-id
(ack-spout-msg executor-data task-data message-id
{:stream out-stream-id :values values}
- (if (sampler) 0))))
+ (if (sampler) 0) "0:")))
(or out-tasks [])
))]]
(builtin-metrics/register-all (:builtin-metrics task-data) storm-conf (:user-context task-data))
(builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
:receive receive-queue}
storm-conf (:user-context task-data))
+ (when (instance? ICredentialsListener spout-obj) (.setCredentials spout-obj initial-credentials))
(.open spout-obj
storm-conf
@@ -543,7 +563,7 @@
(while (not (.isEmpty overflow-buffer))
(let [[out-task out-tuple] (.peek overflow-buffer)]
(transfer-fn out-task out-tuple false nil)
- (.removeFirst overflow-buffer)))
+ (.poll overflow-buffer)))
(catch InsufficientCapacityException e
))
@@ -591,8 +611,9 @@
(let [curr (or (.get pending key) (long 0))]
(.put pending key (bit-xor curr id))))
-(defmethod mk-threads :bolt [executor-data task-datas]
- (let [execute-sampler (mk-stats-sampler (:storm-conf executor-data))
+(defmethod mk-threads :bolt [executor-data task-datas initial-credentials]
+ (let [storm-conf (:storm-conf executor-data)
+ execute-sampler (mk-stats-sampler storm-conf)
executor-stats (:stats executor-data)
{:keys [storm-conf component-id worker-context transfer-fn report-error sampler
open-or-prepare-was-called?]} executor-data
@@ -616,6 +637,11 @@
;; need to do it this way to avoid reflection
(let [stream-id (.getSourceStreamId tuple)]
(condp = stream-id
+ Constants/CREDENTIALS_CHANGED_STREAM_ID
+ (let [task-data (get task-datas task-id)
+ bolt-obj (:object task-data)]
+ (when (instance? ICredentialsListener bolt-obj)
+ (.setCredentials bolt-obj (.getValue tuple 0))))
Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data task-datas tuple)
(let [task-data (get task-datas task-id)
^IBolt bolt-obj (:object task-data)
@@ -629,6 +655,9 @@
(.setExecuteSampleStartTime tuple now))
(.execute bolt-obj tuple)
(let [delta (tuple-execute-time-delta! tuple)]
+ (when (= true (storm-conf TOPOLOGY-DEBUG))
+ (log-message "Execute done TUPLE " tuple " TASK: " task-id " DELTA: " delta))
+
(task/apply-hooks user-context .boltExecute (BoltExecuteInfo. tuple task-id delta))
(when delta
(builtin-metrics/bolt-execute-tuple! (:builtin-metrics task-data)
@@ -639,7 +668,16 @@
(stats/bolt-execute-tuple! executor-stats
(.getSourceComponent tuple)
(.getSourceStreamId tuple)
- delta)))))))]
+ delta)))))))
+
+ ;; the overflow buffer is used to ensure that bolts do not block when emitting
+ ;; this ensures that the bolt can always clear the incoming messages, which
+ ;; prevents deadlock from occurs across the topology
+ ;; (e.g. Spout -> BoltA -> Splitter -> BoltB -> BoltA, and all
+ ;; buffers filled up)
+ ;; the overflow buffer is might gradually fill degrading the performance gradually
+ ;; eventually running out of memory, but at least prevent live-locks/deadlocks.
+ overflow-buffer (ConcurrentLinkedQueue.)]
;; TODO: can get any SubscribedState objects out of the context now
@@ -673,9 +711,11 @@
values
task-id
stream
- (MessageId/makeId anchors-to-ids)))))
+ (MessageId/makeId anchors-to-ids))
+ overflow-buffer)))
(or out-tasks [])))]]
(builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
+ (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials))
(if (= component-id Constants/SYSTEM_COMPONENT_ID)
(builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
:receive (:receive-queue executor-data)
@@ -701,9 +741,12 @@
(fast-map-iter [[root id] (.. tuple getMessageId getAnchorsToIds)]
(task/send-unanchored task-data
ACKER-ACK-STREAM-ID
- [root (bit-xor id ack-val)])
+ [root (bit-xor id ack-val)] overflow-buffer)
))
- (let [delta (tuple-time-delta! tuple)]
+ (let [delta (tuple-time-delta! tuple)
+ debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+ (when debug?
+ (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
(task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
(when delta
(builtin-metrics/bolt-acked-tuple! (:builtin-metrics task-data)
@@ -719,8 +762,11 @@
(fast-list-iter [root (.. tuple getMessageId getAnchors)]
(task/send-unanchored task-data
ACKER-FAIL-STREAM-ID
- [root]))
- (let [delta (tuple-time-delta! tuple)]
+ [root] overflow-buffer))
+ (let [delta (tuple-time-delta! tuple)
+ debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+ (when debug?
+ (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
(task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
(when delta
(builtin-metrics/bolt-failed-tuple! (:builtin-metrics task-data)
@@ -743,6 +789,16 @@
(disruptor/consumer-started! receive-queue)
(fn []
(disruptor/consume-batch-when-available receive-queue event-handler)
+ ;; try to clear the overflow-buffer
+ (try-cause
+ (while (not (.isEmpty overflow-buffer))
+ (let [[out-task out-tuple] (.peek overflow-buffer)]
+ (transfer-fn out-task out-tuple false nil)
+ (.poll overflow-buffer)))
+ (catch InsufficientCapacityException e
+ (when (= true (storm-conf TOPOLOGY-DEBUG))
+ (log-message "Insufficient Capacity on queue to emit by bolt " component-id ":" (keys task-datas) ))
+ ))
0)))
:kill-fn (:report-error-and-die executor-data)
:factory? true
diff --git a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
index 2a48f55b5c3..36a737fdbfe 100644
--- a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
@@ -15,38 +15,180 @@
;; limitations under the License.
(ns backtype.storm.daemon.logviewer
(:use compojure.core)
+ (:use [clojure.set :only [difference]])
+ (:use [clojure.string :only [blank?]])
(:use [hiccup core page-helpers])
- (:use [backtype.storm config util log])
+ (:use [backtype.storm config util log timer])
+ (:use [backtype.storm.ui helpers])
(:use [ring.adapter.jetty :only [run-jetty]])
(:import [org.slf4j LoggerFactory])
(:import [ch.qos.logback.classic Logger])
- (:import [org.apache.commons.logging LogFactory])
- (:import [org.apache.commons.logging.impl Log4JLogger])
(:import [ch.qos.logback.core FileAppender])
- (:import [org.apache.log4j Level])
- (:import [java.io File])
+ (:import [java.io File FileFilter FileInputStream])
+ (:import [org.yaml.snakeyaml Yaml]
+ [org.yaml.snakeyaml.constructor SafeConstructor])
+ (:import [backtype.storm.ui InvalidRequestException]
+ [backtype.storm.security.auth AuthUtils])
(:require [compojure.route :as route]
[compojure.handler :as handler]
+ [ring.middleware.keyword-params]
+ [ring.util.response :as resp])
+ (:require [backtype.storm.daemon common [supervisor :as supervisor]])
+ (:import [java.io File FileFilter])
+ (:require [compojure.route :as route]
+ [compojure.handler :as handler]
+ [ring.util.response :as resp]
[clojure.string :as string])
(:gen-class))
-(defn tail-file [path tail root-dir]
- (let [flen (.length (clojure.java.io/file path))
- skip (- flen tail)
- log-dir (.getCanonicalFile (File. root-dir))
- log-file (File. path)]
- (if (= log-dir (.getParentFile log-file))
- (with-open [input (clojure.java.io/input-stream path)
- output (java.io.ByteArrayOutputStream.)]
- (if (> skip 0) (.skip input skip))
- (let [buffer (make-array Byte/TYPE 1024)]
- (loop []
- (let [size (.read input buffer)]
- (when (and (pos? size) (< (.size output) tail))
- (do (.write output buffer 0 size)
- (recur))))))
- (.toString output)) "File not found")
- ))
+(def ^:dynamic *STORM-CONF* (read-storm-config))
+
+(defn cleanup-cutoff-age-millis [conf now-millis]
+ (- now-millis (* (conf LOGVIEWER-CLEANUP-AGE-MINS) 60 1000)))
+
+(defn mk-FileFilter-for-log-cleanup [conf now-millis]
+ (let [cutoff-age-millis (cleanup-cutoff-age-millis conf now-millis)]
+ (reify FileFilter (^boolean accept [this ^File file]
+ (boolean (and
+ (.isFile file)
+ (re-find worker-log-filename-pattern (.getName file))
+ (<= (.lastModified file) cutoff-age-millis)))))))
+
+(defn select-files-for-cleanup [conf now-millis root-dir]
+ (let [file-filter (mk-FileFilter-for-log-cleanup conf now-millis)]
+ (.listFiles (File. root-dir) file-filter)))
+
+(defn get-metadata-file-for-log-root-name [root-name root-dir]
+ (let [metaFile (clojure.java.io/file root-dir "metadata"
+ (str root-name ".yaml"))]
+ (if (.exists metaFile)
+ metaFile
+ (do
+ (log-warn "Could not find " (.getCanonicalPath metaFile)
+ " to clean up for " root-name)
+ nil))))
+
+(defn get-worker-id-from-metadata-file [metaFile]
+ (get (clojure-from-yaml-file metaFile) "worker-id"))
+
+(defn get-topo-owner-from-metadata-file [metaFile]
+ (get (clojure-from-yaml-file metaFile) TOPOLOGY-SUBMITTER-USER))
+
+(defn get-log-root->files-map [log-files]
+ "Returns a map of \"root name\" to a the set of files in log-files having the
+ root name. The \"root name\" of a log file is the part of the name preceding
+ the extension."
+ (reduce #(assoc %1 ;; The accumulated map so far
+ (first %2) ;; key: The root name of the log file
+ (conj (%1 (first %2) #{}) (second %2))) ;; val: The set of log files with the root name
+ {} ;; initial (empty) map
+ (map #(list
+ (second (re-find worker-log-filename-pattern (.getName %))) ;; The root name of the log file
+ %) ;; The log file
+ log-files)))
+
+(defn identify-worker-log-files [log-files root-dir]
+ (into {} (for [log-root-entry (get-log-root->files-map log-files)
+ :let [metaFile (get-metadata-file-for-log-root-name
+ (key log-root-entry) root-dir)
+ log-root (key log-root-entry)
+ files (val log-root-entry)]
+ :when metaFile]
+ {(get-worker-id-from-metadata-file metaFile)
+ {:owner (get-topo-owner-from-metadata-file metaFile)
+ :files
+ ;; If each log for this root name is to be deleted, then
+ ;; include the metadata file also.
+ (if (empty? (difference
+ (set (filter #(re-find (re-pattern log-root) %)
+ (read-dir-contents root-dir)))
+ (set (map #(.getName %) files))))
+ (conj files metaFile)
+ ;; Otherwise, keep the list of files as it is.
+ files)}})))
+
+(defn get-dead-worker-files-and-owners [conf now-secs log-files root-dir]
+ (if (empty? log-files)
+ {}
+ (let [id->heartbeat (supervisor/read-worker-heartbeats conf)
+ alive-ids (keys (remove
+ #(or (not (val %))
+ (supervisor/is-worker-hb-timed-out? now-secs (val %) conf))
+ id->heartbeat))
+ id->entries (identify-worker-log-files log-files root-dir)]
+ (for [[id {:keys [owner files]}] id->entries
+ :when (not (contains? (set alive-ids) id))]
+ {:owner owner
+ :files files}))))
+
+(defn cleanup-fn! [log-root-dir]
+ (let [now-secs (current-time-secs)
+ old-log-files (select-files-for-cleanup *STORM-CONF* (* now-secs 1000) log-root-dir)
+ dead-worker-files (get-dead-worker-files-and-owners *STORM-CONF* now-secs old-log-files log-root-dir)]
+ (log-debug "log cleanup: now(" now-secs
+ ") old log files (" (seq (map #(.getName %) old-log-files))
+ ") dead worker files (" (seq (map #(.getName %) dead-worker-files)) ")")
+ (dofor [{:keys [owner files]} dead-worker-files
+ file files]
+ (let [path (.getCanonicalPath file)]
+ (log-message "Cleaning up: Removing " path)
+ (try
+ (if (or (blank? owner) (re-matches #".*\.yaml$" path))
+ (rmr path)
+ ;; worker-launcher does not actually launch a worker process. It
+ ;; merely executes one of a prescribed set of commands. In this case, we ask it
+ ;; to delete a file as the owner of that file.
+ (supervisor/worker-launcher *STORM-CONF* owner (str "rmr " path)))
+ (catch Exception ex
+ (log-error ex)))))))
+
+(defn start-log-cleaner! [conf log-root-dir]
+ (let [interval-secs (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)]
+ (when interval-secs
+ (log-debug "starting log cleanup thread at interval: " interval-secs)
+ (schedule-recurring (mk-timer :thread-name "logviewer-cleanup")
+ 0 ;; Start immediately.
+ interval-secs
+ (fn [] (cleanup-fn! log-root-dir))))))
+
+(defn page-file
+ ([path tail]
+ (let [flen (.length (clojure.java.io/file path))
+ skip (- flen tail)]
+ (page-file path skip tail)))
+ ([path start length]
+ (with-open [input (FileInputStream. path)
+ output (java.io.ByteArrayOutputStream.)]
+ (if (>= start (.length (clojure.java.io/file path)))
+ (throw
+ (InvalidRequestException. "Cannot start past the end of the file")))
+ (if (> start 0)
+ ;; FileInputStream#skip may not work the first time.
+ (loop [skipped 0]
+ (let [skipped (+ skipped (.skip input (- start skipped)))]
+ (if (< skipped start) (recur skipped)))))
+ (let [buffer (make-array Byte/TYPE 1024)]
+ (loop []
+ (when (< (.size output) length)
+ (let [size (.read input buffer 0 (min 1024 (- length (.size output))))]
+ (when (pos? size)
+ (.write output buffer 0 size)
+ (recur)))))
+ (.toString output)))))
+
+(defn get-log-user-whitelist [fname]
+ (let [wl-file (get-log-metadata-file fname)
+ m (clojure-from-yaml-file wl-file)]
+ (if-let [whitelist (.get m LOGS-USERS)] whitelist [])))
+
+(defn authorized-log-user? [user fname conf]
+ (if (or (blank? user) (blank? fname))
+ nil
+ (let [whitelist (get-log-user-whitelist fname)
+ logs-users (concat (conf LOGS-USERS)
+ (conf NIMBUS-ADMINS)
+ whitelist)]
+ (some #(= % user) logs-users))))
(defn log-root-dir
"Given an appender name, as configured, get the parent directory of the appender's log file.
@@ -59,61 +201,168 @@ Note that if anything goes wrong, this will throw an Error and exit."
(throw
(RuntimeException. "Log viewer could not find configured appender, or the appender is not a FileAppender. Please check that the appender name configured in storm and logback agree.")))))
-(defn log-page [file tail grep root-dir]
- (let [path (.getCanonicalPath (File. root-dir file))
- tail (if tail
- (min 10485760 (Integer/parseInt tail))
- 10240)
- tail-string (tail-file path tail root-dir)]
- (if grep
- (clojure.string/join "\n "
- (filter #(.contains % grep) (.split tail-string "\n")))
- (.replaceAll tail-string "\n" "\n "))))
-
-(defn log-level-page [name level]
- (let [log (LogFactory/getLog name)]
- (if level
- (if (instance? Log4JLogger log)
- (.setLevel (.getLogger log) (Level/toLevel level))))
- (str "effective log level for " name " is " (.getLevel (.getLogger log)))))
-
-(defn log-template [body]
- (html4
- [:head
- [:title "Storm log viewer"]
- (include-css "/css/bootstrap-1.4.0.css")
- (include-css "/css/style.css")
- (include-js "/js/jquery-1.6.2.min.js")
- (include-js "/js/jquery.tablesorter.min.js")
- (include-js "/js/jquery.cookies.2.2.0.min.js")
- (include-js "/js/script.js")
- ]
- [:body
- (seq body)
- ]))
+(defn pager-links [fname start length file-size]
+ (let [prev-start (max 0 (- start length))
+ next-start (if (> file-size 0)
+ (min (max 0 (- file-size length)) (+ start length))
+ (+ start length))]
+ [[:div.pagination
+ [:ul
+ (concat
+ [[(if (< prev-start start) (keyword "li") (keyword "li.disabled"))
+ (link-to (url "/log"
+ {:file fname
+ :start (max 0 (- start length))
+ :length length})
+ "Prev")]]
+ [[:li (link-to
+ (url "/log"
+ {:file fname
+ :start 0
+ :length length})
+ "First")]]
+ [[:li (link-to
+ (url "/log"
+ {:file fname
+ :length length})
+ "Last")]]
+ [[(if (> next-start start) (keyword "li.next") (keyword "li.next.disabled"))
+ (link-to (url "/log"
+ {:file fname
+ :start (min (max 0 (- file-size length))
+ (+ start length))
+ :length length})
+ "Next")]])]]]))
+
+(defn- download-link [fname]
+ [[:p (link-to (url-format "/download/%s" fname) "Download Full Log")]])
+
+(defn log-page [fname start length grep user root-dir]
+ (if (or (blank? (*STORM-CONF* UI-FILTER))
+ (authorized-log-user? user fname *STORM-CONF*))
+ (let [file (.getCanonicalFile (File. root-dir fname))
+ file-length (.length file)
+ path (.getCanonicalPath file)]
+ (if (= (File. root-dir)
+ (.getParentFile file))
+ (let [default-length 51200
+ length (if length
+ (min 10485760 length)
+ default-length)
+ log-string (escape-html
+ (if start
+ (page-file path start length)
+ (page-file path length)))
+ start (or start (- file-length length))]
+ (if grep
+ (html [:pre#logContent
+ (if grep
+ (filter #(.contains % grep)
+ (.split log-string "\n"))
+ log-string)])
+ (let [pager-data (pager-links fname start length file-length)]
+ (html (concat pager-data
+ (download-link fname)
+ [[:pre#logContent log-string]]
+ pager-data)))))
+ (-> (resp/response "Page not found")
+ (resp/status 404))))
+ (unauthorized-user-html user)))
+
+(defn download-log-file [fname req resp user ^String root-dir]
+ (let [file (.getCanonicalFile (File. root-dir fname))]
+ (if (= (File. root-dir) (.getParentFile file))
+ (if (or (blank? (*STORM-CONF* UI-FILTER))
+ (authorized-log-user? user fname *STORM-CONF*))
+ (-> (resp/response file)
+ (resp/content-type "application/octet-stream"))
+ (unauthorized-user-html user))
+ (-> (resp/response "Page not found")
+ (resp/status 404)))))
+
+(defn log-template
+ ([body] (log-template body nil nil))
+ ([body fname user]
+ (html4
+ [:head
+ [:title (str (escape-html fname) " - Storm Log Viewer")]
+ (include-css "/css/bootstrap-1.4.0.css")
+ (include-css "/css/style.css")
+ ]
+ [:body
+ (concat
+ (when (not (blank? user)) [[:div.ui-user [:p "User: " user]]])
+ [[:h3 (escape-html fname)]]
+ (seq body))
+ ])))
+
+(def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*))
+
+(defn- parse-long-from-map [m k]
+ (try
+ (Long/parseLong (k m))
+ (catch NumberFormatException ex
+ (throw (InvalidRequestException.
+ (str "Could not make an integer out of the query parameter '"
+ (name k) "'")
+ ex)))))
(defroutes log-routes
(GET "/log" [:as req & m]
- (log-template (log-page (:file m) (:tail m) (:grep m) (:log-root req))))
- (GET "/loglevel" [:as {cookies :cookies} & m]
- (log-template (log-level-page (:name m) (:level m))))
+ (try
+ (let [servlet-request (:servlet-request req)
+ log-root (:log-root req)
+ user (.getUserName http-creds-handler servlet-request)
+ start (if (:start m) (parse-long-from-map m :start))
+ length (if (:length m) (parse-long-from-map m :length))]
+ (log-template (log-page (:file m) start length (:grep m) user log-root)
+ (:file m) user))
+ (catch InvalidRequestException ex
+ (log-error ex)
+ (ring-response-from-exception ex))))
+ (GET "/download/:file" [:as {:keys [servlet-request servlet-response log-root]} file & m]
+ (try
+ (let [user (.getUserName http-creds-handler servlet-request)]
+ (download-log-file file servlet-request servlet-response user log-root))
+ (catch InvalidRequestException ex
+ (log-error ex)
+ (ring-response-from-exception ex))))
(route/resources "/")
(route/not-found "Page not found"))
-(def logapp
- (handler/site log-routes)
- )
-
(defn conf-middleware
"For passing the storm configuration with each request."
[app log-root]
(fn [req]
(app (assoc req :log-root log-root))))
-(defn start-logviewer [port log-root]
- (run-jetty (conf-middleware logapp log-root) {:port port}))
+(defn start-logviewer! [conf log-root-dir]
+ (try
+ (let [header-buffer-size (int (.get conf UI-HEADER-BUFFER-BYTES))
+ filter-class (conf UI-FILTER)
+ filter-params (conf UI-FILTER-PARAMS)
+ logapp (handler/api log-routes) ;; query params as map
+ middle (conf-middleware logapp log-root-dir)
+ filters-confs (if (conf UI-FILTER)
+ [{:filter-class filter-class
+ :filter-params (or (conf UI-FILTER-PARAMS) {})}]
+ [])
+ filters-confs (concat filters-confs
+ [{:filter-class "org.mortbay.servlet.GzipFilter"
+ :filter-name "Gzipper"
+ :filter-params {}}])]
+ (run-jetty middle
+ {:port (int (conf LOGVIEWER-PORT))
+ :join? false
+ :configurator (fn [server]
+ (doseq [connector (.getConnectors server)]
+ (.setHeaderBufferSize connector header-buffer-size))
+ (config-filter server middle filters-confs))}))
+ (catch Exception ex
+ (log-error ex))))
(defn -main []
(let [conf (read-storm-config)
log-root (log-root-dir (conf LOGVIEWER-APPENDER-NAME))]
- (start-logviewer (int (conf LOGVIEWER-PORT)) log-root)))
+ (start-log-cleaner! conf log-root)
+ (start-logviewer! conf log-root)))
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index bf22a1bcc5c..0bd5337d35e 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -14,19 +14,18 @@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.daemon.nimbus
- (:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
- (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
- (:import [org.apache.thrift.exception])
- (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
(:import [java.nio ByteBuffer])
(:import [java.io FileNotFoundException])
(:import [java.nio.channels Channels WritableByteChannel])
+ (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils])
(:use [backtype.storm.scheduler.DefaultScheduler])
(:import [backtype.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
+ (:import [backtype.storm.generated AuthorizationException])
(:use [backtype.storm bootstrap util])
(:use [backtype.storm.config :only [validate-configs-with-schemas]])
(:use [backtype.storm.daemon common])
+ (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
(:gen-class
:methods [^{:static true} [launch [backtype.storm.scheduler.INimbus] void]]))
@@ -59,13 +58,22 @@
scheduler
))
+(def NIMBUS-ZK-ACLS
+ [(first ZooDefs$Ids/CREATOR_ALL_ACL)
+ (ACL. (bit-or ZooDefs$Perms/READ ZooDefs$Perms/CREATE) ZooDefs$Ids/ANYONE_ID_UNSAFE)])
+
(defn nimbus-data [conf inimbus]
(let [forced-scheduler (.getForcedScheduler inimbus)]
{:conf conf
:inimbus inimbus
+ :authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
:submitted-count (atom 0)
- :storm-cluster-state (cluster/mk-storm-cluster-state conf)
+ :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
+ (Utils/isZkAuthenticationConfiguredStormServer
+ conf)
+ NIMBUS-ZK-ACLS))
:submit-lock (Object.)
+ :cred-update-lock (Object.)
:heartbeats-cache (atom {})
:downloaders (file-cache-map conf)
:uploaders (file-cache-map conf)
@@ -76,6 +84,8 @@
(halt-process! 20 "Error when processing an event")
))
:scheduler (mk-scheduler conf inimbus)
+ :id->sched-status (atom {})
+ :cred-renewers (AuthUtils/GetCredentialRenewers conf)
}))
(defn inbox [nimbus]
@@ -302,6 +312,7 @@
(defn- setup-storm-code [conf storm-id tmp-jar-location storm-conf topology]
(let [stormroot (master-stormdist-root conf storm-id)]
+ (log-message "nimbus file location:" stormroot)
(FileUtils/forceMkdir (File. stormroot))
(FileUtils/cleanDirectory (File. stormroot))
(setup-jar conf tmp-jar-location stormroot)
@@ -590,6 +601,7 @@
new-scheduler-assignments (.getAssignments cluster)
;; add more information to convert SchedulerAssignment to Assignment
new-topology->executor->node+port (compute-topology->executor->node+port new-scheduler-assignments)]
+ (reset! (:id->sched-status nimbus) (.getStatusMap cluster))
;; print some useful information.
(doseq [[topology-id executor->node+port] new-topology->executor->node+port
:let [old-executor->node+port (-> topology-id
@@ -663,6 +675,7 @@
topologies
scratch-topology-id)
+ topology->executor->node+port (merge (into {} (for [id assigned-topology-ids] {id nil})) topology->executor->node+port)
now-secs (current-time-secs)
@@ -725,7 +738,8 @@
(current-time-secs)
{:type topology-initial-status}
(storm-conf TOPOLOGY-WORKERS)
- num-executors))))
+ num-executors
+ (storm-conf TOPOLOGY-SUBMITTER-USER)))))
;; Master:
;; job submit:
@@ -745,6 +759,19 @@
(throw (AlreadyAliveException. (str storm-name " is already active"))))
))
+(defn check-authorization!
+ ([nimbus storm-name storm-conf operation context]
+ (let [aclHandler (:authorization-handler nimbus)
+ ctx (or context (ReqContext/context))
+ check-conf (if storm-conf storm-conf (if storm-name {TOPOLOGY-NAME storm-name}))]
+ (log-message "[req " (.requestID ctx) "] Access from: " (.remoteAddress ctx) " principal:" (.principal ctx) " op:" operation)
+ (if aclHandler
+ (if-not (.permit aclHandler ctx operation check-conf)
+ (throw (AuthorizationException. (str operation (if storm-name (str " on topology " storm-name)) " is not authorized")))
+ ))))
+ ([nimbus storm-name storm-conf operation]
+ (check-authorization! nimbus storm-name storm-conf operation (ReqContext/context))))
+
(defn code-ids [conf]
(-> conf
master-stormdist-root
@@ -830,6 +857,23 @@
(swap! (:heartbeats-cache nimbus) dissoc id))
))))
+(defn renew-credentials [nimbus]
+ (let [storm-cluster-state (:storm-cluster-state nimbus)
+ renewers (:cred-renewers nimbus)
+ update-lock (:cred-update-lock nimbus)
+ assigned-ids (set (.active-storms storm-cluster-state))]
+ (when-not (empty? assigned-ids)
+ (doseq [id assigned-ids]
+ (locking update-lock
+ (let [orig-creds (.credentials storm-cluster-state id nil)]
+ (if orig-creds
+ (let [new-creds (HashMap. orig-creds)]
+ (doseq [renewer renewers]
+ (log-message "Renewing Creds For " id " with " renewer))
+ (when-not (= orig-creds new-creds)
+ (.set-credentials! storm-cluster-state id new-creds)
+ )))))))))
+
(defn- file-older-than? [now seconds file]
(<= (+ (.lastModified file) (to-millis seconds)) (to-millis now)))
@@ -872,26 +916,63 @@
(throw (InvalidTopologyException.
("Topology name cannot be blank"))))))
-(defn- try-read-storm-conf [conf storm-id]
+;; We will only file at //
+;; to be accessed via Thrift
+;; ex., storm-local/nimbus/stormdist/aa-1-1377104853/stormjar.jar
+(defn check-file-access [conf file-path]
+ (log-debug "check file access:" file-path)
+ (try
+ (if (not= (.getCanonicalFile (File. (master-stormdist-root conf)))
+ (-> (File. file-path) .getCanonicalFile .getParentFile .getParentFile))
+ (throw (AuthorizationException. (str "Invalid file path: " file-path))))
+ (catch Exception e
+ (throw (AuthorizationException. (str "Invalid file path: " file-path))))))
+
+(defn try-read-storm-conf [conf storm-id]
(try-cause
(read-storm-conf conf storm-id)
(catch FileNotFoundException e
- (throw (NotAliveException. storm-id)))
+ (throw (NotAliveException. (str storm-id))))
)
)
-(defn- try-read-storm-topology [conf storm-id]
+(defn try-read-storm-conf-from-name [conf storm-name nimbus]
+ (let [storm-cluster-state (:storm-cluster-state nimbus)
+ id (get-storm-id storm-cluster-state storm-name)]
+ (try-read-storm-conf conf id)))
+
+(defn try-read-storm-topology [conf storm-id]
(try-cause
(read-storm-topology conf storm-id)
(catch FileNotFoundException e
- (throw (NotAliveException. storm-id)))
+ (throw (NotAliveException. (str storm-id))))
)
)
+(defn validate-topology-size [topo-conf nimbus-conf topology]
+ (let [workers-count (get topo-conf TOPOLOGY-WORKERS)
+ workers-allowed (get nimbus-conf NIMBUS-SLOTS-PER-TOPOLOGY)
+ num-executors (->> (all-components topology) (map-val num-start-executors))
+ executors-count (reduce + (vals num-executors))
+ executors-allowed (get nimbus-conf NIMBUS-EXECUTORS-PER-TOPOLOGY)]
+ (when (and
+ (not (nil? executors-allowed))
+ (> executors-count executors-allowed))
+ (throw
+ (InvalidTopologyException.
+ (str "Failed to submit topology. Topology requests more than " executors-allowed " executors."))))
+ (when (and
+ (not (nil? workers-allowed))
+ (> workers-count workers-allowed))
+ (throw
+ (InvalidTopologyException.
+ (str "Failed to submit topology. Topology requests more than " workers-allowed " workers."))))))
+
(defserverfn service-handler [conf inimbus]
(.prepare inimbus conf (master-inimbus-dir conf))
(log-message "Starting Nimbus with conf " conf)
- (let [nimbus (nimbus-data conf inimbus)]
+ (let [nimbus (nimbus-data conf inimbus)
+ principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)]
(.prepare ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) conf)
(cleanup-corrupt-topologies! nimbus)
(doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
@@ -912,6 +993,12 @@
(fn []
(clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS))
))
+ (schedule-recurring (:timer nimbus)
+ 0
+ (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS)
+ (fn []
+ (renew-credentials nimbus)))
+
(reify Nimbus$Iface
(^void submitTopologyWithOpts
[this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology
@@ -919,6 +1006,7 @@
(try
(assert (not-nil? submitOptions))
(validate-topology-name! storm-name)
+ (check-authorization! nimbus storm-name nil "submitTopology")
(check-storm-active! nimbus storm-name false)
(let [topo-conf (from-json serializedConf)]
(try
@@ -931,21 +1019,44 @@
topology))
(swap! (:submitted-count nimbus) inc)
(let [storm-id (str storm-name "-" @(:submitted-count nimbus) "-" (current-time-secs))
- storm-conf (normalize-conf
+ credentials (.get_creds submitOptions)
+ credentials (when credentials (.get_creds credentials))
+ topo-conf (from-json serializedConf)
+ storm-conf-submitted (normalize-conf
conf
- (-> serializedConf
- from-json
- (assoc STORM-ID storm-id)
+ (-> topo-conf
+ (assoc STORM-ID storm-id)
(assoc TOPOLOGY-NAME storm-name))
topology)
+ req (ReqContext/context)
+ principal (.principal req)
+ submitter-principal (if principal (.toString principal))
+ submitter-user (.toLocal principal-to-local principal)
+ topo-acl (distinct (remove nil? (conj (.get storm-conf-submitted TOPOLOGY-USERS) submitter-principal, submitter-user)))
+ storm-conf (-> storm-conf-submitted
+ (assoc TOPOLOGY-SUBMITTER-PRINCIPAL (if submitter-principal submitter-principal ""))
+ (assoc TOPOLOGY-SUBMITTER-USER (if submitter-user submitter-user "")) ;Don't let the user set who we launch as
+ (assoc TOPOLOGY-USERS topo-acl)
+ (assoc STORM-ZOOKEEPER-SUPERACL (.get conf STORM-ZOOKEEPER-SUPERACL)))
+ storm-conf (if (Utils/isZkAuthenticationConfiguredStormServer conf)
+ storm-conf
+ (dissoc storm-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
total-storm-conf (merge conf storm-conf)
topology (normalize-topology total-storm-conf topology)
storm-cluster-state (:storm-cluster-state nimbus)]
+ (if (and (conf SUPERVISOR-RUN-WORKER-AS-USER) (or (nil? submitter-user) (.isEmpty (.trim submitter-user))))
+ (throw (AuthorizationException. "Could not determine the user to run this topology as.")))
(system-topology! total-storm-conf topology) ;; this validates the structure of the topology
+ (validate-topology-size topo-conf conf topology)
+ (when (and (Utils/isZkAuthenticationConfiguredStormServer conf)
+ (not (Utils/isZkAuthenticationConfiguredTopology storm-conf)))
+ (throw (IllegalArgumentException. "The cluster is configured for zookeeper authentication, but no payload was provided.")))
(log-message "Received topology submission for " storm-name " with conf " storm-conf)
;; lock protects against multiple topologies being submitted at once and
;; cleanup thread killing topology in b/w assignment and starting the topology
(locking (:submit-lock nimbus)
+ ;;cred-update-lock is not needed here because creds are being added for the first time.
+ (.set-credentials! storm-cluster-state storm-id credentials storm-conf)
(setup-storm-code conf storm-id uploadedJarLocation storm-conf topology)
(.setup-heartbeats! storm-cluster-state storm-id)
(let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive
@@ -962,10 +1073,12 @@
(SubmitOptions. TopologyInitialStatus/ACTIVE)))
(^void killTopology [this ^String name]
- (.killTopologyWithOpts this name (KillOptions.)))
+ (.killTopologyWithOpts this name (KillOptions.)))
(^void killTopologyWithOpts [this ^String storm-name ^KillOptions options]
(check-storm-active! nimbus storm-name true)
+ (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
+ (check-authorization! nimbus storm-name topology-conf "killTopology"))
(let [wait-amt (if (.is_set_wait_secs options)
(.get_wait_secs options)
)]
@@ -974,6 +1087,8 @@
(^void rebalance [this ^String storm-name ^RebalanceOptions options]
(check-storm-active! nimbus storm-name true)
+ (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
+ (check-authorization! nimbus storm-name topology-conf "rebalance"))
(let [wait-amt (if (.is_set_wait_secs options)
(.get_wait_secs options))
num-workers (if (.is_set_num_workers options)
@@ -989,13 +1104,26 @@
))
(activate [this storm-name]
+ (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
+ (check-authorization! nimbus storm-name topology-conf "activate"))
(transition-name! nimbus storm-name :activate true)
)
(deactivate [this storm-name]
+ (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
+ (check-authorization! nimbus storm-name topology-conf "deactivate"))
(transition-name! nimbus storm-name :inactivate true))
+ (uploadNewCredentials [this storm-name credentials]
+ (let [storm-cluster-state (:storm-cluster-state nimbus)
+ storm-id (get-storm-id storm-cluster-state storm-name)
+ topology-conf (try-read-storm-conf conf storm-id)
+ creds (when credentials (.get_creds credentials))]
+ (check-authorization! nimbus storm-name topology-conf "uploadNewCredentials")
+ (locking (:cred-update-lock nimbus) (.set-credentials! storm-cluster-state storm-id creds topology-conf))))
+
(beginFileUpload [this]
+ (check-authorization! nimbus nil nil "fileUpload")
(let [fileloc (str (inbox nimbus) "/stormjar-" (uuid) ".jar")]
(.put (:uploaders nimbus)
fileloc
@@ -1005,6 +1133,7 @@
))
(^void uploadChunk [this ^String location ^ByteBuffer chunk]
+ (check-authorization! nimbus nil nil "fileUpload")
(let [uploaders (:uploaders nimbus)
^WritableByteChannel channel (.get uploaders location)]
(when-not channel
@@ -1015,6 +1144,7 @@
))
(^void finishFileUpload [this ^String location]
+ (check-authorization! nimbus nil nil "fileUpload")
(let [uploaders (:uploaders nimbus)
^WritableByteChannel channel (.get uploaders location)]
(when-not channel
@@ -1026,6 +1156,8 @@
))
(^String beginFileDownload [this ^String file]
+ (check-authorization! nimbus nil nil "fileDownload")
+ (check-file-access (:conf nimbus) file)
(let [is (BufferFileInputStream. file)
id (uuid)]
(.put (:downloaders nimbus) id is)
@@ -1033,6 +1165,7 @@
))
(^ByteBuffer downloadChunk [this ^String id]
+ (check-authorization! nimbus nil nil "fileDownload")
(let [downloaders (:downloaders nimbus)
^BufferFileInputStream is (.get downloaders id)]
(when-not is
@@ -1046,18 +1179,29 @@
)))
(^String getNimbusConf [this]
+ (check-authorization! nimbus nil nil "getNimbusConf")
(to-json (:conf nimbus)))
(^String getTopologyConf [this ^String id]
- (to-json (try-read-storm-conf conf id)))
+ (let [topology-conf (try-read-storm-conf conf id)
+ storm-name (topology-conf TOPOLOGY-NAME)]
+ (check-authorization! nimbus storm-name topology-conf "getTopologyConf")
+ (to-json topology-conf)))
(^StormTopology getTopology [this ^String id]
- (system-topology! (try-read-storm-conf conf id) (try-read-storm-topology conf id)))
+ (let [topology-conf (try-read-storm-conf conf id)
+ storm-name (topology-conf TOPOLOGY-NAME)]
+ (check-authorization! nimbus storm-name topology-conf "getTopology")
+ (system-topology! topology-conf (try-read-storm-topology conf id))))
(^StormTopology getUserTopology [this ^String id]
- (try-read-storm-topology conf id))
+ (let [topology-conf (try-read-storm-conf conf id)
+ storm-name (topology-conf TOPOLOGY-NAME)]
+ (check-authorization! nimbus storm-name topology-conf "getUserTopology")
+ (try-read-storm-topology topology-conf id)))
(^ClusterSummary getClusterInfo [this]
+ (check-authorization! nimbus nil nil "getClusterInfo")
(let [storm-cluster-state (:storm-cluster-state nimbus)
supervisor-infos (all-supervisor-info storm-cluster-state)
;; TODO: need to get the port info about supervisors...
@@ -1073,9 +1217,9 @@
))
nimbus-uptime ((:uptime nimbus))
bases (topology-bases storm-cluster-state)
- topology-summaries (dofor [[id base] bases]
- (let [assignment (.assignment-info storm-cluster-state id nil)]
- (TopologySummary. id
+ topology-summaries (dofor [[id base] bases :when base]
+ (let [assignment (.assignment-info storm-cluster-state id nil)
+ topo-summ (TopologySummary. id
(:storm-name base)
(->> (:executor->node+port assignment)
keys
@@ -1089,7 +1233,10 @@
set
count)
(time-delta (:launch-time-secs base))
- (extract-status-str base))
+ (extract-status-str base))]
+ (when-let [owner (:owner base)] (.set_owner topo-summ owner))
+ (when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
+ topo-summ
))]
(ClusterSummary. supervisor-summaries
nimbus-uptime
@@ -1098,8 +1245,14 @@
(^TopologyInfo getTopologyInfo [this ^String storm-id]
(let [storm-cluster-state (:storm-cluster-state nimbus)
- task->component (storm-task-info (try-read-storm-topology conf storm-id) (try-read-storm-conf conf storm-id))
+ topology-conf (try-read-storm-conf conf storm-id)
+ storm-name (topology-conf TOPOLOGY-NAME)
+ _ (check-authorization! nimbus storm-name topology-conf "getTopologyInfo")
+ task->component (storm-task-info (try-read-storm-topology conf storm-id) topology-conf)
base (.storm-base storm-cluster-state storm-id nil)
+ storm-name (if base (:storm-name base) (throw (NotAliveException. (str storm-id))))
+ launch-time-secs (if base (:launch-time-secs base) (throw (NotAliveException. (str storm-id))))
+ task->component (storm-task-info (try-read-storm-topology conf storm-id) topology-conf)
assignment (.assignment-info storm-cluster-state storm-id nil)
beats (.executor-beats storm-cluster-state storm-id (:executor->node+port assignment))
all-components (-> task->component reverse-map keys)
@@ -1120,14 +1273,16 @@
(nil-to-zero (:uptime heartbeat)))
(.set_stats stats))
))
- ]
- (TopologyInfo. storm-id
- (:storm-name base)
- (time-delta (:launch-time-secs base))
- executor-summaries
- (extract-status-str base)
- errors
- )
+ topo-info (TopologyInfo. storm-id
+ storm-name
+ (time-delta launch-time-secs)
+ executor-summaries
+ (extract-status-str base)
+ errors
+ )]
+ (when-let [owner (:owner base)] (.set_owner topo-info owner))
+ (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))
+ topo-info
))
Shutdownable
@@ -1146,16 +1301,13 @@
(defn launch-server! [conf nimbus]
(validate-distributed-mode! conf)
(let [service-handler (service-handler conf nimbus)
- options (-> (TNonblockingServerSocket. (int (conf NIMBUS-THRIFT-PORT)))
- (THsHaServer$Args.)
- (.workerThreads 64)
- (.protocolFactory (TBinaryProtocol$Factory. false true (conf NIMBUS-THRIFT-MAX-BUFFER-SIZE)))
- (.processor (Nimbus$Processor. service-handler))
- )
- server (THsHaServer. (do (set! (. options maxReadBufferBytes)(conf NIMBUS-THRIFT-MAX-BUFFER-SIZE)) options))]
+ ;;TODO need to honor NIMBUS-THRIFT-MAX-BUFFER-SIZE for different transports
+ server (ThriftServer. conf (Nimbus$Processor. service-handler)
+ ThriftConnectionType/NIMBUS)]
(.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.shutdown service-handler) (.stop server))))
(log-message "Starting Nimbus server...")
- (.serve server)))
+ (.serve server)
+ service-handler))
;; distributed implementation
@@ -1175,7 +1327,10 @@
)
(defn -launch [nimbus]
- (launch-server! (read-storm-config) nimbus))
+ (let [conf (merge
+ (read-storm-config)
+ (read-yaml-config "storm-cluster-auth.yaml" false))]
+ (launch-server! conf nimbus)))
(defn standalone-nimbus []
(reify INimbus
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 7566a7957c1..1f8e7e14dad 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -14,10 +14,14 @@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns backtype.storm.daemon.supervisor
+ (:import [java.io OutputStreamWriter BufferedWriter IOException])
(:import [backtype.storm.scheduler ISupervisor])
(:use [backtype.storm bootstrap])
(:use [backtype.storm.daemon common])
(:require [backtype.storm.daemon [worker :as worker]])
+ (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
+ (:import [org.yaml.snakeyaml Yaml]
+ [org.yaml.snakeyaml.constructor SafeConstructor])
(:gen-class
:methods [^{:static true} [launch [backtype.storm.scheduler.ISupervisor] void]]))
@@ -62,7 +66,7 @@
"Returns map from port to struct containing :storm-id and :executors"
[assignments-snapshot assignment-id]
(->> (dofor [sid (keys assignments-snapshot)] (read-my-executors assignments-snapshot sid assignment-id))
- (apply merge-with (fn [& ignored] (throw-runtime "Should not have multiple topologies assigned to one port")))))
+ (apply merge-with (fn [& params] (throw-runtime (str "Should not have multiple topologies assigned to one port " params))))))
(defn- read-storm-code-locations
[assignments-snapshot]
@@ -98,6 +102,18 @@
(= (disj (set (:executors worker-heartbeat)) Constants/SYSTEM_EXECUTOR_ID)
(set (:executors local-assignment))))))
+(let [dead-workers (atom #{})]
+ (defn get-dead-workers []
+ @dead-workers)
+ (defn add-dead-worker [worker]
+ (swap! dead-workers conj worker))
+ (defn remove-dead-worker [worker]
+ (swap! dead-workers disj worker)))
+
+(defn is-worker-hb-timed-out? [now hb conf]
+ (> (- now (:time-secs hb))
+ (conf SUPERVISOR-WORKER-TIMEOUT-SECS)))
+
(defn read-allocated-workers
"Returns map from worker id to worker heartbeat. if the heartbeat is nil, then the worker is dead (timed out or never wrote heartbeat)"
[supervisor assigned-executors now]
@@ -114,8 +130,11 @@
(or (not (contains? approved-ids id))
(not (matches-an-assignment? hb assigned-executors)))
:disallowed
- (> (- now (:time-secs hb))
- (conf SUPERVISOR-WORKER-TIMEOUT-SECS))
+ (or
+ (when (get (get-dead-workers) id)
+ (log-message "Worker Process " id " has died!")
+ true)
+ (is-worker-hb-timed-out? now hb conf))
:timed-out
true
:valid)]
@@ -151,35 +170,76 @@
(defn generate-supervisor-id []
(uuid))
-(defn try-cleanup-worker [conf id]
+(defnk worker-launcher [conf user args :environment {} :log-prefix nil :exit-code-callback nil]
+ (let [_ (when (clojure.string/blank? user)
+ (throw (java.lang.IllegalArgumentException.
+ "User cannot be blank when calling worker-launcher.")))
+ wl-initial (conf SUPERVISOR-WORKER-LAUNCHER)
+ storm-home (System/getProperty "storm.home")
+ wl (if wl-initial wl-initial (str storm-home "/bin/worker-launcher"))
+ command (concat [wl user] args)]
+ (log-message "Running as user:" user " command:" (pr-str command))
+ (launch-process command :environment environment :log-prefix log-prefix :exit-code-callback exit-code-callback)
+ ))
+
+(defnk worker-launcher-and-wait [conf user args :environment {} :log-prefix nil]
+ (let [process (worker-launcher conf user args :environment environment)]
+ (if log-prefix
+ (read-and-log-stream log-prefix (.getInputStream process)))
+ (try
+ (.waitFor process)
+ (catch InterruptedException e
+ (log-message log-prefix " interrupted.")))
+ (.exitValue process)))
+
+(defn try-cleanup-worker [conf id user]
(try
- (rmr (worker-heartbeats-root conf id))
- ;; this avoids a race condition with worker or subprocess writing pid around same time
- (rmpath (worker-pids-root conf id))
- (rmpath (worker-root conf id))
+ (if (.exists (File. (worker-root conf id)))
+ (do
+ (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
+ (worker-launcher-and-wait conf user ["rmr" (worker-root conf id)] :log-prefix (str "rmr " id) )
+ (do
+ (rmr (worker-heartbeats-root conf id))
+ ;; this avoids a race condition with worker or subprocess writing pid around same time
+ (rmpath (worker-pids-root conf id))
+ (rmpath (worker-root conf id))))
+ (remove-worker-user! conf id)
+ (remove-dead-worker id)
+ ))
+ (catch IOException e
+ (log-warn-error e "Failed to cleanup worker " id ". Will retry later"))
(catch RuntimeException e
(log-warn-error e "Failed to cleanup worker " id ". Will retry later")
)
(catch java.io.FileNotFoundException e (log-message (.getMessage e)))
- (catch java.io.IOException e (log-message (.getMessage e)))
))
(defn shutdown-worker [supervisor id]
(log-message "Shutting down " (:supervisor-id supervisor) ":" id)
(let [conf (:conf supervisor)
pids (read-dir-contents (worker-pids-root conf id))
- thread-pid (@(:worker-thread-pids-atom supervisor) id)]
+ thread-pid (@(:worker-thread-pids-atom supervisor) id)
+ as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)
+ user (get-worker-user conf id)]
(when thread-pid
(psim/kill-process thread-pid))
(doseq [pid pids]
- (ensure-process-killed! pid)
- (try
- (rmpath (worker-pid-path conf id pid))
- (catch Exception e)) ;; on windows, the supervisor may still holds the lock on the worker directory
- )
- (try-cleanup-worker conf id))
+ (if as-user
+ (worker-launcher-and-wait conf user ["signal" pid "9"] :log-prefix (str "kill -9 " pid))
+ (ensure-process-killed! pid))
+ (if as-user
+ (worker-launcher-and-wait conf user ["rmr" (worker-pid-path conf id pid)] :log-prefix (str "rmr for " pid))
+ (try
+ (rmpath (worker-pid-path conf id pid))
+ (catch Exception e)) ;; on windows, the supervisor may still holds the lock on the worker directory
+ ))
+ (try-cleanup-worker conf id user))
(log-message "Shut down " (:supervisor-id supervisor) ":" id))
+(def SUPERVISOR-ZK-ACLS
+ [(first ZooDefs$Ids/CREATOR_ALL_ACL)
+ (ACL. (bit-or ZooDefs$Perms/READ ZooDefs$Perms/CREATE) ZooDefs$Ids/ANYONE_ID_UNSAFE)])
+
(defn supervisor-data [conf shared-context ^ISupervisor isupervisor]
{:conf conf
:shared-context shared-context
@@ -187,7 +247,10 @@
:active (atom true)
:uptime (uptime-computer)
:worker-thread-pids-atom (atom {})
- :storm-cluster-state (cluster/mk-storm-cluster-state conf)
+ :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
+ (Utils/isZkAuthenticationConfiguredStormServer
+ conf)
+ SUPERVISOR-ZK-ACLS))
:local-state (supervisor-state conf)
:supervisor-id (.getSupervisorId isupervisor)
:assignment-id (.getAssignmentId isupervisor)
@@ -240,7 +303,8 @@
(shutdown-worker supervisor id)
))
(doseq [id (vals new-worker-ids)]
- (local-mkdirs (worker-pids-root conf id)))
+ (local-mkdirs (worker-pids-root conf id))
+ (local-mkdirs (worker-heartbeats-root conf id)))
(.put local-state LS-APPROVED-WORKERS
(merge
(select-keys (.get local-state LS-APPROVED-WORKERS)
@@ -422,8 +486,11 @@
(.shutdown supervisor)
)
-;; distributed implementation
+(defn setup-storm-code-dir [conf storm-conf dir]
+ (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
+ (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
+;; distributed implementation
(defmethod download-storm-code
:distributed [conf storm-id master-code-dir]
;; Downloading to permanent location is atomic
@@ -436,7 +503,31 @@
(Utils/downloadFromMaster conf (master-stormconf-path master-code-dir) (supervisor-stormconf-path tmproot))
(extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
(FileUtils/moveDirectory (File. tmproot) (File. stormroot))
- ))
+ (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)
+ ))
+
+(defn write-log-metadata-to-yaml-file! [storm-id port data conf]
+ (let [file (get-log-metadata-file storm-id port)]
+ ;;run worker as user needs the directory to have special permissions
+ ;; or it is insecure
+ (when (and (not (conf SUPERVISOR-RUN-WORKER-AS-USER))
+ (not (.exists (.getParentFile file))))
+ (.mkdirs (.getParentFile file)))
+ (let [writer (java.io.FileWriter. file)
+ yaml (Yaml.)]
+ (try
+ (.dump yaml data writer)
+ (finally
+ (.close writer))))))
+
+(defn write-log-metadata! [storm-conf user worker-id storm-id port conf]
+ (let [data {TOPOLOGY-SUBMITTER-USER user
+ "worker-id" worker-id
+ LOGS-USERS (sort (distinct (remove nil?
+ (concat
+ (storm-conf LOGS-USERS)
+ (storm-conf TOPOLOGY-USERS)))))}]
+ (write-log-metadata-to-yaml-file! storm-id port data conf)))
(defn jlp [stormroot conf]
(let [resource-root (str stormroot File/separator RESOURCES-SUBDIR)
@@ -445,11 +536,23 @@
arch-resource-root (str resource-root File/separator os "-" arch)]
(str arch-resource-root File/pathSeparator resource-root File/pathSeparator (conf JAVA-LIBRARY-PATH))))
-(defn- substitute-worker-childopts [value port]
- (let [sub-fn (fn [s] (.replaceAll s "%ID%" (str port)))]
- (if (list? value)
- (map sub-fn value)
- (-> value sub-fn (.split " ")))))
+(defn substitute-childopts
+ [childopts worker-id storm-id port]
+ (
+ let [replacement-map {"%ID%" (str port)
+ "%WORKER-ID%" (str worker-id)
+ "%STORM-ID%" (str storm-id)
+ "%WORKER-PORT%" (str port)}
+ sub-fn (fn [s]
+ (reduce (fn [string entry]
+ (apply clojure.string/replace string entry))
+ s replacement-map))]
+ (if-not (nil? childopts)
+ (if (sequential? childopts)
+ (map sub-fn childopts)
+ (-> childopts sub-fn (.split " ")))
+ nil)
+ ))
(defn java-cmd []
(let [java-home (.get (System/getenv) "JAVA_HOME")]
@@ -462,21 +565,25 @@
(defmethod launch-worker
:distributed [supervisor storm-id port worker-id]
(let [conf (:conf supervisor)
+ run-worker-as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)
storm-home (System/getProperty "storm.home")
stormroot (supervisor-stormdist-root conf storm-id)
jlp (jlp stormroot conf)
stormjar (supervisor-stormjar-path stormroot)
storm-conf (read-supervisor-storm-conf conf storm-id)
classpath (add-to-classpath (current-classpath) [stormjar])
- worker-childopts (when-let [s (conf WORKER-CHILDOPTS)]
- (substitute-worker-childopts s port))
- topo-worker-childopts (when-let [s (storm-conf TOPOLOGY-WORKER-CHILDOPTS)]
- (substitute-worker-childopts s port))
- logfilename (str "worker-" port ".log")
+ top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
+ gc-opts (substitute-childopts (if top-gc-opts top-gc-opts (conf WORKER-GC-CHILDOPTS)) worker-id storm-id port)
+ user (storm-conf TOPOLOGY-SUBMITTER-USER)
+ logfilename (logs-filename storm-id port)
+
+ worker-childopts (substitute-childopts (conf WORKER-CHILDOPTS) worker-id storm-id port)
+ topo-worker-childopts (substitute-childopts (storm-conf TOPOLOGY-WORKER-CHILDOPTS) worker-id storm-id port)
command (concat
[(java-cmd) "-server"]
worker-childopts
topo-worker-childopts
+ gc-opts
[(str "-Djava.library.path=" jlp)
(str "-Dlogfile.name=" logfilename)
(str "-Dstorm.home=" storm-home)
@@ -490,13 +597,21 @@
(:assignment-id supervisor)
port
worker-id])
- command (->> command (map str) (filter (complement empty?)))
- shell-cmd (->> command
- (map #(str \' (clojure.string/escape % {\' "\\'"}) \'))
- (clojure.string/join " "))]
- (log-message "Launching worker with command: " shell-cmd)
- (launch-process command :environment {"LD_LIBRARY_PATH" jlp})
- ))
+ command (->> command (map str) (filter (complement empty?)))]
+
+ (log-message "Launching worker with command: " (shell-cmd command))
+ (write-log-metadata! storm-conf user worker-id storm-id port conf)
+ (set-worker-user! conf worker-id user)
+ (let [log-prefix (str "Worker Process " worker-id)
+ callback (fn [exit-code]
+ (log-message log-prefix " exited with code: " exit-code)
+ (add-dead-worker worker-id))]
+ (remove-dead-worker worker-id)
+ (if run-worker-as-user
+ (let [worker-dir (worker-root conf worker-id)]
+ (worker-launcher conf user ["worker" worker-dir (write-script worker-dir command :environment {"LD_LIBRARY_PATH" jlp})] :log-prefix log-prefix :exit-code-callback callback))
+ (launch-process command :environment {"LD_LIBRARY_PATH" jlp} :log-prefix log-prefix :exit-code-callback callback)
+ ))))
;; local implementation
@@ -536,6 +651,7 @@
(:assignment-id supervisor)
port
worker-id)]
+ (set-worker-user! conf worker-id "")
(psim/register-process pid worker)
(swap! (:worker-thread-pids-atom supervisor) assoc worker-id pid)
))
diff --git a/storm-core/src/clj/backtype/storm/daemon/task.clj b/storm-core/src/clj/backtype/storm/daemon/task.clj
index 36501506d78..6a61cea6fad 100644
--- a/storm-core/src/clj/backtype/storm/daemon/task.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/task.clj
@@ -126,12 +126,9 @@
emit-sampler (mk-stats-sampler storm-conf)
stream->component->grouper (:stream->component->grouper executor-data)
user-context (:user-context task-data)
- executor-stats (:stats executor-data)
- debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+ executor-stats (:stats executor-data)]
(fn ([^Integer out-task-id ^String stream ^List values]
- (when debug?
- (log-message "Emitting direct: " out-task-id "; " component-id " " stream " " values))
(let [target-component (.getComponentId worker-context out-task-id)
component->grouping (get stream->component->grouper stream)
grouping (get component->grouping target-component)
@@ -148,8 +145,6 @@
(if out-task-id [out-task-id])
))
([^String stream ^List values]
- (when debug?
- (log-message "Emitting: " component-id " " stream " " values))
(let [out-tasks (ArrayList.)]
(fast-map-iter [[out-component grouper] (get stream->component->grouper stream)]
(when (= :direct grouper)
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 921c259b2f8..265ed4b1006 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -20,6 +20,9 @@
(:import [java.util.concurrent Executors])
(:import [backtype.storm.messaging TransportFactory])
(:import [backtype.storm.messaging IContext IConnection])
+ (:import [backtype.storm.security.auth AuthUtils])
+ (:import [javax.security.auth Subject])
+ (:import [java.security PrivilegedExceptionAction])
(:gen-class))
(bootstrap)
@@ -106,23 +109,37 @@
(log-warn "Received invalid messages for unknown tasks. Dropping... ")
)))))))
+(defn- assert-can-serialize [^KryoTupleSerializer serializer tuple-batch]
+ "Check that all of the tuples can be serialized by serializing them."
+ (fast-list-iter [[task tuple :as pair] tuple-batch]
+ (.serialize serializer tuple)))
+
(defn mk-transfer-fn [worker]
(let [local-tasks (-> worker :task-ids set)
local-transfer (:transfer-local-fn worker)
- ^DisruptorQueue transfer-queue (:transfer-queue worker)]
- (fn [^KryoTupleSerializer serializer tuple-batch]
- (let [local (ArrayList.)
- remote (ArrayList.)]
- (fast-list-iter [[task tuple :as pair] tuple-batch]
- (if (local-tasks task)
- (.add local pair)
- (.add remote pair)
- ))
- (local-transfer local)
- ;; not using map because the lazy seq shows up in perf profiles
- (let [serialized-pairs (fast-list-for [[task ^TupleImpl tuple] remote] [task (.serialize serializer tuple)])]
- (disruptor/publish transfer-queue serialized-pairs)
- )))))
+ ^DisruptorQueue transfer-queue (:transfer-queue worker)
+ try-serialize-local ((:conf worker) TOPOLOGY-TESTING-ALWAYS-TRY-SERIALIZE)
+ transfer-fn
+ (fn [^KryoTupleSerializer serializer tuple-batch]
+ (let [local (ArrayList.)
+ remote (ArrayList.)]
+ (fast-list-iter [[task tuple :as pair] tuple-batch]
+ (if (local-tasks task)
+ (.add local pair)
+ (.add remote pair)
+ ))
+ (local-transfer local)
+ ;; not using map because the lazy seq shows up in perf profiles
+ (let [serialized-pairs (fast-list-for [[task ^TupleImpl tuple] remote] [task (.serialize serializer tuple)])]
+ (disruptor/publish transfer-queue serialized-pairs)
+ )))]
+ (if try-serialize-local
+ (do
+ (log-warn "WILL TRY TO SERIALIZE ALL TUPLES (Turn off " TOPOLOGY-TESTING-ALWAYS-TRY-SERIALIZE " for production)")
+ (fn [^KryoTupleSerializer serializer tuple-batch]
+ (assert-can-serialize serializer tuple-batch)
+ (transfer-fn serializer tuple-batch)))
+ transfer-fn)))
(defn- mk-receive-queue-map [storm-conf executors]
(->> executors
@@ -164,21 +181,17 @@
(halt-process! 20 "Error when processing an event")
)))
-(defn worker-data [conf mq-context storm-id assignment-id port worker-id]
- (let [cluster-state (cluster/mk-distributed-cluster-state conf)
- storm-cluster-state (cluster/mk-storm-cluster-state cluster-state)
- storm-conf (read-supervisor-storm-conf conf storm-id)
- executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port))
- transfer-queue (disruptor/disruptor-queue (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
- :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
- executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
-
- receive-queue-map (->> executor-receive-queue-map
- (mapcat (fn [[e queue]] (for [t (executor-id->tasks e)] [t queue])))
- (into {}))
-
- topology (read-supervisor-topology conf storm-id)]
- (recursive-map
+(defn recursive-map-worker-data [conf mq-context storm-id assignment-id port
+ storm-conf
+ worker-id
+ cluster-state
+ storm-cluster-state
+ executors
+ transfer-queue
+ executor-receive-queue-map
+ receive-queue-map
+ topology]
+ (recursive-map
:conf conf
:mq-context (if mq-context
mq-context
@@ -197,6 +210,7 @@
:system-topology (system-topology! storm-conf topology)
:heartbeat-timer (mk-halting-timer)
:refresh-connections-timer (mk-halting-timer)
+ :refresh-credentials-timer (mk-halting-timer)
:refresh-active-timer (mk-halting-timer)
:executor-heartbeat-timer (mk-halting-timer)
:user-timer (mk-halting-timer)
@@ -219,7 +233,31 @@
:user-shared-resources (mk-user-resources <>)
:transfer-local-fn (mk-transfer-local-fn <>)
:transfer-fn (mk-transfer-fn <>)
- )))
+ ))
+
+(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state]
+ (let [executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port))
+ transfer-queue (disruptor/disruptor-queue (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
+ :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
+ executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
+
+ receive-queue-map (->> executor-receive-queue-map
+ (mapcat (fn [[e queue]] (for [t (executor-id->tasks e)] [t queue])))
+ (into {}))
+
+ topology (read-supervisor-topology conf storm-id)]
+ (recursive-map-worker-data
+ conf mq-context storm-id assignment-id port
+ storm-conf
+ worker-id
+ cluster-state
+ storm-cluster-state
+ executors
+ transfer-queue
+ executor-receive-queue-map
+ receive-queue-map
+ topology)
+ ))
(defn- endpoint->string [[node port]]
(str port "/" node))
@@ -336,6 +374,11 @@
(.shutdownNow (get dr WorkerTopologyContext/SHARED_EXECUTOR))
(log-message "Shut down default resources")))
+(defn- override-login-config-with-system-property [conf]
+ (if-let [login_conf_file (System/getProperty "java.security.auth.login.config")]
+ (assoc conf "java.security.auth.login.config" login_conf_file)
+ conf))
+
;; TODO: should worker even take the storm-id as input? this should be
;; deducable from cluster state (by searching through assignments)
;; what about if there's inconsistency in assignments? -> but nimbus
@@ -350,7 +393,17 @@
;; process. supervisor will register it in this case
(when (= :distributed (cluster-mode conf))
(touch (worker-pid-path conf worker-id (process-pid))))
- (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id)
+ (let [storm-conf (read-supervisor-storm-conf conf storm-id)
+ storm-conf (override-login-config-with-system-property storm-conf)
+ acls (Utils/getWorkerACL storm-conf)
+ cluster-state (cluster/mk-distributed-cluster-state conf :auth-conf storm-conf :acls acls)
+ storm-cluster-state (cluster/mk-storm-cluster-state cluster-state :acls acls)
+ initial-credentials (.credentials storm-cluster-state storm-id nil)
+ auto-creds (AuthUtils/GetAutoCredentials storm-conf)
+ subject (AuthUtils/populateSubject nil auto-creds initial-credentials)]
+ (Subject/doAs subject (reify PrivilegedExceptionAction
+ (run [this]
+ (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state)
heartbeat-fn #(do-heartbeat worker)
;; do this here so that the worker process dies if this fails
@@ -363,13 +416,12 @@
_ (schedule-recurring (:heartbeat-timer worker) 0 (conf WORKER-HEARTBEAT-FREQUENCY-SECS) heartbeat-fn)
_ (schedule-recurring (:executor-heartbeat-timer worker) 0 (conf TASK-HEARTBEAT-FREQUENCY-SECS) #(do-executor-heartbeats worker :executors @executors))
-
refresh-connections (mk-refresh-connections worker)
_ (refresh-connections nil)
_ (refresh-storm-active worker nil)
- _ (reset! executors (dofor [e (:executors worker)] (executor/mk-executor worker e)))
+ _ (reset! executors (dofor [e (:executors worker)] (executor/mk-executor worker e initial-credentials)))
receive-thread-shutdown (launch-receive-thread worker)
transfer-tuples (mk-transfer-tuples-handler worker)
@@ -400,6 +452,7 @@
(log-message "Shut down transfer thread")
(cancel-timer (:heartbeat-timer worker))
(cancel-timer (:refresh-connections-timer worker))
+ (cancel-timer (:refresh-credentials-timer worker))
(cancel-timer (:refresh-active-timer worker))
(cancel-timer (:executor-heartbeat-timer worker))
(cancel-timer (:user-timer worker))
@@ -423,19 +476,29 @@
(and
(timer-waiting? (:heartbeat-timer worker))
(timer-waiting? (:refresh-connections-timer worker))
+ (timer-waiting? (:refresh-credentials-timer worker))
(timer-waiting? (:refresh-active-timer worker))
(timer-waiting? (:executor-heartbeat-timer worker))
(timer-waiting? (:user-timer worker))
))
- )]
-
+ )
+ credentials (atom initial-credentials)
+ check-credentials-changed (fn []
+ (let [new-creds (.credentials (:storm-cluster-state worker) storm-id nil)]
+ (when-not (= new-creds @credentials) ;;This does not have to be atomic, worst case we update when one is not needed
+ (AuthUtils/updateSubject subject auto-creds new-creds)
+ (dofor [e @executors] (.credentials-changed e new-creds))
+ (reset! credentials new-creds))))
+ ]
+ (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed)))
+ (schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS) check-credentials-changed)
(schedule-recurring (:refresh-connections-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) refresh-connections)
(schedule-recurring (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker))
(log-message "Worker has topology config " (:storm-conf worker))
(log-message "Worker " worker-id " for storm " storm-id " on " assignment-id ":" port " has finished loading")
ret
- ))
+ ))))))
(defmethod mk-suicide-fn
:local [conf]
diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index 7bbe23844eb..ec4de87ff0f 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -188,10 +188,16 @@
(defmacro while-timeout [timeout-ms condition & body]
`(let [end-time# (+ (System/currentTimeMillis) ~timeout-ms)]
+ (log-debug "Looping until " '~condition)
(while ~condition
(when (> (System/currentTimeMillis) end-time#)
- (throw (AssertionError. (str "Test timed out (" ~timeout-ms "ms)"))))
- ~@body)))
+ (let [thread-dump# (Utils/threadDump)]
+ (log-message "Condition " '~condition " not met in " ~timeout-ms "ms")
+ (log-message thread-dump#)
+ (throw (AssertionError. (str "Test timed out (" ~timeout-ms "ms) " '~condition)))))
+ ~@body)
+ (log-debug "Condition met " '~condition)
+ ))
(defn wait-until-cluster-waiting
"Wait until the cluster is idle. Should be used with time simulation."
@@ -204,8 +210,8 @@
supervisors
workers) ; because a worker may already be dead
]
- (while-timeout TEST-TIMEOUT-MS (not (every? (memfn waiting?) daemons))
- (Thread/sleep 10)
+ (while-timeout TEST-TIMEOUT-MS (not (every? (memfn waiting?) daemons))
+ (Thread/sleep (rand-int 20))
;; (doseq [d daemons]
;; (if-not ((memfn waiting?) d)
;; (println d)))
@@ -280,7 +286,9 @@
(defn mk-capture-launch-fn [capture-atom]
(fn [supervisor storm-id port worker-id]
(let [supervisor-id (:supervisor-id supervisor)
+ conf (:conf supervisor)
existing (get @capture-atom [supervisor-id port] [])]
+ (set-worker-user! conf worker-id "")
(swap! capture-atom assoc [supervisor-id port] (conj existing storm-id))
)))
@@ -471,6 +479,9 @@
(let [storm-id (common/get-storm-id state storm-name)]
+ ;;Give the topology time to come up without using it to wait for the spouts to complete
+ (simulate-wait cluster-map)
+
(while-timeout TEST-TIMEOUT-MS (not (every? exhausted? (spout-objects spouts)))
(simulate-wait cluster-map))
@@ -586,7 +597,7 @@
;; (println "Spout emitted: " (global-amt track-id "spout-emitted"))
;; (println "Processed: " (global-amt track-id "processed"))
;; (println "Transferred: " (global-amt track-id "transferred"))
- (Thread/sleep 500))
+ (Thread/sleep (rand-int 200)))
(reset! (:last-spout-emit tracked-topology) target)
)))
@@ -620,3 +631,9 @@
(atom false))]
(TupleImpl. context values 1 stream)
))
+
+(defmacro with-timeout [millis unit & body]
+ `(let [f# (future ~@body)]
+ (try
+ (.get f# ~millis ~unit)
+ (finally (future-cancel f#)))))
diff --git a/storm-core/src/clj/backtype/storm/testing4j.clj b/storm-core/src/clj/backtype/storm/testing4j.clj
index 5d44604dde4..d5f6550c9fa 100644
--- a/storm-core/src/clj/backtype/storm/testing4j.clj
+++ b/storm-core/src/clj/backtype/storm/testing4j.clj
@@ -15,7 +15,7 @@
;; limitations under the License.
(ns backtype.storm.testing4j
(:import [java.util Map List Collection ArrayList])
- (:require [backtype.storm LocalCluster])
+ (:require [backtype.storm [LocalCluster :as LocalCluster]])
(:import [backtype.storm Config ILocalCluster LocalCluster])
(:import [backtype.storm.generated StormTopology])
(:import [backtype.storm.daemon nimbus])
@@ -151,4 +151,4 @@
(let [stream (or (.getStream param) Utils/DEFAULT_STREAM_ID)
component (or (.getComponent param) "component")
fields (.getFields param)]
- (test-tuple values :stream stream :component component :fields fields)))))
\ No newline at end of file
+ (test-tuple values :stream stream :component component :fields fields)))))
diff --git a/storm-core/src/clj/backtype/storm/thrift.clj b/storm-core/src/clj/backtype/storm/thrift.clj
index ec8ce5d35b6..c2eb0c73617 100644
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@ -18,13 +18,12 @@
(:import [backtype.storm.generated JavaObject Grouping Nimbus StormTopology StormTopology$_Fields
Bolt Nimbus$Client Nimbus$Iface ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
GlobalStreamId ComponentObject ComponentObject$_Fields ShellComponent])
- (:import [backtype.storm.utils Utils])
+ (:import [backtype.storm.utils Utils NimbusClient])
(:import [backtype.storm Constants])
(:import [backtype.storm.grouping CustomStreamGrouping])
(:import [backtype.storm.topology TopologyBuilder])
(:import [backtype.storm.clojure RichShellBolt RichShellSpout])
- (:import [org.apache.thrift.protocol TBinaryProtocol TProtocol])
- (:import [org.apache.thrift.transport TTransport TFramedTransport TSocket])
+ (:import [org.apache.thrift.transport TTransport])
(:use [backtype.storm util config log])
)
@@ -65,10 +64,10 @@
(defn nimbus-client-and-conn [host port]
(log-message "Connecting to Nimbus at " host ":" port)
- (let [transport (TFramedTransport. (TSocket. host port))
- prot (TBinaryProtocol. transport)
- client (Nimbus$Client. prot)]
- (.open transport)
+ (let [conf (read-storm-config)
+ nimbusClient (NimbusClient. conf host port nil)
+ client (.getClient nimbusClient)
+ transport (.transport nimbusClient)]
[client transport] ))
(defmacro with-nimbus-connection [[client-sym host port] & body]
diff --git a/storm-core/src/clj/backtype/storm/timer.clj b/storm-core/src/clj/backtype/storm/timer.clj
index 78b1f1c2466..5eef1d3a2d9 100644
--- a/storm-core/src/clj/backtype/storm/timer.clj
+++ b/storm-core/src/clj/backtype/storm/timer.clj
@@ -23,7 +23,7 @@
;; The timer defined in this file is very similar to java.util.Timer, except it integrates with
;; Storm's time simulation capabilities. This lets us test code that does asynchronous work on the timer thread
-(defnk mk-timer [:kill-fn (fn [& _] )]
+(defnk mk-timer [:kill-fn (fn [& _] ) :thread-name nil]
(let [queue (PriorityQueue. 10
(reify Comparator
(compare [this o1 o2]
@@ -35,35 +35,37 @@
active (atom true)
lock (Object.)
notifier (Semaphore. 0)
- timer-thread (Thread.
- (fn []
- (while @active
- (try
- (let [[time-millis _ _ :as elem] (locking lock (.peek queue))]
- (if (and elem (>= (current-time-millis) time-millis))
- ;; imperative to not run the function inside the timer lock
- ;; otherwise, it's possible to deadlock if function deals with other locks
- ;; (like the submit lock)
- (let [afn (locking lock (second (.poll queue)))]
- (afn))
- (if time-millis ;; if any events are scheduled
- ;; sleep until event generation
- ;; note that if any recurring events are scheduled then we will always go through
- ;; this branch, sleeping only the exact necessary amount of time
- (Time/sleep (- time-millis (current-time-millis)))
- ;; else poll to see if any new event was scheduled
- ;; this is in essence the response time for detecting any new event schedulings when
- ;; there are no scheduled events
- (Time/sleep 1000))
- ))
- (catch Throwable t
- ;; because the interrupted exception can be wrapped in a runtimeexception
- (when-not (exception-cause? InterruptedException t)
- (kill-fn t)
- (reset! active false)
- (throw t))
- )))
- (.release notifier)))]
+ thread-fn (fn []
+ (while @active
+ (try
+ (let [[time-millis _ _ :as elem] (locking lock (.peek queue))]
+ (if (and elem (>= (current-time-millis) time-millis))
+ ;; imperative to not run the function inside the timer lock
+ ;; otherwise, it's possible to deadlock if function deals with other locks
+ ;; (like the submit lock)
+ (let [afn (locking lock (second (.poll queue)))]
+ (afn))
+ (if time-millis ;; if any events are scheduled
+ ;; sleep until event generation
+ ;; note that if any recurring events are scheduled then we will always go through
+ ;; this branch, sleeping only the exact necessary amount of time
+ (Time/sleep (- time-millis (current-time-millis)))
+ ;; else poll to see if any new event was scheduled
+ ;; this is in essence the response time for detecting any new event schedulings when
+ ;; there are no scheduled events
+ (Time/sleep 1000))
+ ))
+ (catch Throwable t
+ ;; because the interrupted exception can be wrapped in a runtimeexception
+ (when-not (exception-cause? InterruptedException t)
+ (kill-fn t)
+ (reset! active false)
+ (throw t))
+ )))
+ (.release notifier))
+ timer-thread (if (clojure.string/blank? thread-name)
+ (Thread. thread-fn)
+ (Thread. thread-fn thread-name))]
(.setDaemon timer-thread true)
(.setPriority timer-thread Thread/MAX_PRIORITY)
(.start timer-thread)
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index ad1a03896a4..6d7cd44f3f7 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -21,14 +21,16 @@
(:use [backtype.storm.ui helpers])
(:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID system-id?]]])
(:use [ring.adapter.jetty :only [run-jetty]])
- (:use [clojure.string :only [trim]])
+ (:use [clojure.string :only [blank? lower-case trim]])
(:import [backtype.storm.utils Utils])
(:import [backtype.storm.generated ExecutorSpecificStats
ExecutorStats ExecutorSummary TopologyInfo SpoutStats BoltStats
ErrorInfo ClusterSummary SupervisorSummary TopologySummary
Nimbus$Client StormTopology GlobalStreamId RebalanceOptions
KillOptions])
- (:import [java.io File])
+ (:import [backtype.storm.security.auth AuthUtils])
+ (:import [java.io File PrintWriter StringWriter])
+ (:import [java.net URLDecoder])
(:require [compojure.route :as route]
[compojure.handler :as handler]
[ring.util.response :as resp]
@@ -51,6 +53,7 @@
(def tips
"Defines a mapping of help texts for elements of the UI pages."
{:sys-stats "Use this to toggle inclusion of storm system components."
+ :user "This should be you."
:version (str "The version of storm installed on the UI node. (Hopefully, "
"this is the same on all storm nodes!)")
:nimbus-uptime (str "The duration the current Nimbus instance has been "
@@ -65,9 +68,13 @@
:name "The name given to the topology by when it was submitted."
:name-link "Click the name to view the Topology's information."
:topo-id "The unique ID given to a Topology each time it is launched."
+ :owner "The user that submitted the Topology, if authentication is enabled."
:status "The status can be one of ACTIVE, INACTIVE, KILLED, or REBALANCING."
:topo-uptime "The time since the Topology was submitted."
:num-workers "The number of Workers (processes)."
+ :scheduler-info (str "This shows information from the scheduler about the "
+ "latest attempt to schedule the Topology on the "
+ "cluster.")
:sup-id (str "A unique identifier given to a Supervisor when it joins the "
"cluster.")
:sup-host (str "The hostname reported by the remote host. (Note that this "
@@ -112,27 +119,33 @@
(defn mk-system-toggle-button [include-sys?]
[:p {:class "js-only"}
- [:span.tip.right {:title (:sys-stats tips)}
- [:input {:type "button"
+ [:span.tip.right {:title (:sys-stats tips)}
+ [:input {:type "button"
:value (str (if include-sys? "Hide" "Show") " System Stats")
:onclick "toggleSys()"}]]])
-(defn ui-template [body]
- (html4
- [:head
- [:title "Storm UI"]
- (include-css "/css/bootstrap-1.4.0.css")
- (include-css "/css/style.css")
- (include-js "/js/jquery-1.6.2.min.js")
- (include-js "/js/jquery.tablesorter.min.js")
- (include-js "/js/jquery.cookies.2.2.0.min.js")
- (include-js "/js/bootstrap-twipsy.js")
- (include-js "/js/script.js")
- ]
- [:body
- [:h1 (link-to "/" "Storm UI")]
- (seq body)
- ]))
+(defn ui-template
+ ([body] (ui-template body nil))
+ ([body user]
+ (html4
+ [:head
+ [:title "Storm UI"]
+ (include-css "/css/bootstrap-1.4.0.css")
+ (include-css "/css/style.css")
+ (include-js "/js/jquery-1.6.2.min.js")
+ (include-js "/js/jquery.tablesorter.min.js")
+ (include-js "/js/jquery.cookies.2.2.0.min.js")
+ (include-js "/js/bootstrap-twipsy.js")
+ (include-js "/js/script.js")
+ ]
+ [:body
+ (concat
+ (when (not (blank? user))
+ [[:div.ui-user
+ [:p [:span.tip.below {:title (:user tips)} "User: " user]]]])
+ [[:h1 (link-to "/" "Storm UI")]]
+ (seq body))
+ ])))
(defn read-storm-version []
(let [storm-home (System/getProperty "storm.home")
@@ -190,6 +203,8 @@
:title (str (:name tips) " " (:name-link tips))}}
{:text "Id" :attr {:class "tip right"
:title (:topo-id tips)}}
+ {:text "Owner" :attr {:class "tip above"
+ :title (:owner tips)}}
{:text "Status" :attr {:class "tip above"
:title (:status tips)}}
{:text "Uptime" :attr {:class "tip above"
@@ -199,17 +214,21 @@
{:text "Num executors" :attr {:class "tip above"
:title (:num-execs tips)}}
{:text "Num tasks" :attr {:class "tip above"
- :title (:num-tasks tips)}}]
+ :title (:num-tasks tips)}}
+ {:text "Scheduler Info" :attr {:class "tip left"
+ :title (:scheduler-info tips)}}]
(for [^TopologySummary t summs]
[(topology-link (.get_id t) (.get_name t))
(escape-html (.get_id t))
+ (escape-html (.get_owner t))
(.get_status t)
(pretty-uptime-sec (.get_uptime_secs t))
(.get_num_workers t)
(.get_num_executors t)
(.get_num_tasks t)
+ (.get_sched_status t)
])
- :time-cols [3]
+ :time-cols [4]
:sort-list "[[0,0]]"
))
@@ -421,6 +440,8 @@
:title (:name tips)}}
{:text "Id" :attr {:class "tip right"
:title (:topo-id tips)}}
+ {:text "Owner" :attr {:class "tip above"
+ :title (:owner tips)}}
{:text "Status" :attr {:class "tip above"
:title (:status tips)}}
{:text "Uptime" :attr {:class "tip above"
@@ -430,14 +451,18 @@
{:text "Num executors" :attr {:class "tip above"
:title (:num-execs tips)}}
{:text "Num tasks" :attr {:class "tip above"
- :title (:num-tasks tips)}}]
+ :title (:num-tasks tips)}}
+ {:text "Scheduler Info" :attr {:class "tip left"
+ :title (:scheduler-info tips)}}]
[[(escape-html (.get_name summ))
(escape-html (.get_id summ))
+ (escape-html (.get_owner summ))
(.get_status summ)
(pretty-uptime-sec (.get_uptime_secs summ))
(count workers)
(count executors)
(sum-tasks executors)
+ (.get_sched_status summ)
]]
)))
@@ -519,9 +544,10 @@
(defn component-link [storm-id id]
(link-to (url-format "/topology/%s/component/%s" storm-id id) (escape-html id)))
-(defn worker-log-link [host port]
- (link-to (url-format "http://%s:%s/log?file=worker-%s.log"
- host (*STORM-CONF* LOGVIEWER-PORT) port) (str port)))
+(defn worker-log-link [host port topology-id]
+ (let [fname (logs-filename topology-id port)]
+ (link-to (url-format (str "http://%s:%s/log?file=%s")
+ host (*STORM-CONF* LOGVIEWER-PORT) fname) (str port))))
(defn render-capacity [capacity]
(let [capacity (nil-to-zero capacity)]
@@ -649,7 +675,30 @@
(StringEscapeUtils/escapeJavaScript name) "', '"
command "', " is-wait ", " default-wait ")")}])
-(defn topology-page [id window include-sys?]
+(defn- ui-actions-enabled? []
+ (= "true" (lower-case (*STORM-CONF* UI-ACTIONS-ENABLED))))
+
+(defn- topology-actions [id name status msg-timeout]
+ (if (ui-actions-enabled?)
+ (concat
+ [[:h2 {:class "js-only"} "Topology actions"]]
+ [[:p {:class "js-only"} (concat
+ [(topology-action-button id name "Activate" "activate" false 0 (= "INACTIVE" status))]
+ [(topology-action-button id name "Deactivate" "deactivate" false 0 (= "ACTIVE" status))]
+ [(topology-action-button id name "Rebalance" "rebalance" true msg-timeout (or (= "ACTIVE" status) (= "INACTIVE" status)))]
+ [(topology-action-button id name "Kill" "kill" true msg-timeout (not= "KILLED" status))]
+ )]] )
+ []))
+
+(defn authorized-ui-user? [user conf topology-conf]
+ (let [ui-users (concat (conf UI-USERS)
+ (conf NIMBUS-ADMINS)
+ (topology-conf UI-USERS)
+ (topology-conf TOPOLOGY-USERS))]
+ (and (not (blank? user))
+ (some #(= % user) ui-users))))
+
+(defn topology-page [id window include-sys? user]
(with-nimbus nimbus
(let [window (if window window ":all-time")
window-hint (window-hint window)
@@ -665,25 +714,24 @@
status (.get_status summ)
msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)
]
- (concat
- [[:h2 "Topology summary"]]
- [(topology-summary-table summ)]
- [[:h2 {:class "js-only"} "Topology actions"]]
- [[:p {:class "js-only"} (concat
- [(topology-action-button id name "Activate" "activate" false 0 (= "INACTIVE" status))]
- [(topology-action-button id name "Deactivate" "deactivate" false 0 (= "ACTIVE" status))]
- [(topology-action-button id name "Rebalance" "rebalance" true msg-timeout (or (= "ACTIVE" status) (= "INACTIVE" status)))]
- [(topology-action-button id name "Kill" "kill" true msg-timeout (not= "KILLED" status))]
- )]]
- [[:h2 "Topology stats"]]
- (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
- [[:h2 "Spouts (" window-hint ")"]]
- (spout-comp-table id spout-comp-summs (.get_errors summ) window include-sys?)
- [[:h2 "Bolts (" window-hint ")"]]
- (bolt-comp-table id bolt-comp-summs (.get_errors summ) window include-sys?)
- [[:h2 "Topology Configuration"]]
- (configuration-table topology-conf)
- ))))
+ (if (or (blank? (*STORM-CONF* UI-FILTER))
+ (authorized-ui-user? user *STORM-CONF* topology-conf))
+ (concat
+ [[:h2 "Topology summary"]]
+ [(topology-summary-table summ)]
+ (topology-actions id name status msg-timeout)
+ [[:h2 "Topology stats"]]
+ (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
+ [[:h2 "Spouts (" window-hint ")"]]
+ (spout-comp-table id spout-comp-summs (.get_errors summ) window include-sys?)
+ [[:h2 "Bolts (" window-hint ")"]]
+ (bolt-comp-table id bolt-comp-summs (.get_errors summ) window include-sys?)
+ [[:h2 "Topology Configuration"]]
+ (configuration-table topology-conf)
+ [(mk-system-toggle-button include-sys?)]
+ )
+
+ (unauthorized-user-html user)))))
(defn component-task-summs [^TopologyInfo summ topology id]
(let [spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
@@ -781,7 +829,7 @@
[(pretty-executor-info (.get_executor_info e))
(pretty-uptime-sec (.get_uptime_secs e))
(.get_host e)
- (worker-log-link (.get_host e) (.get_port e))
+ (worker-log-link (.get_host e) (.get_port e) topology-id)
(nil-to-zero (:emitted stats))
(nil-to-zero (:transferred stats))
(float-str (:complete-latencies stats))
@@ -896,7 +944,7 @@
[(pretty-executor-info (.get_executor_info e))
(pretty-uptime-sec (.get_uptime_secs e))
(.get_host e)
- (worker-log-link (.get_host e) (.get_port e))
+ (worker-log-link (.get_host e) (.get_port e) topology-id)
(nil-to-zero (:emitted stats))
(nil-to-zero (:transferred stats))
(render-capacity (compute-executor-capacity e))
@@ -977,7 +1025,7 @@
:sort-list "[[0,1]]"
)))
-(defn component-page [topology-id component window include-sys?]
+(defn component-page [topology-id component window include-sys? user]
(with-nimbus nimbus
(let [window (if window window ":all-time")
summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
@@ -985,89 +1033,106 @@
type (component-type topology component)
summs (component-task-summs summ topology component)
spec (cond (= type :spout) (spout-page window summ component summs include-sys?)
- (= type :bolt) (bolt-page window summ component summs include-sys?))]
- (concat
- [[:h2 "Component summary"]
- (table [{:text "Id" :attr {:class "tip right"
- :title (:comp-id tips)}}
- {:text "Topology" :attr {:class "tip above"
- :title (str (:name tips) " " (:name-link tips))}}
- {:text "Executors" :attr {:class "tip above"
- :title (:num-execs tips)}}
- {:text "Tasks" :attr {:class "tip above"
- :title (:num-tasks tips)}}]
- [[(escape-html component)
- (topology-link (.get_id summ) (.get_name summ))
- (count summs)
- (sum-tasks summs)
- ]])]
- spec
- [[:h2 "Errors"]
- (errors-table (get (.get_errors summ) component))]
- ))))
+ (= type :bolt) (bolt-page window summ component summs include-sys?))
+ topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))]
+ (if (or (blank? (*STORM-CONF* UI-FILTER))
+ (authorized-ui-user? user *STORM-CONF* topology-conf))
+ (concat
+ [[:h2 "Component summary"]
+ (table [{:text "Id" :attr {:class "tip right"
+ :title (:comp-id tips)}}
+ {:text "Topology" :attr {:class "tip above"
+ :title (str (:name tips) " " (:name-link tips))}}
+ {:text "Executors" :attr {:class "tip above"
+ :title (:num-execs tips)}}
+ {:text "Tasks" :attr {:class "tip above"
+ :title (:num-tasks tips)}}]
+ [[(escape-html component)
+ (topology-link (.get_id summ) (.get_name summ))
+ (count summs)
+ (sum-tasks summs)
+ ]])]
+ spec
+ [[:h2 "Errors"]
+ (errors-table (get (.get_errors summ) component))
+ (mk-system-toggle-button include-sys?)])
+
+ (unauthorized-user-html user)))))
(defn get-include-sys? [cookies]
(let [sys? (get cookies "sys")
sys? (if (or (nil? sys?) (= "false" (:value sys?))) false true)]
sys?))
-(defroutes main-routes
- (GET "/" [:as {cookies :cookies}]
- (-> (main-page)
- ui-template))
- (GET "/topology/:id" [:as {cookies :cookies} id & m]
- (let [include-sys? (get-include-sys? cookies)
- id (url-decode id)]
- (try
- (-> (topology-page (url-decode id) (:window m) include-sys?)
- (concat [(mk-system-toggle-button include-sys?)])
- ui-template)
- (catch Exception e (resp/redirect "/")))))
- (GET "/topology/:id/component/:component" [:as {cookies :cookies} id component & m]
- (let [include-sys? (get-include-sys? cookies)
- id (url-decode id)
- component (url-decode component)]
- (-> (component-page id component (:window m) include-sys?)
- (concat [(mk-system-toggle-button include-sys?)])
- ui-template)))
- (POST "/topology/:id/activate" [id]
- (with-nimbus nimbus
- (let [id (url-decode id)
- tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
- name (.get_name tplg)]
- (.activate nimbus name)
- (log-message "Activating topology '" name "'")))
- (resp/redirect (str "/topology/" id)))
- (POST "/topology/:id/deactivate" [id]
- (with-nimbus nimbus
- (let [id (url-decode id)
- tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
- name (.get_name tplg)]
- (.deactivate nimbus name)
- (log-message "Deactivating topology '" name "'")))
- (resp/redirect (str "/topology/" id)))
- (POST "/topology/:id/rebalance/:wait-time" [id wait-time]
- (with-nimbus nimbus
- (let [id (url-decode id)
- tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
- name (.get_name tplg)
- options (RebalanceOptions.)]
- (.set_wait_secs options (Integer/parseInt wait-time))
- (.rebalance nimbus name options)
- (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
- (resp/redirect (str "/topology/" id)))
- (POST "/topology/:id/kill/:wait-time" [id wait-time]
- (with-nimbus nimbus
- (let [id (url-decode id)
- tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
- name (.get_name tplg)
- options (KillOptions.)]
- (.set_wait_secs options (Integer/parseInt wait-time))
- (.killTopologyWithOpts nimbus name options)
- (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
- (resp/redirect (str "/topology/" id)))
- (route/resources "/")
- (route/not-found "Page not found"))
+(def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*))
+
+(if (ui-actions-enabled?)
+ (defroutes main-routes
+ (GET "/" [:as {servlet-request :servlet-request}]
+ (ui-template (main-page)
+ (.getUserName http-creds-handler servlet-request)))
+ (GET "/topology/:id" [:as {:keys [cookies servlet-request]} id & m]
+ (let [include-sys? (get-include-sys? cookies)
+ user (.getUserName http-creds-handler servlet-request)]
+ (ui-template (topology-page (URLDecoder/decode id) (:window m) include-sys? user)
+ user)))
+ (GET "/topology/:id/component/:component" [:as {:keys [cookies servlet-request]}
+ id component & m]
+ (let [include-sys? (get-include-sys? cookies)
+ user (.getUserName http-creds-handler servlet-request)]
+ (ui-template (component-page (URLDecoder/decode id) component (:window m) include-sys? user)
+ user)))
+ (POST "/topology/:id/activate" [id]
+ (with-nimbus nimbus
+ (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus (URLDecoder/decode id))
+ name (.get_name tplg)]
+ (.activate nimbus name)
+ (log-message "Activating topology '" name "'")))
+ (resp/redirect (str "/topology/" id)))
+ (POST "/topology/:id/deactivate" [id]
+ (with-nimbus nimbus
+ (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus (URLDecoder/decode id))
+ name (.get_name tplg)]
+ (.deactivate nimbus name)
+ (log-message "Deactivating topology '" name "'")))
+ (resp/redirect (str "/topology/" id)))
+ (POST "/topology/:id/rebalance/:wait-time" [id wait-time]
+ (with-nimbus nimbus
+ (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus (URLDecoder/decode id))
+ name (.get_name tplg)
+ options (RebalanceOptions.)]
+ (.set_wait_secs options (Integer/parseInt wait-time))
+ (.rebalance nimbus name options)
+ (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
+ (resp/redirect (str "/topology/" id)))
+ (POST "/topology/:id/kill/:wait-time" [id wait-time]
+ (with-nimbus nimbus
+ (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus (URLDecoder/decode id))
+ name (.get_name tplg)
+ options (KillOptions.)]
+ (.set_wait_secs options (Integer/parseInt wait-time))
+ (.killTopologyWithOpts nimbus name options)
+ (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
+ (resp/redirect (str "/topology/" id)))
+ (route/resources "/")
+ (route/not-found "Page not found"))
+
+ (defroutes main-routes
+ (GET "/" [:as {servlet-request :servlet-request}]
+ (ui-template (main-page)
+ (.getUserName http-creds-handler servlet-request)))
+ (GET "/topology/:id" [:as {:keys [cookies servlet-request]} id & m]
+ (let [include-sys? (get-include-sys? cookies)
+ user (.getUserName http-creds-handler servlet-request)]
+ (ui-template (topology-page (URLDecoder/decode id) (:window m) include-sys? user) user)))
+ (GET "/topology/:id/component/:component" [:as {:keys [cookies servlet-request]}
+ id component & m]
+ (let [include-sys? (get-include-sys? cookies)
+ user (.getUserName http-creds-handler servlet-request)]
+ (ui-template (component-page (URLDecoder/decode id) component (:window m) include-sys? user)
+ user)))
+ (route/resources "/")
+ (route/not-found "Page not found")))
(defn exception->html [ex]
(concat
@@ -1082,8 +1147,9 @@
(handler request)
(catch Exception ex
(-> (resp/response (ui-template (exception->html ex)))
- (resp/status 500)
- (resp/content-type "text/html"))
+ (resp/status 500)
+ (resp/content-type "text/html"))
+ (log-error ex)
))))
(def app
@@ -1091,7 +1157,19 @@
(wrap-reload '[backtype.storm.ui.core])
catch-errors)))
-(defn start-server! [] (run-jetty app {:port (Integer. (*STORM-CONF* UI-PORT))
- :join? false}))
+(defn start-server! []
+ (try
+ (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)}]]
+ (run-jetty app {:port (conf UI-PORT)
+ :join? false
+ :configurator (fn [server]
+ (doseq [connector (.getConnectors server)]
+ (.setHeaderBufferSize connector header-buffer-size))
+ (config-filter server app filters-confs))}))
+ (catch Exception ex
+ (log-error ex))))
(defn -main [] (start-server!))
diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj
index dcd4a211db1..8f339b7e148 100644
--- a/storm-core/src/clj/backtype/storm/ui/helpers.clj
+++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj
@@ -16,10 +16,15 @@
(ns backtype.storm.ui.helpers
(:use compojure.core)
(:use [hiccup core page-helpers])
- (:use [clojure [string :only [join]]])
- (:use [backtype.storm.util :only [uuid defnk url-encode]])
+ (:use [clojure
+ [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 [clj-time coerce format])
(:import [backtype.storm.generated ExecutorInfo ExecutorSummary])
+ (:import [org.mortbay.jetty.security SslSocketConnector])
+ (:require [ring.util servlet])
(:require [compojure.route :as route]
[compojure.handler :as handler]))
@@ -147,3 +152,39 @@ $(\"table#%s\").each(function(i) { $(this).tablesorter({ sortList: %s, headers:
(defn pretty-executor-info [^ExecutorInfo e]
(str "[" (.get_task_start e) "-" (.get_task_end e) "]"))
+(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"]))
+ (.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]
+ (when (> port 0)
+ (.addConnector server (mk-ssl-connector port ks-path ks-password ks-type))))
+
+(defn config-filter [server handler filters-confs]
+ (if filters-confs
+ (let [servlet-holder (org.mortbay.jetty.servlet.ServletHolder.
+ (ring.util.servlet/servlet handler))
+ context (doto (org.mortbay.jetty.servlet.Context. server "/")
+ (.addServlet servlet-holder "/"))]
+ (doseq [{:keys [filter-name filter-class filter-params]} filters-confs]
+ (if filter-class
+ (let [filter-holder (doto (org.mortbay.jetty.servlet.FilterHolder.)
+ (.setClassName filter-class)
+ (.setName (or filter-name filter-class))
+ (.setInitParameters (or filter-params {})))]
+ (.addFilter context filter-holder "/*" org.mortbay.jetty.Handler/ALL))))
+ (.addHandler server context))))
+
+(defn ring-response-from-exception [ex]
+ {:headers {}
+ :status 400
+ :body (.getMessage ex)})
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index 6714fcd447d..45b56825f73 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -24,12 +24,15 @@
(:import [java.util.zip ZipFile])
(:import [java.util.concurrent.locks ReentrantReadWriteLock])
(:import [java.util.concurrent Semaphore])
- (:import [java.io File FileOutputStream StringWriter PrintWriter IOException])
+ (:import [java.io File FileOutputStream RandomAccessFile StringWriter
+ PrintWriter BufferedReader InputStreamReader IOException])
(:import [java.lang.management ManagementFactory])
(:import [org.apache.commons.exec DefaultExecutor CommandLine])
(:import [org.apache.commons.io FileUtils])
(:import [org.apache.commons.exec ExecuteException])
(:import [org.json.simple JSONValue])
+ (:import [org.yaml.snakeyaml Yaml]
+ [org.yaml.snakeyaml.constructor SafeConstructor])
(:require [clojure [string :as str]])
(:import [clojure.lang RT])
(:require [clojure [set :as set]])
@@ -397,13 +400,16 @@
(log-message "Error when trying to kill " pid ". Process is probably already dead."))
))
-(defnk launch-process [command :environment {}]
- (let [builder (ProcessBuilder. command)
- process-env (.environment builder)]
- (doseq [[k v] environment]
- (.put process-env k v))
- (.start builder)
- ))
+(defn read-and-log-stream [prefix stream]
+ (try
+ (let [reader (BufferedReader. (InputStreamReader. stream))]
+ (loop []
+ (if-let [line (.readLine reader)]
+ (do
+ (log-warn (str prefix ":" line))
+ (recur)))))
+ (catch IOException e
+ (log-warn "Error while trying to log stream" e))))
(defn sleep-secs [secs]
(when (pos? secs)
@@ -463,6 +469,39 @@
))
))
+(defn shell-cmd [command]
+ (->> command
+ (map #(str \' (clojure.string/escape % {\' "'\"'\"'"}) \'))
+ (clojure.string/join " ")))
+
+(defnk write-script [dir command :environment {}]
+ (let [script-src (str "#!/bin/bash\n" (clojure.string/join "" (map (fn [[k v]] (str (shell-cmd ["export" (str k "=" v)]) ";\n")) environment)) "\nexec " (shell-cmd command) ";")
+ script-path (str dir "/storm-worker-script.sh")
+ - (spit script-path script-src)]
+ script-path
+ ))
+
+(defnk launch-process [command :environment {} :log-prefix nil :exit-code-callback nil]
+ (let [builder (ProcessBuilder. command)
+ process-env (.environment builder)]
+ (.redirectErrorStream builder true)
+ (doseq [[k v] environment]
+ (.put process-env k v))
+ (let [process (.start builder)]
+ (if (or log-prefix exit-code-callback)
+ (async-loop
+ (fn []
+ (if log-prefix
+ (read-and-log-stream log-prefix (.getInputStream process)))
+ (when exit-code-callback
+ (try
+ (.waitFor process)
+ (catch InterruptedException e
+ (log-message log-prefix " interrupted.")))
+ (exit-code-callback (.exitValue process)))
+ nil)))
+ process)))
+
(defn exists-file? [path]
(.exists (File. path)))
@@ -888,3 +927,32 @@
(meta form))
(list form x)))
([x form & more] `(-<> (-<> ~x ~form) ~@more)))
+
+(def LOG-DIR
+ (.getCanonicalPath
+ (clojure.java.io/file (System/getProperty "storm.home") "logs")))
+
+(defn- logs-rootname [storm-id port]
+ (str storm-id "-worker-" port))
+
+(defn logs-filename [storm-id port]
+ (str (logs-rootname storm-id port) ".log"))
+
+(defn logs-metadata-filename [storm-id port]
+ (str (logs-rootname storm-id port) ".yaml"))
+
+(def worker-log-filename-pattern #"((.*-\d+-\d+)-worker-(\d+)).log")
+
+(defn get-log-metadata-file
+ ([fname]
+ (if-let [[_ _ id port] (re-matches worker-log-filename-pattern fname)]
+ (get-log-metadata-file id port)))
+ ([id port]
+ (clojure.java.io/file LOG-DIR "metadata" (logs-metadata-filename id port))))
+
+(defn clojure-from-yaml-file [yamlFile]
+ (try
+ (let [obj (.load (Yaml. (SafeConstructor.)) (java.io.FileReader. yamlFile))]
+ (clojurify-structure obj))
+ (catch Exception ex
+ (log-error ex))))
diff --git a/storm-core/src/clj/backtype/storm/zookeeper.clj b/storm-core/src/clj/backtype/storm/zookeeper.clj
index ab3e0b2932f..2abd20c090c 100644
--- a/storm-core/src/clj/backtype/storm/zookeeper.clj
+++ b/storm-core/src/clj/backtype/storm/zookeeper.clj
@@ -81,12 +81,13 @@
:sequential CreateMode/PERSISTENT_SEQUENTIAL})
(defn create-node
- ([^CuratorFramework zk ^String path ^bytes data mode]
- (try
- (.. zk (create) (withMode (zk-create-modes mode)) (withACL ZooDefs$Ids/OPEN_ACL_UNSAFE) (forPath (normalize-path path) data))
- (catch Exception e (throw (wrap-in-runtime e)))))
- ([^CuratorFramework zk ^String path ^bytes data]
- (create-node zk path data :persistent)))
+ ([^CuratorFramework zk ^String path ^bytes data mode acls]
+ (let [mode (zk-create-modes mode)]
+ (try
+ (.. zk (create) (withMode mode) (withACL acls) (forPath (normalize-path path) data))
+ (catch Exception e (throw (wrap-in-runtime e))))))
+ ([^CuratorFramework zk ^String path ^bytes data acls]
+ (create-node zk path data :persistent acls)))
(defn exists-node? [^CuratorFramework zk ^String path watch?]
((complement nil?)
@@ -102,12 +103,12 @@
(when-not force (throw e)))
(catch Exception e (throw (wrap-in-runtime e)))))
-(defn mkdirs [^CuratorFramework zk ^String path]
+(defn mkdirs [^CuratorFramework zk ^String path acls]
(let [path (normalize-path path)]
(when-not (or (= path "/") (exists-node? zk path false))
- (mkdirs zk (parent-path path))
+ (mkdirs zk (parent-path path) acls)
(try-cause
- (create-node zk path (barr 7) :persistent)
+ (create-node zk path (barr 7) :persistent acls)
(catch KeeperException$NodeExistsException e
;; this can happen when multiple clients doing mkdir at same time
))
diff --git a/storm-core/src/clj/storm/trident/testing.clj b/storm-core/src/clj/storm/trident/testing.clj
index 3207173b9ac..ac5fcab4e85 100644
--- a/storm-core/src/clj/storm/trident/testing.clj
+++ b/storm-core/src/clj/storm/trident/testing.clj
@@ -16,6 +16,7 @@
(ns storm.trident.testing
(:require [backtype.storm.LocalDRPC :as LocalDRPC])
(:import [storm.trident.testing FeederBatchSpout FeederCommitterBatchSpout MemoryMapState MemoryMapState$Factory TuplifyArgs])
+ (:require [backtype.storm [LocalDRPC]])
(:import [backtype.storm LocalDRPC])
(:import [backtype.storm.tuple Fields])
(:import [backtype.storm.generated KillOptions])
diff --git a/storm-core/src/dev/drpc-simple-acl-test-scenario.yaml b/storm-core/src/dev/drpc-simple-acl-test-scenario.yaml
new file mode 100644
index 00000000000..82c03c0199c
--- /dev/null
+++ b/storm-core/src/dev/drpc-simple-acl-test-scenario.yaml
@@ -0,0 +1,11 @@
+# For the function "jump", alice can perform client operations, and bob can
+# perform invocation operations.
+drpc.authorizer.acl:
+ "jump":
+ "client.users":
+ - "alice"
+ - "bob"
+ "invocation.user": "charlie"
+ "partial":
+ "client.users":
+ - "alice"
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 83ee23248b3..07da267a1bd 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -20,6 +20,7 @@
import backtype.storm.ConfigValidation;
import backtype.storm.serialization.IKryoDecorator;
import backtype.storm.serialization.IKryoFactory;
+import backtype.storm.utils.Utils;
import com.esotericsoftware.kryo.Serializer;
import java.util.ArrayList;
import java.util.HashMap;
@@ -43,6 +44,13 @@
* Spouts.
*/
public class Config extends HashMap {
+ /**
+ * This is part of a temporary workaround to a ZK bug, it is the 'scheme:acl' for
+ * the user Nimbus and Supervisors use to authenticate with ZK.
+ */
+ public static final String STORM_ZOOKEEPER_SUPERACL = "storm.zookeeper.superACL";
+ public static final Object STORM_ZOOKEEPER_SUPERACL_SCHEMA = String.class;
+
/**
* The transporter for communication among Storm tasks
*/
@@ -131,7 +139,13 @@ public class Config extends HashMap {
public static final Object STORM_LOCAL_HOSTNAME_SCHEMA = String.class;
/**
- * The transport plug-in for Thrift client/server communication
+ * The plugin that will convert a principal to a local user.
+ */
+ public static final String STORM_PRINCIPAL_TO_LOCAL_PLUGIN = "storm.principal.tolocal";
+ public static final Object STORM_PRINCIPAL_TO_LOCAL_PLUGIN_SCHEMA = String.class;
+
+ /**
+ * The default transport plug-in for Thrift client/server communication
*/
public static final String STORM_THRIFT_TRANSPORT_PLUGIN = "storm.thrift.transport";
public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
@@ -143,6 +157,13 @@ public class Config extends HashMap {
public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer";
public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class;
+ /**
+ * Try to serialize all tuples, even for local transfers. This should only be used
+ * for testing, as a sanity check that all of your tuples are setup properly.
+ */
+ public static final String TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE = "topology.testing.always.try.serialize";
+ public static final Object TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE_SCHEMA = Boolean.class;
+
/**
* Whether or not to use ZeroMQ for messaging in local mode. If this is set
* to false, then Storm will use a pure-Java messaging system. The purpose
@@ -191,29 +212,71 @@ public class Config extends HashMap {
public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING_SCHEMA = Number.class;
/**
- * The Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
+ * The cluster Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
*/
public static final String STORM_ZOOKEEPER_AUTH_SCHEME="storm.zookeeper.auth.scheme";
public static final Object STORM_ZOOKEEPER_AUTH_SCHEME_SCHEMA = String.class;
/**
- * A string representing the payload for Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication.
+ * A string representing the payload for cluster Zookeeper authentication.
+ * It gets serialized using UTF-8 encoding during authentication.
+ * Note that if this is set to something with a secret (as when using
+ * digest authentication) then it should only be set in the
+ * storm-cluster-auth.yaml file.
+ * This file storm-cluster-auth.yaml should then be protected with
+ * appropriate permissions that deny access from workers.
*/
public static final String STORM_ZOOKEEPER_AUTH_PAYLOAD="storm.zookeeper.auth.payload";
public static final Object STORM_ZOOKEEPER_AUTH_PAYLOAD_SCHEMA = String.class;
+ /**
+ * The topology Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
+ */
+ public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME="storm.zookeeper.topology.auth.scheme";
+ public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME_SCHEMA = String.class;
+
+ /**
+ * A string representing the payload for topology Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication.
+ */
+ public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD="storm.zookeeper.topology.auth.payload";
+ public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD_SCHEMA = String.class;
+
/**
* The id assigned to a running topology. The id is the storm name with a unique nonce appended.
*/
public static final String STORM_ID = "storm.id";
public static final Object STORM_ID_SCHEMA = String.class;
+ /**
+ * The number of times to retry a Nimbus operation.
+ */
+ public static final String STORM_NIMBUS_RETRY_TIMES="storm.nimbus.retry.times";
+ public static final Object STORM_NIMBUS_RETRY_TIMES_SCHEMA = Number.class;
+
+ /**
+ * The starting interval between exponential backoff retries of a Nimbus operation.
+ */
+ public static final String STORM_NIMBUS_RETRY_INTERVAL="storm.nimbus.retry.interval.millis";
+ public static final Object STORM_NIMBUS_RETRY_INTERVAL_SCHEMA = Number.class;
+
+ /**
+ * The ceiling of the interval between retries of a client connect to Nimbus operation.
+ */
+ public static final String STORM_NIMBUS_RETRY_INTERVAL_CEILING="storm.nimbus.retry.intervalceiling.millis";
+ public static final Object STORM_NIMBUS_RETRY_INTERVAL_CEILING_SCHEMA = Number.class;
+
/**
* The host that the master server is running on.
*/
public static final String NIMBUS_HOST = "nimbus.host";
public static final Object NIMBUS_HOST_SCHEMA = String.class;
+ /**
+ * The Nimbus transport plug-in for Thrift client/server communication
+ */
+ public static final String NIMBUS_THRIFT_TRANSPORT_PLUGIN = "nimbus.thrift.transport";
+ public static final Object NIMBUS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
+
/**
* Which port the Thrift interface of Nimbus should run on. Clients should
* connect to this port to upload jars and submit topologies.
@@ -221,13 +284,33 @@ public class Config extends HashMap {
public static final String NIMBUS_THRIFT_PORT = "nimbus.thrift.port";
public static final Object NIMBUS_THRIFT_PORT_SCHEMA = Number.class;
+ /**
+ * The number of threads that should be used by the nimbus thrift server.
+ */
+ public static final String NIMBUS_THRIFT_THREADS = "nimbus.thrift.threads";
+ public static final Object NIMBUS_THRIFT_THREADS_SCHEMA = Number.class;
+
+ /**
+ * A list of users that are cluster admins and can run any command. To use this set
+ * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
+ */
+ public static final String NIMBUS_ADMINS = "nimbus.admins";
+ public static final Object NIMBUS_ADMINS_SCHEMA = ConfigValidation.StringsValidator;
+
+ /**
+ * A list of users that run the supervisors and should be authorized to interact with
+ * nimbus as a supervisor would. To use this set
+ * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
+ */
+ public static final String NIMBUS_SUPERVISOR_USERS = "nimbus.supervisor.users";
+ public static final Object NIMBUS_SUPERVISOR_USERS_SCHEMA = ConfigValidation.StringsValidator;
+
/**
* The maximum buffer size thrift should use when reading messages.
*/
public static final String NIMBUS_THRIFT_MAX_BUFFER_SIZE = "nimbus.thrift.max_buffer_size";
public static final Object NIMBUS_THRIFT_MAX_BUFFER_SIZE_SCHEMA = Number.class;
-
/**
* This parameter is used by the storm-deploy project to configure the
* jvm options for the nimbus daemon.
@@ -317,6 +400,18 @@ public class Config extends HashMap {
public static final String NIMBUS_AUTHORIZER = "nimbus.authorizer";
public static final Object NIMBUS_AUTHORIZER_SCHEMA = String.class;
+ /**
+ * How often nimbus should wake up to renew credentials if needed.
+ */
+ public static final String NIMBUS_CREDENTIAL_RENEW_FREQ_SECS = "nimbus.credential.renewers.freq.secs";
+ public static final Object NIMBUS_CREDENTIAL_RENEW_FREQ_SECS_SCHEMA = Number.class;
+
+ /**
+ * A list of credential renewers that nimbus should load.
+ */
+ public static final String NIMBUS_CREDENTIAL_RENEWERS = "nimbus.credential.renewers.classes";
+ public static final Object NIMBUS_CREDENTIAL_RENEWERS_SCHEMA = ConfigValidation.StringsValidator;
+
/**
* Storm UI binds to this port.
*/
@@ -335,6 +430,24 @@ public class Config extends HashMap {
public static final String LOGVIEWER_CHILDOPTS = "logviewer.childopts";
public static final Object LOGVIEWER_CHILDOPTS_SCHEMA = String.class;
+ /**
+ * How often to clean up old log files
+ */
+ public static final String LOGVIEWER_CLEANUP_INTERVAL_SECS = "logviewer.cleanup.interval.secs";
+ public static final Object LOGVIEWER_CLEANUP_INTERVAL_SECS_SCHEMA = ConfigValidation.PositiveIntegerValidator;
+
+ /**
+ * How many minutes since a log was last modified for the log to be considered for clean-up
+ */
+ public static final String LOGVIEWER_CLEANUP_AGE_MINS = "logviewer.cleanup.age.mins";
+ public static final Object LOGVIEWER_CLEANUP_AGE_MINS_SCHEMA = ConfigValidation.PositiveIntegerValidator;
+
+ /**
+ * A list of users allowed to view logs via the Log Viewer
+ */
+ public static final String LOGS_USERS = "logs.users";
+ public static final Object LOGS_USERS_SCHEMA = ConfigValidation.StringsValidator;
+
/**
* Appender name used by log viewer to determine log directory.
*/
@@ -347,36 +460,156 @@ public class Config extends HashMap {
public static final String UI_CHILDOPTS = "ui.childopts";
public static final Object UI_CHILDOPTS_SCHEMA = String.class;
+ /**
+ * A class implementing javax.servlet.Filter for authenticating/filtering UI requests
+ */
+ public static final String UI_FILTER = "ui.filter";
+ public static final Object UI_FILTER_SCHEMA = String.class;
+
+ /**
+ * Initialization parameters for the javax.servlet.Filter
+ */
+ public static final String UI_FILTER_PARAMS = "ui.filter.params";
+ public static final Object UI_FILTER_PARAMS_SCHEMA = Map.class;
+
+ /**
+ * The size of the header buffer for the UI in bytes
+ */
+ public static final String UI_HEADER_BUFFER_BYTES = "ui.header.buffer.bytes";
+ public static final Object UI_HEADER_BUFFER_BYTES_SCHEMA = Number.class;
+
+ /**
+ * A list of users allowed to view topologies via the UI
+ */
+ public static final String UI_USERS = "ui.users";
+ public static final Object UI_USERS_SCHEMA = ConfigValidation.StringsValidator;
+
+ /**
+ * Whether or not actions should be enabled. When disabled, requests to
+ * modify the state of topologies via HTTP will not be honored.
+ *
+ * Defaults to true.
+ */
+ public static final String UI_ACTIONS_ENABLED = "ui.actions.enabled";
+ public static final Object UI_ACTIONS_ENABLED_SCHEMA = Boolean.class;
+
/**
* List of DRPC servers so that the DRPCSpout knows who to talk to.
*/
public static final String DRPC_SERVERS = "drpc.servers";
public static final Object DRPC_SERVERS_SCHEMA = ConfigValidation.StringsValidator;
+ /**
+ * This port is used by Storm DRPC for receiving HTTP DPRC requests from clients.
+ */
+ public static final String DRPC_HTTP_PORT = "drpc.http.port";
+ public static final Object DRPC_HTTP_PORT_SCHEMA = Number.class;
+
+ /**
+ * This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients.
+ */
+ public static final String DRPC_HTTPS_PORT = "drpc.https.port";
+ public static final Object DRPC_HTTPS_PORT_SCHEMA = Number.class;
+
+ /**
+ * Path to the keystore used by Storm DRPC for setting up HTTPS (SSL).
+ */
+ public static final String DRPC_HTTPS_KEYSTORE_PATH = "drpc.https.keystore.path";
+ public static final Object DRPC_HTTPS_KEYSTORE_PATH_SCHEMA = String.class;
+
+ /**
+ * Password to the keystore used by Storm DRPC for setting up HTTPS (SSL).
+ */
+ public static final String DRPC_HTTPS_KEYSTORE_PASSWORD = "drpc.https.keystore.password";
+ public static final Object DRPC_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class;
+
+ /**
+ * Type of keystore 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_KEYSTORE_TYPE = "drpc.https.keystore.type";
+ public static final Object DRPC_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class;
+
+ /**
+ * The DRPC transport plug-in for Thrift client/server communication
+ */
+ public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport";
+ public static final Object DRPC_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
+
/**
* This port is used by Storm DRPC for receiving DPRC requests from clients.
*/
public static final String DRPC_PORT = "drpc.port";
public static final Object DRPC_PORT_SCHEMA = Number.class;
+ /**
+ * Class name for authorization plugin for DRPC client
+ */
+ public static final String DRPC_AUTHORIZER = "drpc.authorizer";
+ public static final Object DRPC_AUTHORIZER_SCHEMA = String.class;
+
+ /**
+ * The Access Control List for the DRPC Authorizer.
+ * @see DRPCSimpleAclAuthorizer
+ */
+ public static final String DRPC_AUTHORIZER_ACL = "drpc.authorizer.acl";
+ public static final Object DRPC_AUTHORIZER_ACL_SCHEMA = Map.class;
+
+ /**
+ * File name of the DRPC Authorizer ACL.
+ * @see DRPCSimpleAclAuthorizer
+ */
+ public static final String DRPC_AUTHORIZER_ACL_FILENAME = "drpc.authorizer.acl.filename";
+ public static final Object DRPC_AUTHORIZER_ACL_FILENAME_SCHEMA = String.class;
+
+ /**
+ * Whether the DRPCSimpleAclAuthorizer should deny requests for operations
+ * involving functions that have no explicit ACL entry. When set to false
+ * (the default) DRPC functions that have no entry in the ACL will be
+ * permitted, which is appropriate for a development environment. When set
+ * to true, explicit ACL entries are required for every DRPC function, and
+ * any request for functions will be denied.
+ * @see DRPCSimpleAclAuthorizer
+ */
+ public static final String DRPC_AUTHORIZER_ACL_STRICT = "drpc.authorizer.acl.strict";
+ public static final Object DRPC_AUTHORIZER_ACL_STRICT_SCHEMA = Boolean.class;
+
/**
* DRPC thrift server worker threads
*/
public static final String DRPC_WORKER_THREADS = "drpc.worker.threads";
public static final Object DRPC_WORKER_THREADS_SCHEMA = Number.class;
+ /**
+ * The maximum buffer size thrift should use when reading messages for DRPC.
+ */
+ public static final String DRPC_MAX_BUFFER_SIZE = "drpc.max_buffer_size";
+ public static final Object DRPC_MAX_BUFFER_SIZE_SCHEMA = Number.class;
+
/**
* DRPC thrift server queue size
*/
public static final String DRPC_QUEUE_SIZE = "drpc.queue.size";
public static final Object DRPC_QUEUE_SIZE_SCHEMA = Number.class;
+ /**
+ * The DRPC invocations transport plug-in for Thrift client/server communication
+ */
+ public static final String DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN = "drpc.invocations.thrift.transport";
+ public static final Object DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
+
/**
* This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.
*/
public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port";
public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = Number.class;
+ /**
+ * DRPC invocations thrift server worker threads
+ */
+ public static final String DRPC_INVOCATIONS_THREADS = "drpc.invocations.threads";
+ public static final Object DRPC_INVOCATIONS_THREADS_SCHEMA = Number.class;
+
/**
* The timeout on DRPC requests within the DRPC server. Defaults to 10 minutes. Note that requests can also
* timeout based on the socket timeout on the DRPC client, and separately based on the topology message
@@ -392,7 +625,19 @@ public class Config extends HashMap {
public static final Object DRPC_CHILDOPTS_SCHEMA = String.class;
/**
- * the metadata configed on the supervisor
+ * Class name of the HTTP credentials plugin for the UI.
+ */
+ public static final String UI_HTTP_CREDS_PLUGIN = "ui.http.creds.plugin";
+ public static final Object UI_HTTP_CREDS_PLUGIN_SCHEMA = String.class;
+
+ /**
+ * Class name of the HTTP credentials plugin for DRPC.
+ */
+ public static final String DRPC_HTTP_CREDS_PLUGIN = "drpc.http.creds.plugin";
+ public static final Object DRPC_HTTP_CREDS_PLUGIN_SCHEMA = String.class;
+
+ /**
+ * the metadata configured on the supervisor
*/
public static final String SUPERVISOR_SCHEDULER_META = "supervisor.scheduler.meta";
public static final Object SUPERVISOR_SCHEDULER_META_SCHEMA = Map.class;
@@ -403,7 +648,31 @@ public class Config extends HashMap {
*/
public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
public static final Object SUPERVISOR_SLOTS_PORTS_SCHEMA = ConfigValidation.NumbersValidator;
+
+ /**
+ * A number representing the maximum number of workers any single topology can acquire.
+ */
+ public static final String NIMBUS_SLOTS_PER_TOPOLOGY = "nimbus.slots.perTopology";
+ public static final Object NIMBUS_SLOTS_PER_TOPOLOGY_SCHEMA = Number.class;
+
+ /**
+ * A class implementing javax.servlet.Filter for DRPC HTTP requests
+ */
+ public static final String DRPC_HTTP_FILTER = "drpc.http.filter";
+ public static final Object DRPC_HTTP_FILTER_SCHEMA = String.class;
+
+ /**
+ * Initialization parameters for the javax.servlet.Filter of the DRPC HTTP
+ * service
+ */
+ public static final String DRPC_HTTP_FILTER_PARAMS = "drpc.http.filter.params";
+ public static final Object DRPC_HTTP_FILTER_PARAMS_SCHEMA = Map.class;
+ /**
+ * A number representing the maximum number of executors any single topology can acquire.
+ */
+ public static final String NIMBUS_EXECUTORS_PER_TOPOLOGY = "nimbus.executors.perTopology";
+ public static final Object NIMBUS_EXECUTORS_PER_TOPOLOGY_SCHEMA = Number.class;
/**
* This parameter is used by the storm-deploy project to configure the
@@ -412,7 +681,6 @@ public class Config extends HashMap {
public static final String SUPERVISOR_CHILDOPTS = "supervisor.childopts";
public static final Object SUPERVISOR_CHILDOPTS_SCHEMA = String.class;
-
/**
* How long a worker can go without heartbeating before the supervisor tries to
* restart the worker process.
@@ -420,7 +688,6 @@ public class Config extends HashMap {
public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
public static final Object SUPERVISOR_WORKER_TIMEOUT_SECS_SCHEMA = Number.class;
-
/**
* How long a worker can go without heartbeating during the initial launch before
* the supervisor tries to restart the worker process. This value override
@@ -430,7 +697,6 @@ public class Config extends HashMap {
public static final String SUPERVISOR_WORKER_START_TIMEOUT_SECS = "supervisor.worker.start.timeout.secs";
public static final Object SUPERVISOR_WORKER_START_TIMEOUT_SECS_SCHEMA = Number.class;
-
/**
* Whether or not the supervisor should launch workers assigned to it. Defaults
* to true -- and you should probably never change this value. This configuration
@@ -439,7 +705,6 @@ public class Config extends HashMap {
public static final String SUPERVISOR_ENABLE = "supervisor.enable";
public static final Object SUPERVISOR_ENABLE_SCHEMA = Boolean.class;
-
/**
* how often the supervisor sends a heartbeat to the master.
*/
@@ -454,13 +719,35 @@ public class Config extends HashMap {
public static final String SUPERVISOR_MONITOR_FREQUENCY_SECS = "supervisor.monitor.frequency.secs";
public static final Object SUPERVISOR_MONITOR_FREQUENCY_SECS_SCHEMA = Number.class;
+ /**
+ * Should the supervior try to run the worker as the lauching user or not. Defaults to false.
+ */
+ public static final String SUPERVISOR_RUN_WORKER_AS_USER = "supervisor.run.worker.as.user";
+ public static final Object SUPERVISOR_RUN_WORKER_AS_USER_SCHEMA = Boolean.class;
+
+ /**
+ * Full path to the worker-laucher executable that will be used to lauch workers when
+ * SUPERVISOR_RUN_WORKER_AS_USER is set to true.
+ */
+ public static final String SUPERVISOR_WORKER_LAUNCHER = "supervisor.worker.launcher";
+ public static final Object SUPERVISOR_WORKER_LAUNCHER_SCHEMA = String.class;
+
/**
* The jvm opts provided to workers launched by this supervisor. All "%ID%" substrings are replaced
- * with an identifier for this worker.
+ * with an identifier for this worker. Also, "%WORKER-ID%", "%STORM-ID%" and "%WORKER-PORT%" are
+ * replaced with appropriate runtime values for this worker.
*/
public static final String WORKER_CHILDOPTS = "worker.childopts";
public static final Object WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
+ /**
+ * The jvm opts provided to workers launched by this supervisor for GC. All "%ID%" substrings are replaced
+ * with an identifier for this worker. Because the JVM complains about multiple GC opts the topology
+ * can override this default value by setting topology.worker.gc.childopts.
+ */
+ public static final String WORKER_GC_CHILDOPTS = "worker.gc.childopts";
+ public static final Object WORKER_GC_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
+
/**
* How often this worker should heartbeat to the supervisor.
*/
@@ -485,6 +772,19 @@ public class Config extends HashMap {
public static final Object TASK_REFRESH_POLL_SECS_SCHEMA = Number.class;
+ /**
+ * How often a task should sync credentials, worst case.
+ */
+ public static final String TASK_CREDENTIALS_POLL_SECS = "task.credentials.poll.secs";
+ public static final Object TASK_CREDENTIALS_POLL_SECS_SCHEMA = Number.class;
+
+
+ /**
+ * A list of users that are allowed to interact with the topology. To use this set
+ * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
+ */
+ public static final String TOPOLOGY_USERS = "topology.users";
+ public static final Object TOPOLOGY_USERS_SCHEMA = ConfigValidation.StringsValidator;
/**
* True if Storm should timeout messages or not. Defaults to true. This is meant to be used
@@ -662,6 +962,12 @@ public class Config extends HashMap {
public static final String TOPOLOGY_WORKER_CHILDOPTS="topology.worker.childopts";
public static final Object TOPOLOGY_WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
+ /**
+ * Topology-specific options GC for the worker child process. This overrides WORKER_GC_CHILDOPTS.
+ */
+ public static final String TOPOLOGY_WORKER_GC_CHILDOPTS="topology.worker.gc.childopts";
+ public static final Object TOPOLOGY_WORKER_GC_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
+
/**
* This config is available for TransactionalSpouts, and contains the id ( a String) for
* the transactional topology. This id is used to store the state of the transactional
@@ -750,9 +1056,33 @@ public class Config extends HashMap {
/**
* Name of the topology. This config is automatically set by Storm when the topology is submitted.
*/
- public static final String TOPOLOGY_NAME="topology.name";
+ public final static String TOPOLOGY_NAME="topology.name";
public static final Object TOPOLOGY_NAME_SCHEMA = String.class;
+ /**
+ * The principal who submitted a topology
+ */
+ public final static String TOPOLOGY_SUBMITTER_PRINCIPAL = "topology.submitter.principal";
+ public static final Object TOPOLOGY_SUBMITTER_PRINCIPAL_SCHEMA = String.class;
+
+ /**
+ * The local user name of the user who submitted a topology.
+ */
+ public static final String TOPOLOGY_SUBMITTER_USER = "topology.submitter.user";
+ public static final Object TOPOLOGY_SUBMITTER_USER_SCHEMA = String.class;
+
+ /**
+ * Array of components that scheduler should try to place on separate hosts.
+ */
+ public static final String TOPOLOGY_SPREAD_COMPONENTS = "topology.spread.components";
+ public static final Object TOPOLOGY_SPREAD_COMPONENTS_SCHEMA = ConfigValidation.StringsValidator;
+
+ /**
+ * A list of IAutoCredentials that the topology should load and use.
+ */
+ public static final String TOPOLOGY_AUTO_CREDENTIALS = "topology.auto-credentials";
+ public static final Object TOPOLOGY_AUTO_CREDENTIALS_SCHEMA = ConfigValidation.StringsValidator;
+
/**
* Max pending tuples in one ShellBolt
*/
@@ -822,8 +1152,22 @@ public class Config extends HashMap {
* to backtype.storm.scheduler.IsolationScheduler to make use of the isolation scheduler.
*/
public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines";
- public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = Map.class;
+ public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = ConfigValidation.MapOfStringToNumberValidator;
+
+ /**
+ * A map from the user name to the number of machines that should that user is allowed to use. Set storm.scheduler
+ * to backtype.storm.scheduler.multitenant.MultitenantScheduler
+ */
+ public static final String MULTITENANT_SCHEDULER_USER_POOLS = "multitenant.scheduler.user.pools";
+ public static final Object MULTITENANT_SCHEDULER_USER_POOLS_SCHEMA = ConfigValidation.MapOfStringToNumberValidator;
+ /**
+ * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler
+ * to backtype.storm.scheduler.multitenant.MultitenantScheduler
+ */
+ public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines";
+ public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class;
+
public static void setDebug(Map conf, boolean isOn) {
conf.put(Config.TOPOLOGY_DEBUG, isOn);
}
diff --git a/storm-core/src/jvm/backtype/storm/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/ConfigValidation.java
index 3accb820ef5..e990921a60e 100644
--- a/storm-core/src/jvm/backtype/storm/ConfigValidation.java
+++ b/storm-core/src/jvm/backtype/storm/ConfigValidation.java
@@ -18,6 +18,8 @@
package backtype.storm;
import java.util.Map;
+import java.util.Map;
+
/**
* Provides functionality for validating configuration fields.
*/
@@ -35,51 +37,147 @@ public static interface FieldValidator {
*/
public void validateField(String name, Object field) throws IllegalArgumentException;
}
+
+ /**
+ * Declares a method for validating configuration values that is nestable.
+ */
+ public static abstract class NestableFieldValidator implements FieldValidator {
+ @Override
+ public void validateField(String name, Object field) throws IllegalArgumentException {
+ validateField(null, name, field);
+ }
+
+ /**
+ * Validates the given field.
+ * @param pd describes the parent wrapping this validator.
+ * @param name the name of the field.
+ * @param field The field to be validated.
+ * @throws IllegalArgumentException if the field fails validation.
+ */
+ public abstract void validateField(String pd, String name, Object field) throws IllegalArgumentException;
+ }
/**
- * Returns a new FieldValidator for a List of the given Class.
+ * Returns a new NestableFieldValidator for a given class.
+ * @param cls the Class the field should be a type of
+ * @param nullAllowed whether or not a value of null is valid
+ * @return a NestableFieldValidator for that class
+ */
+ public static NestableFieldValidator fv(final Class cls, final boolean nullAllowed) {
+ return new NestableFieldValidator() {
+ @Override
+ public void validateField(String pd, String name, Object field)
+ throws IllegalArgumentException {
+ if (nullAllowed && field == null) {
+ return;
+ }
+ if (! cls.isInstance(field)) {
+ throw new IllegalArgumentException(
+ pd + name + " must be a " + cls.getName() + ". ("+field+")");
+ }
+ }
+ };
+ }
+
+ /**
+ * Returns a new NestableFieldValidator for a List of the given Class.
* @param cls the Class of elements composing the list
- * @return a FieldValidator for a list of the given class
+ * @param nullAllowed whether or not a value of null is valid
+ * @return a NestableFieldValidator for a list of the given class
*/
- static FieldValidator FieldListValidatorFactory(final Class cls) {
- return new FieldValidator() {
+ public static NestableFieldValidator listFv(Class cls, boolean nullAllowed) {
+ return listFv(fv(cls, false), nullAllowed);
+ }
+
+ /**
+ * Returns a new NestableFieldValidator for a List where each item is validated by validator.
+ * @param validator used to validate each item in the list
+ * @param nullAllowed whether or not a value of null is valid
+ * @return a NestableFieldValidator for a list with each item validated by a different validator.
+ */
+ public static NestableFieldValidator listFv(final NestableFieldValidator validator,
+ final boolean nullAllowed) {
+ return new NestableFieldValidator() {
@Override
- public void validateField(String name, Object field)
+ public void validateField(String pd, String name, Object field)
throws IllegalArgumentException {
- if (field == null) {
- // A null value is acceptable.
+ if (nullAllowed && field == null) {
return;
}
if (field instanceof Iterable) {
for (Object e : (Iterable)field) {
- if (! cls.isInstance(e)) {
- throw new IllegalArgumentException(
- "Each element of the list " + name + " must be a " +
- cls.getName() + ".");
- }
+ validator.validateField(pd + "Each element of the list ", name, e);
}
return;
}
throw new IllegalArgumentException(
- "Field " + name + " must be an Iterable of " + cls.getName());
+ "Field " + name + " must be an Iterable but was " +
+ ((field == null) ? "null" : ("a " + field.getClass())));
}
};
}
+ /**
+ * Returns a new NestableFieldValidator for a Map of key to val.
+ * @param key the Class of keys in the map
+ * @param val the Class of values in the map
+ * @param nullAllowed whether or not a value of null is valid
+ * @return a NestableFieldValidator for a Map of key to val
+ */
+ public static NestableFieldValidator mapFv(Class key, Class val,
+ boolean nullAllowed) {
+ return mapFv(fv(key, false), fv(val, false), nullAllowed);
+ }
+
+ /**
+ * Returns a new NestableFieldValidator for a Map.
+ * @param key a validator for the keys in the map
+ * @param val a validator for the values in the map
+ * @param nullAllowed whether or not a value of null is valid
+ * @return a NestableFieldValidator for a Map
+ */
+ public static NestableFieldValidator mapFv(final NestableFieldValidator key,
+ final NestableFieldValidator val, final boolean nullAllowed) {
+ return new NestableFieldValidator() {
+ @SuppressWarnings("unchecked")
+ @Override
+ public void validateField(String pd, String name, Object field)
+ throws IllegalArgumentException {
+ if (nullAllowed && field == null) {
+ return;
+ }
+ if (field instanceof Map) {
+ for (Map.Entry