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.0 2.1.3 3.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.clojure tools.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 Core Storm Core Java API and Clojure implementation. + + /etc/storm + + + @@ -76,7 +81,22 @@ clojure-complete test - + + org.clojars.runa + conjure + test + + + org.clojure + clojure-contrib + test + + + org.clojure + data.codec + test + + commons-io @@ -171,11 +191,6 @@ mockito-all test - - 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 entry: ((Map)field).entrySet()) { + key.validateField("Each key of the map ", name, entry.getKey()); + val.validateField("Each value in the map ", name, entry.getValue()); + } + return; + } + throw new IllegalArgumentException( + "Field " + name + " must be a Map"); + } + }; + } + /** * Validates a list of Numbers. */ - public static Object NumbersValidator = FieldListValidatorFactory(Number.class); + public static Object NumbersValidator = listFv(Number.class, true); /** - * Validates is a list of Strings. + * Validates a list of Strings. + */ + public static Object StringsValidator = listFv(String.class, true); + + /** + * Validates a map of Strings to Numbers. */ - public static Object StringsValidator = FieldListValidatorFactory(String.class); + public static Object MapOfStringToNumberValidator = mapFv(String.class, Number.class, true); /** * Validates is a list of Maps. */ - public static Object MapsValidator = FieldListValidatorFactory(Map.class); + public static Object MapsValidator = listFv(Map.class, true); /** * Validates a power of 2. @@ -104,6 +202,28 @@ public void validateField(String name, Object o) throws IllegalArgumentException } }; + /** + * Validates a positive integer. + */ + public static Object PositiveIntegerValidator = new FieldValidator() { + @Override + public void validateField(String name, Object o) throws IllegalArgumentException { + if (o == null) { + // A null value is acceptable. + return; + } + final long i; + if (o instanceof Number && + (i = ((Number)o).longValue()) == ((Number)o).doubleValue()) + { + if (i > 0) { + return; + } + } + throw new IllegalArgumentException("Field " + name + " must be a positive integer."); + } + }; + /** * Validates Kryo Registration */ @@ -141,7 +261,7 @@ public void validateField(String name, Object o) throws IllegalArgumentException */ public static Object StringOrStringListValidator = new FieldValidator() { - private FieldValidator fv = FieldListValidatorFactory(String.class); + private FieldValidator fv = listFv(String.class, false); @Override public void validateField(String name, Object o) throws IllegalArgumentException { diff --git a/storm-core/src/jvm/backtype/storm/Constants.java b/storm-core/src/jvm/backtype/storm/Constants.java index 39d3ffa7293..35c252f8ab3 100644 --- a/storm-core/src/jvm/backtype/storm/Constants.java +++ b/storm-core/src/jvm/backtype/storm/Constants.java @@ -31,5 +31,6 @@ public class Constants { public static final String METRICS_COMPONENT_ID_PREFIX = "__metrics"; public static final String METRICS_STREAM_ID = "__metrics"; public static final String METRICS_TICK_STREAM_ID = "__metrics_tick"; + public static final String CREDENTIALS_CHANGED_STREAM_ID = "__credentials"; } - \ No newline at end of file + diff --git a/storm-core/src/jvm/backtype/storm/ICredentialsListener.java b/storm-core/src/jvm/backtype/storm/ICredentialsListener.java new file mode 100644 index 00000000000..1a7bc1b8422 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/ICredentialsListener.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm; + +import java.util.Map; + +/** + * Allows a bolt or a spout to be informed when the credentials of the topology have changed. + */ +public interface ICredentialsListener { + /** + * Called when the credentials of a topology have changed. + * @param credentials the new credentials, could be null. + */ + public void setCredentials(Map credentials); +} diff --git a/storm-core/src/jvm/backtype/storm/ILocalCluster.java b/storm-core/src/jvm/backtype/storm/ILocalCluster.java index 818dfb095f2..7d5aa35b44a 100644 --- a/storm-core/src/jvm/backtype/storm/ILocalCluster.java +++ b/storm-core/src/jvm/backtype/storm/ILocalCluster.java @@ -26,6 +26,7 @@ import backtype.storm.generated.RebalanceOptions; import backtype.storm.generated.StormTopology; import backtype.storm.generated.TopologyInfo; +import backtype.storm.generated.Credentials; import java.util.Map; @@ -33,6 +34,7 @@ public interface ILocalCluster { void submitTopology(String topologyName, Map conf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException; void submitTopologyWithOpts(String topologyName, Map conf, StormTopology topology, SubmitOptions submitOpts) throws AlreadyAliveException, InvalidTopologyException; + void uploadNewCredentials(String topologyName, Credentials creds); void killTopology(String topologyName) throws NotAliveException; void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException; void activate(String topologyName) throws NotAliveException; diff --git a/storm-core/src/jvm/backtype/storm/StormSubmitter.java b/storm-core/src/jvm/backtype/storm/StormSubmitter.java index 5dfb34b0a19..d5da10380f0 100644 --- a/storm-core/src/jvm/backtype/storm/StormSubmitter.java +++ b/storm-core/src/jvm/backtype/storm/StormSubmitter.java @@ -19,6 +19,8 @@ import java.io.File; import java.nio.ByteBuffer; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import java.util.HashMap; import java.util.Map; @@ -28,13 +30,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import backtype.storm.generated.AlreadyAliveException; -import backtype.storm.generated.ClusterSummary; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.Nimbus; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.SubmitOptions; -import backtype.storm.generated.TopologySummary; +import backtype.storm.security.auth.IAutoCredentials; +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.generated.*; import backtype.storm.utils.BufferFileInputStream; import backtype.storm.utils.NimbusClient; import backtype.storm.utils.Utils; @@ -49,12 +47,99 @@ public class StormSubmitter { private static final int THRIFT_CHUNK_SIZE_BYTES = 307200; - private static Nimbus.Iface localNimbus = null; + private static ILocalCluster localNimbus = null; - public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) { + public static void setLocalNimbus(ILocalCluster localNimbusHandler) { StormSubmitter.localNimbus = localNimbusHandler; } + private static String generateZookeeperDigestSecretPayload() { + return Utils.secureRandomLong() + ":" + Utils.secureRandomLong(); + } + + public static final Pattern zkDigestPattern = Pattern.compile("\\S+:\\S+"); + + public static boolean validateZKDigestPayload(String payload) { + if (payload != null) { + Matcher m = zkDigestPattern.matcher(payload); + return m.matches(); + } + return false; + } + + @SuppressWarnings("unchecked") + public static Map prepareZookeeperAuthentication(Map conf) { + Map toRet = new HashMap(); + + // Is the topology ZooKeeper authentication configuration unset? + if (! conf.containsKey(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) || + conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) == null || + ! validateZKDigestPayload((String) + conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD))) { + + String secretPayload = generateZookeeperDigestSecretPayload(); + toRet.put(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD, secretPayload); + LOG.info("Generated ZooKeeper secret payload for MD5-digest: " + secretPayload); + } + + // This should always be set to digest. + toRet.put(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME, "digest"); + + return toRet; + } + + private static Map populateCredentials(Map conf, Map creds) { + Map ret = new HashMap(); + for (IAutoCredentials autoCred: AuthUtils.GetAutoCredentials(conf)) { + LOG.info("Running "+autoCred); + autoCred.populateCredentials(ret); + } + if (creds != null) { + ret.putAll(creds); + } + return ret; + } + + /** + * Push a new set of credentials to the running topology. + * @param name the name of the topology to push credentials to. + * @param stormConf the topology-specific configuration, if desired. See {@link Config}. + * @param credentials the credentials to push. + * @throws AuthorizationException if you are not authorized ot push credentials. + * @throws NotAliveException if the topology is not alive + * @throws InvalidTopologyException if any other error happens + */ + public static void pushCredentials(String name, Map stormConf, Map credentials) + throws AuthorizationException, NotAliveException, InvalidTopologyException { + stormConf = new HashMap(stormConf); + stormConf.putAll(Utils.readCommandLineOpts()); + Map conf = Utils.readStormConfig(); + conf.putAll(stormConf); + Map fullCreds = populateCredentials(conf, credentials); + if (fullCreds.isEmpty()) { + LOG.warn("No credentials were found to push to "+name); + return; + } + try { + if(localNimbus!=null) { + LOG.info("Pushing Credentials to topology " + name + " in local mode"); + localNimbus.uploadNewCredentials(name, new Credentials(fullCreds)); + } else { + NimbusClient client = NimbusClient.getConfiguredClient(conf); + try { + LOG.info("Uploading new credentials to " + name); + client.getClient().uploadNewCredentials(name, new Credentials(fullCreds)); + } finally { + client.close(); + } + } + LOG.info("Finished submitting topology: " + name); + } catch(TException e) { + throw new RuntimeException(e); + } + } + + /** * Submits a topology to run on the cluster. A topology runs forever or until * explicitly killed. @@ -65,8 +150,10 @@ public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) { * @param topology the processing to execute. * @throws AlreadyAliveException if a topology with this name is already running * @throws InvalidTopologyException if an invalid topology was submitted + * @throws AuthorizationException if authorization is failed */ - public static void submitTopology(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException { + public static void submitTopology(String name, Map stormConf, StormTopology topology) + throws AlreadyAliveException, InvalidTopologyException, AuthorizationException { submitTopology(name, stormConf, topology, null, null); } @@ -82,8 +169,11 @@ public static void submitTopology(String name, Map stormConf, StormTopology topo * @param progressListener to track the progress of the jar upload process * @throws AlreadyAliveException if a topology with this name is already running * @throws InvalidTopologyException if an invalid topology was submitted + * @throws AuthorizationException if authorization is failed */ - public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts, ProgressListener progressListener) throws AlreadyAliveException, InvalidTopologyException { + @SuppressWarnings("unchecked") + public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts, + ProgressListener progressListener) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException { if(!Utils.isValidConf(stormConf)) { throw new IllegalArgumentException("Storm conf is not valid. Must be json-serializable"); } @@ -91,24 +181,45 @@ public static void submitTopology(String name, Map stormConf, StormTopology topo stormConf.putAll(Utils.readCommandLineOpts()); Map conf = Utils.readStormConfig(); conf.putAll(stormConf); + stormConf.putAll(prepareZookeeperAuthentication(conf)); + + Map passedCreds = new HashMap(); + if (opts != null) { + Credentials tmpCreds = opts.get_creds(); + if (tmpCreds != null) { + passedCreds = tmpCreds.get_creds(); + } + } + Map fullCreds = populateCredentials(conf, passedCreds); + if (!fullCreds.isEmpty()) { + if (opts == null) { + opts = new SubmitOptions(TopologyInitialStatus.ACTIVE); + } + opts.set_creds(new Credentials(fullCreds)); + } try { - String serConf = JSONValue.toJSONString(stormConf); if(localNimbus!=null) { LOG.info("Submitting topology " + name + " in local mode"); - localNimbus.submitTopology(name, null, serConf, topology); + if(opts!=null) { + localNimbus.submitTopologyWithOpts(name, stormConf, topology, opts); + } else { + // this is for backwards compatibility + localNimbus.submitTopology(name, stormConf, topology); + } } else { + String serConf = JSONValue.toJSONString(stormConf); NimbusClient client = NimbusClient.getConfiguredClient(conf); if(topologyNameExists(conf, name)) { throw new RuntimeException("Topology with name `" + name + "` already exists on cluster"); } - submitJar(conf, progressListener); + String jar = submitJar(conf, progressListener); try { LOG.info("Submitting topology " + name + " in distributed mode with conf " + serConf); if(opts!=null) { - client.getClient().submitTopologyWithOpts(name, submittedJar, serConf, topology, opts); + client.getClient().submitTopologyWithOpts(name, jar, serConf, topology, opts); } else { // this is for backwards compatibility - client.getClient().submitTopology(name, submittedJar, serConf, topology); + client.getClient().submitTopology(name, jar, serConf, topology); } } catch(InvalidTopologyException e) { LOG.warn("Topology submission exception: "+e.get_msg()); @@ -136,9 +247,10 @@ public static void submitTopology(String name, Map stormConf, StormTopology topo * @param topology the processing to execute. * @throws AlreadyAliveException if a topology with this name is already running * @throws InvalidTopologyException if an invalid topology was submitted + * @throws AuthorizationException if authorization is failed */ - public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException { + public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException { submitTopologyWithProgressBar(name, stormConf, topology, null); } @@ -153,9 +265,10 @@ public static void submitTopologyWithProgressBar(String name, Map stormConf, Sto * @param opts to manipulate the starting of the topology * @throws AlreadyAliveException if a topology with this name is already running * @throws InvalidTopologyException if an invalid topology was submitted + * @throws AuthorizationException if authorization is failed */ - public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology, SubmitOptions opts) throws AlreadyAliveException, InvalidTopologyException { + public static void submitTopologyWithProgressBar(String name, Map stormConf, StormTopology topology, SubmitOptions opts) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException { // show a progress bar so we know we're not stuck (especially on slow connections) submitTopology(name, stormConf, topology, opts, new StormSubmitter.ProgressListener() { @Override @@ -198,16 +311,8 @@ private static boolean topologyNameExists(Map conf, String name) { } } - private static String submittedJar = null; - - private static void submitJar(Map conf, ProgressListener listener) { - if(submittedJar==null) { - LOG.info("Jar not uploaded to master yet. Submitting jar..."); - String localJar = System.getProperty("storm.jar"); - submittedJar = submitJar(conf, localJar, listener); - } else { - LOG.info("Jar already uploaded to master. Not submitting jar."); - } + private static String submitJar(Map conf, ProgressListener listener) { + return submitJar(conf, System.getProperty("storm.jar"), listener); } /** diff --git a/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java b/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java index fadebf67d13..987cde0c52d 100644 --- a/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java +++ b/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java @@ -17,36 +17,33 @@ */ package backtype.storm.drpc; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + import backtype.storm.generated.DRPCRequest; import backtype.storm.generated.DistributedRPCInvocations; +import backtype.storm.generated.AuthorizationException; +import backtype.storm.security.auth.ThriftClient; +import backtype.storm.security.auth.ThriftConnectionType; +import org.apache.thrift.transport.TTransportException; import org.apache.thrift.TException; -import org.apache.thrift.protocol.TBinaryProtocol; -import org.apache.thrift.transport.TFramedTransport; -import org.apache.thrift.transport.TSocket; -import org.apache.thrift.transport.TTransport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class DRPCInvocationsClient implements DistributedRPCInvocations.Iface { - private TTransport conn; - private DistributedRPCInvocations.Client client; +public class DRPCInvocationsClient extends ThriftClient implements DistributedRPCInvocations.Iface { + public static Logger LOG = LoggerFactory.getLogger(DRPCInvocationsClient.class); + private final AtomicReference client = + new AtomicReference(); private String host; - private int port; + private int port; - public DRPCInvocationsClient(String host, int port) { - try { - this.host = host; - this.port = port; - connect(); - } catch(TException e) { - throw new RuntimeException(e); - } - } - - private void connect() throws TException { - conn = new TFramedTransport(new TSocket(host, port)); - client = new DistributedRPCInvocations.Client(new TBinaryProtocol(conn)); - conn.open(); + public DRPCInvocationsClient(Map conf, String host, int port) throws TTransportException { + super(conf, ThriftConnectionType.DRPC_INVOCATIONS, host, port, null); + this.host = host; + this.port = port; + client.set(new DistributedRPCInvocations.Client(_protocol)); } - + public String getHost() { return host; } @@ -55,37 +52,57 @@ public int getPort() { return port; } - public void result(String id, String result) throws TException { + public void reconnectClient() throws TException { + if (client.get() == null) { + reconnect(); + client.set(new DistributedRPCInvocations.Client(_protocol)); + } + } + + public boolean isConnected() { + return client.get() != null; + } + + public void result(String id, String result) throws TException, AuthorizationException { + DistributedRPCInvocations.Client c = client.get(); try { - if(client==null) connect(); - client.result(id, result); + if (c == null) { + throw new TException("Client is not connected..."); + } + c.result(id, result); } catch(TException e) { - client = null; + client.compareAndSet(c, null); throw e; } } - public DRPCRequest fetchRequest(String func) throws TException { + public DRPCRequest fetchRequest(String func) throws TException, AuthorizationException { + DistributedRPCInvocations.Client c = client.get(); try { - if(client==null) connect(); - return client.fetchRequest(func); + if (c == null) { + throw new TException("Client is not connected..."); + } + return c.fetchRequest(func); } catch(TException e) { - client = null; + client.compareAndSet(c, null); throw e; } } - public void failRequest(String id) throws TException { + public void failRequest(String id) throws TException, AuthorizationException { + DistributedRPCInvocations.Client c = client.get(); try { - if(client==null) connect(); - client.failRequest(id); + if (c == null) { + throw new TException("Client is not connected..."); + } + c.failRequest(id); } catch(TException e) { - client = null; + client.compareAndSet(c, null); throw e; } } - public void close() { - conn.close(); + public DistributedRPCInvocations.Client getClient() { + return client.get(); } } diff --git a/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java b/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java index 918cbc02046..82fd6cd5823 100644 --- a/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java +++ b/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java @@ -21,6 +21,7 @@ import backtype.storm.ILocalDRPC; import backtype.storm.generated.DRPCRequest; import backtype.storm.generated.DistributedRPCInvocations; +import backtype.storm.generated.AuthorizationException; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; @@ -31,18 +32,30 @@ import backtype.storm.utils.Utils; import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.LinkedList; import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.Callable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; import org.json.simple.JSONValue; public class DRPCSpout extends BaseRichSpout { + //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS + static final long serialVersionUID = 2387848310969237877L; + public static Logger LOG = LoggerFactory.getLogger(DRPCSpout.class); SpoutOutputCollector _collector; List _clients = new ArrayList(); + transient LinkedList> _futures = null; + transient ExecutorService _backround = null; String _function; String _local_drpc_id = null; @@ -65,11 +78,60 @@ public DRPCSpout(String function, ILocalDRPC drpc) { _function = function; _local_drpc_id = drpc.getServiceId(); } - + + private class Adder implements Callable { + private String server; + private int port; + private Map conf; + + public Adder(String server, int port, Map conf) { + this.server = server; + this.port = port; + this.conf = conf; + } + + @Override + public Void call() throws Exception { + DRPCInvocationsClient c = new DRPCInvocationsClient(conf, server, port); + synchronized (_clients) { + _clients.add(c); + } + return null; + } + } + + private void reconnect(final DRPCInvocationsClient c) { + _futures.add(_backround.submit(new Callable() { + @Override + public Void call() throws Exception { + c.reconnectClient(); + return null; + } + })); + } + + private void checkFutures() { + Iterator> i = _futures.iterator(); + while (i.hasNext()) { + Future f = i.next(); + if (f.isDone()) { + i.remove(); + } + try { + f.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + @Override public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { _collector = collector; if(_local_drpc_id==null) { + _backround = Executors.newCachedThreadPool(); + _futures = new LinkedList>(); + int numTasks = context.getComponentTasks(context.getThisComponentId()).size(); int index = context.getThisTaskIndex(); @@ -78,13 +140,14 @@ public void open(Map conf, TopologyContext context, SpoutOutputCollector collect if(servers == null || servers.isEmpty()) { throw new RuntimeException("No DRPC servers configured for topology"); } - if(numTasks < servers.size()) { - for(String s: servers) { - _clients.add(new DRPCInvocationsClient(s, port)); + + if (numTasks < servers.size()) { + for (String s: servers) { + _futures.add(_backround.submit(new Adder(s, port, conf))); } - } else { + } else { int i = index % servers.size(); - _clients.add(new DRPCInvocationsClient(servers.get(i), port)); + _futures.add(_backround.submit(new Adder(servers.get(i), port, conf))); } } @@ -101,8 +164,18 @@ public void close() { public void nextTuple() { boolean gotRequest = false; if(_local_drpc_id==null) { - for(int i=0; i<_clients.size(); i++) { - DRPCInvocationsClient client = _clients.get(i); + int size = 0; + synchronized (_clients) { + size = _clients.size(); //This will only ever grow, so no need to worry about falling off the end + } + for(int i=0; i 0) { @@ -114,10 +187,17 @@ public void nextTuple() { _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)), new DRPCMessageId(req.get_request_id(), i)); break; } + } catch (TException e) { + reconnect(client); + LOG.error("Failed to fetch DRPC result from DRPC server", e); + } catch (AuthorizationException aze) { + reconnect(client); + LOG.error("Not authorized to fetch DRPC result from DRPC server", aze); } catch (Exception e) { LOG.error("Failed to fetch DRPC result from DRPC server", e); } } + checkFutures(); } else { DistributedRPCInvocations.Iface drpc = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id); if(drpc!=null) { // can happen during shutdown of drpc while topology is still up @@ -133,6 +213,8 @@ public void nextTuple() { } } catch (TException e) { throw new RuntimeException(e); + } catch (AuthorizationException aze) { + throw new RuntimeException(aze); } } } @@ -159,6 +241,8 @@ public void fail(Object msgId) { client.failRequest(did.id); } catch (TException e) { LOG.error("Failed to fail request", e); + } catch (AuthorizationException aze) { + LOG.error("Not authorized to failREquest from DRPC server", aze); } } diff --git a/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java b/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java index 34cca985eb1..3d50679ca7a 100644 --- a/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java +++ b/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java @@ -19,6 +19,7 @@ import backtype.storm.Config; import backtype.storm.generated.DistributedRPCInvocations; +import backtype.storm.generated.AuthorizationException; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.OutputFieldsDeclarer; @@ -33,18 +34,23 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; import org.json.simple.JSONValue; public class ReturnResults extends BaseRichBolt { + //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS + static final long serialVersionUID = -774882142710631591L; + public static final Logger LOG = LoggerFactory.getLogger(ReturnResults.class); OutputCollector _collector; boolean local; - + Map _conf; Map _clients = new HashMap(); @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + _conf = stormConf; _collector = collector; local = stormConf.get(Config.STORM_CLUSTER_MODE).equals("local"); } @@ -68,17 +74,40 @@ public void execute(Tuple input) { }}; if(!_clients.containsKey(server)) { - _clients.put(server, new DRPCInvocationsClient(host, port)); + try { + _clients.put(server, new DRPCInvocationsClient(_conf, host, port)); + } catch (TTransportException ex) { + throw new RuntimeException(ex); + } } client = _clients.get(server); } - + try { client.result(id, result); _collector.ack(input); } catch(TException e) { LOG.error("Failed to return results to DRPC server", e); _collector.fail(input); + if (client instanceof DRPCInvocationsClient) { + try { + LOG.info("reconnecting... "); + ((DRPCInvocationsClient)client).reconnectClient(); //Blocking call + } catch (TException e2) { + throw new RuntimeException(e2); + } + } + } catch (AuthorizationException aze) { + LOG.error("Not authorized to return results to DRPC server", aze); + _collector.fail(input); + if (client instanceof DRPCInvocationsClient) { + try { + LOG.info("reconnecting... "); + ((DRPCInvocationsClient)client).reconnectClient(); //Blocking call + } catch (TException e2) { + throw new RuntimeException(e2); + } + } } } } diff --git a/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java b/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java new file mode 100644 index 00000000000..9efc9da385e --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java @@ -0,0 +1,328 @@ +/** + * Autogenerated by Thrift Compiler (0.7.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + */ +package backtype.storm.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AuthorizationException extends Exception implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AuthorizationException"); + + private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1); + + private String msg; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MSG((short)1, "msg"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MSG + return MSG; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AuthorizationException.class, metaDataMap); + } + + public AuthorizationException() { + } + + public AuthorizationException( + String msg) + { + this(); + this.msg = msg; + } + + /** + * Performs a deep copy on other. + */ + public AuthorizationException(AuthorizationException other) { + if (other.is_set_msg()) { + this.msg = other.msg; + } + } + + public AuthorizationException deepCopy() { + return new AuthorizationException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String get_msg() { + return this.msg; + } + + public void set_msg(String msg) { + this.msg = msg; + } + + public void unset_msg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean is_set_msg() { + return this.msg != null; + } + + public void set_msg_isSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unset_msg(); + } else { + set_msg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return get_msg(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MSG: + return is_set_msg(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof AuthorizationException) + return this.equals((AuthorizationException)that); + return false; + } + + public boolean equals(AuthorizationException that) { + if (that == null) + return false; + + boolean this_present_msg = true && this.is_set_msg(); + boolean that_present_msg = true && that.is_set_msg(); + if (this_present_msg || that_present_msg) { + if (!(this_present_msg && that_present_msg)) + return false; + if (!this.msg.equals(that.msg)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_msg = true && (is_set_msg()); + builder.append(present_msg); + if (present_msg) + builder.append(msg); + + return builder.toHashCode(); + } + + public int compareTo(AuthorizationException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + AuthorizationException typedOther = (AuthorizationException)other; + + lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_msg()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, typedOther.msg); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField field; + iprot.readStructBegin(); + while (true) + { + field = iprot.readFieldBegin(); + if (field.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (field.id) { + case 1: // MSG + if (field.type == org.apache.thrift.protocol.TType.STRING) { + this.msg = iprot.readString(); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (this.msg != null) { + oprot.writeFieldBegin(MSG_FIELD_DESC); + oprot.writeString(this.msg); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("AuthorizationException("); + boolean first = true; + + sb.append("msg:"); + if (this.msg == null) { + sb.append("null"); + } else { + sb.append(this.msg); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_msg()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); + } + + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/Credentials.java b/storm-core/src/jvm/backtype/storm/generated/Credentials.java new file mode 100644 index 00000000000..105cec1d8ba --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/Credentials.java @@ -0,0 +1,373 @@ +/** + * Autogenerated by Thrift Compiler (0.7.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + */ +package backtype.storm.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class Credentials implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials"); + + private static final org.apache.thrift.protocol.TField CREDS_FIELD_DESC = new org.apache.thrift.protocol.TField("creds", org.apache.thrift.protocol.TType.MAP, (short)1); + + private Map creds; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + CREDS((short)1, "creds"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // CREDS + return CREDS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Credentials.class, metaDataMap); + } + + public Credentials() { + } + + public Credentials( + Map creds) + { + this(); + this.creds = creds; + } + + /** + * Performs a deep copy on other. + */ + public Credentials(Credentials other) { + if (other.is_set_creds()) { + Map __this__creds = new HashMap(); + for (Map.Entry other_element : other.creds.entrySet()) { + + String other_element_key = other_element.getKey(); + String other_element_value = other_element.getValue(); + + String __this__creds_copy_key = other_element_key; + + String __this__creds_copy_value = other_element_value; + + __this__creds.put(__this__creds_copy_key, __this__creds_copy_value); + } + this.creds = __this__creds; + } + } + + public Credentials deepCopy() { + return new Credentials(this); + } + + @Override + public void clear() { + this.creds = null; + } + + public int get_creds_size() { + return (this.creds == null) ? 0 : this.creds.size(); + } + + public void put_to_creds(String key, String val) { + if (this.creds == null) { + this.creds = new HashMap(); + } + this.creds.put(key, val); + } + + public Map get_creds() { + return this.creds; + } + + public void set_creds(Map creds) { + this.creds = creds; + } + + public void unset_creds() { + this.creds = null; + } + + /** Returns true if field creds is set (has been assigned a value) and false otherwise */ + public boolean is_set_creds() { + return this.creds != null; + } + + public void set_creds_isSet(boolean value) { + if (!value) { + this.creds = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case CREDS: + if (value == null) { + unset_creds(); + } else { + set_creds((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case CREDS: + return get_creds(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case CREDS: + return is_set_creds(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof Credentials) + return this.equals((Credentials)that); + return false; + } + + public boolean equals(Credentials that) { + if (that == null) + return false; + + boolean this_present_creds = true && this.is_set_creds(); + boolean that_present_creds = true && that.is_set_creds(); + if (this_present_creds || that_present_creds) { + if (!(this_present_creds && that_present_creds)) + return false; + if (!this.creds.equals(that.creds)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_creds = true && (is_set_creds()); + builder.append(present_creds); + if (present_creds) + builder.append(creds); + + return builder.toHashCode(); + } + + public int compareTo(Credentials other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + Credentials typedOther = (Credentials)other; + + lastComparison = Boolean.valueOf(is_set_creds()).compareTo(typedOther.is_set_creds()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_creds()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, typedOther.creds); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField field; + iprot.readStructBegin(); + while (true) + { + field = iprot.readFieldBegin(); + if (field.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (field.id) { + case 1: // CREDS + if (field.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map163 = iprot.readMapBegin(); + this.creds = new HashMap(2*_map163.size); + for (int _i164 = 0; _i164 < _map163.size; ++_i164) + { + String _key165; // required + String _val166; // required + _key165 = iprot.readString(); + _val166 = iprot.readString(); + this.creds.put(_key165, _val166); + } + iprot.readMapEnd(); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (this.creds != null) { + oprot.writeFieldBegin(CREDS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, this.creds.size())); + for (Map.Entry _iter167 : this.creds.entrySet()) + { + oprot.writeString(_iter167.getKey()); + oprot.writeString(_iter167.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("Credentials("); + boolean first = true; + + sb.append("creds:"); + if (this.creds == null) { + sb.append("null"); + } else { + sb.append(this.creds); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_creds()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'creds' is unset! Struct:" + toString()); + } + + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java b/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java index 7922340fedb..06c4f5c473f 100644 --- a/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java +++ b/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java @@ -42,7 +42,7 @@ public class DistributedRPC { public interface Iface { - public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift.TException; + public String execute(String functionName, String funcArgs) throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException; } @@ -72,7 +72,7 @@ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.prot super(iprot, oprot); } - public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift.TException + public String execute(String functionName, String funcArgs) throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException { send_execute(functionName, funcArgs); return recv_execute(); @@ -86,7 +86,7 @@ public void send_execute(String functionName, String funcArgs) throws org.apache sendBase("execute", args); } - public String recv_execute() throws DRPCExecutionException, org.apache.thrift.TException + public String recv_execute() throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException { execute_result result = new execute_result(); receiveBase(result, "execute"); @@ -96,6 +96,9 @@ public String recv_execute() throws DRPCExecutionException, org.apache.thrift.TE if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "execute failed: unknown result"); } @@ -142,7 +145,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public String getResult() throws DRPCExecutionException, org.apache.thrift.TException { + public String getResult() throws DRPCExecutionException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -184,6 +187,8 @@ protected execute_result getResult(I iface, execute_args args) throws org.apache result.success = iface.execute(args.functionName, args.funcArgs); } catch (DRPCExecutionException e) { result.e = e; + } catch (AuthorizationException aze) { + result.aze = aze; } return result; } @@ -590,14 +595,17 @@ public static class execute_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -616,6 +624,8 @@ public static _Fields findByThriftId(int fieldId) { return SUCCESS; case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -664,6 +674,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(execute_result.class, metaDataMap); } @@ -673,11 +685,13 @@ public execute_result() { public execute_result( String success, - DRPCExecutionException e) + DRPCExecutionException e, + AuthorizationException aze) { this(); this.success = success; this.e = e; + this.aze = aze; } /** @@ -690,6 +704,9 @@ public execute_result(execute_result other) { if (other.is_set_e()) { this.e = new DRPCExecutionException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public execute_result deepCopy() { @@ -700,6 +717,7 @@ public execute_result deepCopy() { public void clear() { this.success = null; this.e = null; + this.aze = null; } public String get_success() { @@ -748,6 +766,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -766,6 +807,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -777,6 +826,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -792,6 +844,8 @@ public boolean isSet(_Fields field) { return is_set_success(); case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -827,6 +881,15 @@ public boolean equals(execute_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -844,6 +907,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -875,6 +943,16 @@ public int compareTo(execute_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -907,6 +985,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -927,6 +1013,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -952,6 +1042,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } diff --git a/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java b/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java index dc5bb4579be..e236e41f83a 100644 --- a/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java +++ b/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java @@ -42,11 +42,11 @@ public class DistributedRPCInvocations { public interface Iface { - public void result(String id, String result) throws org.apache.thrift.TException; + public void result(String id, String result) throws AuthorizationException, org.apache.thrift.TException; - public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift.TException; + public DRPCRequest fetchRequest(String functionName) throws AuthorizationException, org.apache.thrift.TException; - public void failRequest(String id) throws org.apache.thrift.TException; + public void failRequest(String id) throws AuthorizationException, org.apache.thrift.TException; } @@ -80,7 +80,7 @@ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.prot super(iprot, oprot); } - public void result(String id, String result) throws org.apache.thrift.TException + public void result(String id, String result) throws AuthorizationException, org.apache.thrift.TException { send_result(id, result); recv_result(); @@ -94,14 +94,17 @@ public void send_result(String id, String result) throws org.apache.thrift.TExce sendBase("result", args); } - public void recv_result() throws org.apache.thrift.TException + public void recv_result() throws AuthorizationException, org.apache.thrift.TException { result_result result = new result_result(); receiveBase(result, "result"); + if (result.aze != null) { + throw result.aze; + } return; } - public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift.TException + public DRPCRequest fetchRequest(String functionName) throws AuthorizationException, org.apache.thrift.TException { send_fetchRequest(functionName); return recv_fetchRequest(); @@ -114,17 +117,20 @@ public void send_fetchRequest(String functionName) throws org.apache.thrift.TExc sendBase("fetchRequest", args); } - public DRPCRequest recv_fetchRequest() throws org.apache.thrift.TException + public DRPCRequest recv_fetchRequest() throws AuthorizationException, org.apache.thrift.TException { fetchRequest_result result = new fetchRequest_result(); receiveBase(result, "fetchRequest"); if (result.is_set_success()) { return result.success; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result"); } - public void failRequest(String id) throws org.apache.thrift.TException + public void failRequest(String id) throws AuthorizationException, org.apache.thrift.TException { send_failRequest(id); recv_failRequest(); @@ -137,10 +143,13 @@ public void send_failRequest(String id) throws org.apache.thrift.TException sendBase("failRequest", args); } - public void recv_failRequest() throws org.apache.thrift.TException + public void recv_failRequest() throws AuthorizationException, org.apache.thrift.TException { failRequest_result result = new failRequest_result(); receiveBase(result, "failRequest"); + if (result.aze != null) { + throw result.aze; + } return; } @@ -187,7 +196,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws org.apache.thrift.TException { + public void getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -219,7 +228,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public DRPCRequest getResult() throws org.apache.thrift.TException { + public DRPCRequest getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -251,7 +260,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws org.apache.thrift.TException { + public void getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -291,7 +300,11 @@ protected result_args getEmptyArgsInstance() { protected result_result getResult(I iface, result_args args) throws org.apache.thrift.TException { result_result result = new result_result(); - iface.result(args.id, args.result); + try { + iface.result(args.id, args.result); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -307,7 +320,11 @@ protected fetchRequest_args getEmptyArgsInstance() { protected fetchRequest_result getResult(I iface, fetchRequest_args args) throws org.apache.thrift.TException { fetchRequest_result result = new fetchRequest_result(); - result.success = iface.fetchRequest(args.functionName); + try { + result.success = iface.fetchRequest(args.functionName); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -323,7 +340,11 @@ protected failRequest_args getEmptyArgsInstance() { protected failRequest_result getResult(I iface, failRequest_args args) throws org.apache.thrift.TException { failRequest_result result = new failRequest_result(); - iface.failRequest(args.id); + try { + iface.failRequest(args.id); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -727,11 +748,13 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException public static class result_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("result_result"); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { -; + AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -746,6 +769,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { + case 1: // AZE + return AZE; default: return null; } @@ -784,9 +809,14 @@ public String getFieldName() { return _fieldName; } } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(result_result.class, metaDataMap); } @@ -794,10 +824,20 @@ public String getFieldName() { public result_result() { } + public result_result( + AuthorizationException aze) + { + this(); + this.aze = aze; + } + /** * Performs a deep copy on other. */ public result_result(result_result other) { + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public result_result deepCopy() { @@ -806,15 +846,50 @@ public result_result deepCopy() { @Override public void clear() { + this.aze = null; + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } } public void setFieldValue(_Fields field, Object value) { switch (field) { + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } public Object getFieldValue(_Fields field) { switch (field) { + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -826,6 +901,8 @@ public boolean isSet(_Fields field) { } switch (field) { + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -843,6 +920,15 @@ public boolean equals(result_result that) { if (that == null) return false; + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -850,6 +936,11 @@ public boolean equals(result_result that) { public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -861,6 +952,16 @@ public int compareTo(result_result other) { int lastComparison = 0; result_result typedOther = (result_result)other; + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -878,6 +979,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t break; } switch (field.id) { + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -890,6 +999,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { oprot.writeStructBegin(STRUCT_DESC); + if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -899,6 +1013,13 @@ public String toString() { StringBuilder sb = new StringBuilder("result_result("); boolean first = true; + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -1230,12 +1351,15 @@ public static class fetchRequest_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -1252,6 +1376,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; + case 1: // AZE + return AZE; default: return null; } @@ -1298,6 +1424,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DRPCRequest.class))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_result.class, metaDataMap); } @@ -1306,10 +1434,12 @@ public fetchRequest_result() { } public fetchRequest_result( - DRPCRequest success) + DRPCRequest success, + AuthorizationException aze) { this(); this.success = success; + this.aze = aze; } /** @@ -1319,6 +1449,9 @@ public fetchRequest_result(fetchRequest_result other) { if (other.is_set_success()) { this.success = new DRPCRequest(other.success); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public fetchRequest_result deepCopy() { @@ -1328,6 +1461,7 @@ public fetchRequest_result deepCopy() { @Override public void clear() { this.success = null; + this.aze = null; } public DRPCRequest get_success() { @@ -1353,6 +1487,29 @@ public void set_success_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -1363,6 +1520,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -1371,6 +1536,9 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -1384,6 +1552,8 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -1410,6 +1580,15 @@ public boolean equals(fetchRequest_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -1422,6 +1601,11 @@ public int hashCode() { if (present_success) builder.append(success); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -1443,6 +1627,16 @@ public int compareTo(fetchRequest_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -1468,6 +1662,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -1484,6 +1686,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(SUCCESS_FIELD_DESC); this.success.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -1501,6 +1707,14 @@ public String toString() { sb.append(this.success); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -1831,11 +2045,13 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException public static class failRequest_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("failRequest_result"); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { -; + AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -1850,6 +2066,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { + case 1: // AZE + return AZE; default: return null; } @@ -1888,9 +2106,14 @@ public String getFieldName() { return _fieldName; } } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(failRequest_result.class, metaDataMap); } @@ -1898,10 +2121,20 @@ public String getFieldName() { public failRequest_result() { } + public failRequest_result( + AuthorizationException aze) + { + this(); + this.aze = aze; + } + /** * Performs a deep copy on other. */ public failRequest_result(failRequest_result other) { + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public failRequest_result deepCopy() { @@ -1910,15 +2143,50 @@ public failRequest_result deepCopy() { @Override public void clear() { + this.aze = null; + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } } public void setFieldValue(_Fields field, Object value) { switch (field) { + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } public Object getFieldValue(_Fields field) { switch (field) { + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -1930,6 +2198,8 @@ public boolean isSet(_Fields field) { } switch (field) { + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -1947,6 +2217,15 @@ public boolean equals(failRequest_result that) { if (that == null) return false; + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -1954,6 +2233,11 @@ public boolean equals(failRequest_result that) { public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -1965,6 +2249,16 @@ public int compareTo(failRequest_result other) { int lastComparison = 0; failRequest_result typedOther = (failRequest_result)other; + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -1982,6 +2276,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t break; } switch (field.id) { + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -1994,6 +2296,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { oprot.writeStructBegin(STRUCT_DESC); + if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -2003,6 +2310,13 @@ public String toString() { StringBuilder sb = new StringBuilder("failRequest_result("); boolean first = true; + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java index e84e12dca6e..7d049016b63 100644 --- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java +++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java @@ -42,41 +42,43 @@ public class Nimbus { public interface Iface { - public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException; + public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException; - public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException; + public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException; - public void killTopology(String name) throws NotAliveException, org.apache.thrift.TException; + public void killTopology(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; - public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift.TException; + public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; - public void activate(String name) throws NotAliveException, org.apache.thrift.TException; + public void activate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; - public void deactivate(String name) throws NotAliveException, org.apache.thrift.TException; + public void deactivate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; - public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException; + public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException; - public String beginFileUpload() throws org.apache.thrift.TException; + public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException; - public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift.TException; + public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException; - public void finishFileUpload(String location) throws org.apache.thrift.TException; + public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException; - public String beginFileDownload(String file) throws org.apache.thrift.TException; + public void finishFileUpload(String location) throws AuthorizationException, org.apache.thrift.TException; - public ByteBuffer downloadChunk(String id) throws org.apache.thrift.TException; + public String beginFileDownload(String file) throws AuthorizationException, org.apache.thrift.TException; - public String getNimbusConf() throws org.apache.thrift.TException; + public ByteBuffer downloadChunk(String id) throws AuthorizationException, org.apache.thrift.TException; - public ClusterSummary getClusterInfo() throws org.apache.thrift.TException; + public String getNimbusConf() throws AuthorizationException, org.apache.thrift.TException; - public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift.TException; + public ClusterSummary getClusterInfo() throws AuthorizationException, org.apache.thrift.TException; - public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift.TException; + public TopologyInfo getTopologyInfo(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; - public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift.TException; + public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; - public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift.TException; + public StormTopology getTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; + + public StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; } @@ -96,6 +98,8 @@ public interface AsyncIface { public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; @@ -140,7 +144,7 @@ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.prot super(iprot, oprot); } - public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException + public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { send_submitTopology(name, uploadedJarLocation, jsonConf, topology); recv_submitTopology(); @@ -156,7 +160,7 @@ public void send_submitTopology(String name, String uploadedJarLocation, String sendBase("submitTopology", args); } - public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException + public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { submitTopology_result result = new submitTopology_result(); receiveBase(result, "submitTopology"); @@ -166,10 +170,13 @@ public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyE if (result.ite != null) { throw result.ite; } + if (result.aze != null) { + throw result.aze; + } return; } - public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException + public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options); recv_submitTopologyWithOpts(); @@ -186,7 +193,7 @@ public void send_submitTopologyWithOpts(String name, String uploadedJarLocation, sendBase("submitTopologyWithOpts", args); } - public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException + public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); receiveBase(result, "submitTopologyWithOpts"); @@ -196,10 +203,13 @@ public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidT if (result.ite != null) { throw result.ite; } + if (result.aze != null) { + throw result.aze; + } return; } - public void killTopology(String name) throws NotAliveException, org.apache.thrift.TException + public void killTopology(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_killTopology(name); recv_killTopology(); @@ -212,17 +222,20 @@ public void send_killTopology(String name) throws org.apache.thrift.TException sendBase("killTopology", args); } - public void recv_killTopology() throws NotAliveException, org.apache.thrift.TException + public void recv_killTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { killTopology_result result = new killTopology_result(); receiveBase(result, "killTopology"); if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } return; } - public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift.TException + public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_killTopologyWithOpts(name, options); recv_killTopologyWithOpts(); @@ -236,17 +249,20 @@ public void send_killTopologyWithOpts(String name, KillOptions options) throws o sendBase("killTopologyWithOpts", args); } - public void recv_killTopologyWithOpts() throws NotAliveException, org.apache.thrift.TException + public void recv_killTopologyWithOpts() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { killTopologyWithOpts_result result = new killTopologyWithOpts_result(); receiveBase(result, "killTopologyWithOpts"); if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } return; } - public void activate(String name) throws NotAliveException, org.apache.thrift.TException + public void activate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_activate(name); recv_activate(); @@ -259,17 +275,20 @@ public void send_activate(String name) throws org.apache.thrift.TException sendBase("activate", args); } - public void recv_activate() throws NotAliveException, org.apache.thrift.TException + public void recv_activate() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { activate_result result = new activate_result(); receiveBase(result, "activate"); if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } return; } - public void deactivate(String name) throws NotAliveException, org.apache.thrift.TException + public void deactivate(String name) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_deactivate(name); recv_deactivate(); @@ -282,17 +301,20 @@ public void send_deactivate(String name) throws org.apache.thrift.TException sendBase("deactivate", args); } - public void recv_deactivate() throws NotAliveException, org.apache.thrift.TException + public void recv_deactivate() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { deactivate_result result = new deactivate_result(); receiveBase(result, "deactivate"); if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } return; } - public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException + public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { send_rebalance(name, options); recv_rebalance(); @@ -306,7 +328,7 @@ public void send_rebalance(String name, RebalanceOptions options) throws org.apa sendBase("rebalance", args); } - public void recv_rebalance() throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException + public void recv_rebalance() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { rebalance_result result = new rebalance_result(); receiveBase(result, "rebalance"); @@ -316,10 +338,43 @@ public void recv_rebalance() throws NotAliveException, InvalidTopologyException, if (result.ite != null) { throw result.ite; } + if (result.aze != null) { + throw result.aze; + } + return; + } + + public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException + { + send_uploadNewCredentials(name, creds); + recv_uploadNewCredentials(); + } + + public void send_uploadNewCredentials(String name, Credentials creds) throws org.apache.thrift.TException + { + uploadNewCredentials_args args = new uploadNewCredentials_args(); + args.set_name(name); + args.set_creds(creds); + sendBase("uploadNewCredentials", args); + } + + public void recv_uploadNewCredentials() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException + { + uploadNewCredentials_result result = new uploadNewCredentials_result(); + receiveBase(result, "uploadNewCredentials"); + if (result.e != null) { + throw result.e; + } + if (result.ite != null) { + throw result.ite; + } + if (result.aze != null) { + throw result.aze; + } return; } - public String beginFileUpload() throws org.apache.thrift.TException + public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException { send_beginFileUpload(); return recv_beginFileUpload(); @@ -331,17 +386,20 @@ public void send_beginFileUpload() throws org.apache.thrift.TException sendBase("beginFileUpload", args); } - public String recv_beginFileUpload() throws org.apache.thrift.TException + public String recv_beginFileUpload() throws AuthorizationException, org.apache.thrift.TException { beginFileUpload_result result = new beginFileUpload_result(); receiveBase(result, "beginFileUpload"); if (result.is_set_success()) { return result.success; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result"); } - public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift.TException + public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException { send_uploadChunk(location, chunk); recv_uploadChunk(); @@ -355,14 +413,17 @@ public void send_uploadChunk(String location, ByteBuffer chunk) throws org.apach sendBase("uploadChunk", args); } - public void recv_uploadChunk() throws org.apache.thrift.TException + public void recv_uploadChunk() throws AuthorizationException, org.apache.thrift.TException { uploadChunk_result result = new uploadChunk_result(); receiveBase(result, "uploadChunk"); + if (result.aze != null) { + throw result.aze; + } return; } - public void finishFileUpload(String location) throws org.apache.thrift.TException + public void finishFileUpload(String location) throws AuthorizationException, org.apache.thrift.TException { send_finishFileUpload(location); recv_finishFileUpload(); @@ -375,14 +436,17 @@ public void send_finishFileUpload(String location) throws org.apache.thrift.TExc sendBase("finishFileUpload", args); } - public void recv_finishFileUpload() throws org.apache.thrift.TException + public void recv_finishFileUpload() throws AuthorizationException, org.apache.thrift.TException { finishFileUpload_result result = new finishFileUpload_result(); receiveBase(result, "finishFileUpload"); + if (result.aze != null) { + throw result.aze; + } return; } - public String beginFileDownload(String file) throws org.apache.thrift.TException + public String beginFileDownload(String file) throws AuthorizationException, org.apache.thrift.TException { send_beginFileDownload(file); return recv_beginFileDownload(); @@ -395,17 +459,20 @@ public void send_beginFileDownload(String file) throws org.apache.thrift.TExcept sendBase("beginFileDownload", args); } - public String recv_beginFileDownload() throws org.apache.thrift.TException + public String recv_beginFileDownload() throws AuthorizationException, org.apache.thrift.TException { beginFileDownload_result result = new beginFileDownload_result(); receiveBase(result, "beginFileDownload"); if (result.is_set_success()) { return result.success; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result"); } - public ByteBuffer downloadChunk(String id) throws org.apache.thrift.TException + public ByteBuffer downloadChunk(String id) throws AuthorizationException, org.apache.thrift.TException { send_downloadChunk(id); return recv_downloadChunk(); @@ -418,17 +485,20 @@ public void send_downloadChunk(String id) throws org.apache.thrift.TException sendBase("downloadChunk", args); } - public ByteBuffer recv_downloadChunk() throws org.apache.thrift.TException + public ByteBuffer recv_downloadChunk() throws AuthorizationException, org.apache.thrift.TException { downloadChunk_result result = new downloadChunk_result(); receiveBase(result, "downloadChunk"); if (result.is_set_success()) { return result.success; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); } - public String getNimbusConf() throws org.apache.thrift.TException + public String getNimbusConf() throws AuthorizationException, org.apache.thrift.TException { send_getNimbusConf(); return recv_getNimbusConf(); @@ -440,17 +510,20 @@ public void send_getNimbusConf() throws org.apache.thrift.TException sendBase("getNimbusConf", args); } - public String recv_getNimbusConf() throws org.apache.thrift.TException + public String recv_getNimbusConf() throws AuthorizationException, org.apache.thrift.TException { getNimbusConf_result result = new getNimbusConf_result(); receiveBase(result, "getNimbusConf"); if (result.is_set_success()) { return result.success; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result"); } - public ClusterSummary getClusterInfo() throws org.apache.thrift.TException + public ClusterSummary getClusterInfo() throws AuthorizationException, org.apache.thrift.TException { send_getClusterInfo(); return recv_getClusterInfo(); @@ -462,17 +535,20 @@ public void send_getClusterInfo() throws org.apache.thrift.TException sendBase("getClusterInfo", args); } - public ClusterSummary recv_getClusterInfo() throws org.apache.thrift.TException + public ClusterSummary recv_getClusterInfo() throws AuthorizationException, org.apache.thrift.TException { getClusterInfo_result result = new getClusterInfo_result(); receiveBase(result, "getClusterInfo"); if (result.is_set_success()) { return result.success; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result"); } - public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift.TException + public TopologyInfo getTopologyInfo(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_getTopologyInfo(id); return recv_getTopologyInfo(); @@ -485,7 +561,7 @@ public void send_getTopologyInfo(String id) throws org.apache.thrift.TException sendBase("getTopologyInfo", args); } - public TopologyInfo recv_getTopologyInfo() throws NotAliveException, org.apache.thrift.TException + public TopologyInfo recv_getTopologyInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { getTopologyInfo_result result = new getTopologyInfo_result(); receiveBase(result, "getTopologyInfo"); @@ -495,10 +571,13 @@ public TopologyInfo recv_getTopologyInfo() throws NotAliveException, org.apache. if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result"); } - public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift.TException + public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_getTopologyConf(id); return recv_getTopologyConf(); @@ -511,7 +590,7 @@ public void send_getTopologyConf(String id) throws org.apache.thrift.TException sendBase("getTopologyConf", args); } - public String recv_getTopologyConf() throws NotAliveException, org.apache.thrift.TException + public String recv_getTopologyConf() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { getTopologyConf_result result = new getTopologyConf_result(); receiveBase(result, "getTopologyConf"); @@ -521,10 +600,13 @@ public String recv_getTopologyConf() throws NotAliveException, org.apache.thrift if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result"); } - public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift.TException + public StormTopology getTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_getTopology(id); return recv_getTopology(); @@ -537,7 +619,7 @@ public void send_getTopology(String id) throws org.apache.thrift.TException sendBase("getTopology", args); } - public StormTopology recv_getTopology() throws NotAliveException, org.apache.thrift.TException + public StormTopology recv_getTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { getTopology_result result = new getTopology_result(); receiveBase(result, "getTopology"); @@ -547,10 +629,13 @@ public StormTopology recv_getTopology() throws NotAliveException, org.apache.thr if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopology failed: unknown result"); } - public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift.TException + public StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_getUserTopology(id); return recv_getUserTopology(); @@ -563,7 +648,7 @@ public void send_getUserTopology(String id) throws org.apache.thrift.TException sendBase("getUserTopology", args); } - public StormTopology recv_getUserTopology() throws NotAliveException, org.apache.thrift.TException + public StormTopology recv_getUserTopology() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { getUserTopology_result result = new getUserTopology_result(); receiveBase(result, "getUserTopology"); @@ -573,6 +658,9 @@ public StormTopology recv_getUserTopology() throws NotAliveException, org.apache if (result.e != null) { throw result.e; } + if (result.aze != null) { + throw result.aze; + } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result"); } @@ -625,7 +713,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException { + public void getResult() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -669,7 +757,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws AlreadyAliveException, InvalidTopologyException, org.apache.thrift.TException { + public void getResult() throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -701,7 +789,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws NotAliveException, org.apache.thrift.TException { + public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -736,7 +824,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws NotAliveException, org.apache.thrift.TException { + public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -768,7 +856,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws NotAliveException, org.apache.thrift.TException { + public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -800,7 +888,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws NotAliveException, org.apache.thrift.TException { + public void getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -835,7 +923,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException { + public void getResult() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -845,6 +933,41 @@ public void getResult() throws NotAliveException, InvalidTopologyException, org. } } + public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + uploadNewCredentials_call method_call = new uploadNewCredentials_call(name, creds, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class uploadNewCredentials_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + private Credentials creds; + public uploadNewCredentials_call(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + this.creds = creds; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("uploadNewCredentials", org.apache.thrift.protocol.TMessageType.CALL, 0)); + uploadNewCredentials_args args = new uploadNewCredentials_args(); + args.set_name(name); + args.set_creds(creds); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_uploadNewCredentials(); + } + } + public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); beginFileUpload_call method_call = new beginFileUpload_call(resultHandler, this, ___protocolFactory, ___transport); @@ -864,7 +987,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public String getResult() throws org.apache.thrift.TException { + public String getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -899,7 +1022,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws org.apache.thrift.TException { + public void getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -931,7 +1054,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public void getResult() throws org.apache.thrift.TException { + public void getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -963,7 +1086,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public String getResult() throws org.apache.thrift.TException { + public String getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -995,7 +1118,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public ByteBuffer getResult() throws org.apache.thrift.TException { + public ByteBuffer getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -1024,7 +1147,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public String getResult() throws org.apache.thrift.TException { + public String getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -1053,7 +1176,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public ClusterSummary getResult() throws org.apache.thrift.TException { + public ClusterSummary getResult() throws AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -1085,7 +1208,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public TopologyInfo getResult() throws NotAliveException, org.apache.thrift.TException { + public TopologyInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -1117,7 +1240,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public String getResult() throws NotAliveException, org.apache.thrift.TException { + public String getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -1149,7 +1272,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public StormTopology getResult() throws NotAliveException, org.apache.thrift.TException { + public StormTopology getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -1181,7 +1304,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public StormTopology getResult() throws NotAliveException, org.apache.thrift.TException { + public StormTopology getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } @@ -1211,6 +1334,7 @@ protected Processor(I iface, Map extends org.apache.thrift.ProcessFunction { + public uploadNewCredentials() { + super("uploadNewCredentials"); + } + + protected uploadNewCredentials_args getEmptyArgsInstance() { + return new uploadNewCredentials_args(); + } + + protected uploadNewCredentials_result getResult(I iface, uploadNewCredentials_args args) throws org.apache.thrift.TException { + uploadNewCredentials_result result = new uploadNewCredentials_result(); + try { + iface.uploadNewCredentials(args.name, args.creds); + } catch (NotAliveException e) { + result.e = e; + } catch (InvalidTopologyException ite) { + result.ite = ite; + } catch (AuthorizationException aze) { + result.aze = aze; } return result; } @@ -1382,7 +1544,11 @@ protected beginFileUpload_args getEmptyArgsInstance() { protected beginFileUpload_result getResult(I iface, beginFileUpload_args args) throws org.apache.thrift.TException { beginFileUpload_result result = new beginFileUpload_result(); - result.success = iface.beginFileUpload(); + try { + result.success = iface.beginFileUpload(); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -1398,7 +1564,11 @@ protected uploadChunk_args getEmptyArgsInstance() { protected uploadChunk_result getResult(I iface, uploadChunk_args args) throws org.apache.thrift.TException { uploadChunk_result result = new uploadChunk_result(); - iface.uploadChunk(args.location, args.chunk); + try { + iface.uploadChunk(args.location, args.chunk); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -1414,7 +1584,11 @@ protected finishFileUpload_args getEmptyArgsInstance() { protected finishFileUpload_result getResult(I iface, finishFileUpload_args args) throws org.apache.thrift.TException { finishFileUpload_result result = new finishFileUpload_result(); - iface.finishFileUpload(args.location); + try { + iface.finishFileUpload(args.location); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -1430,7 +1604,11 @@ protected beginFileDownload_args getEmptyArgsInstance() { protected beginFileDownload_result getResult(I iface, beginFileDownload_args args) throws org.apache.thrift.TException { beginFileDownload_result result = new beginFileDownload_result(); - result.success = iface.beginFileDownload(args.file); + try { + result.success = iface.beginFileDownload(args.file); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -1446,7 +1624,11 @@ protected downloadChunk_args getEmptyArgsInstance() { protected downloadChunk_result getResult(I iface, downloadChunk_args args) throws org.apache.thrift.TException { downloadChunk_result result = new downloadChunk_result(); - result.success = iface.downloadChunk(args.id); + try { + result.success = iface.downloadChunk(args.id); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -1462,7 +1644,11 @@ protected getNimbusConf_args getEmptyArgsInstance() { protected getNimbusConf_result getResult(I iface, getNimbusConf_args args) throws org.apache.thrift.TException { getNimbusConf_result result = new getNimbusConf_result(); - result.success = iface.getNimbusConf(); + try { + result.success = iface.getNimbusConf(); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -1478,7 +1664,11 @@ protected getClusterInfo_args getEmptyArgsInstance() { protected getClusterInfo_result getResult(I iface, getClusterInfo_args args) throws org.apache.thrift.TException { getClusterInfo_result result = new getClusterInfo_result(); - result.success = iface.getClusterInfo(); + try { + result.success = iface.getClusterInfo(); + } catch (AuthorizationException aze) { + result.aze = aze; + } return result; } } @@ -1498,6 +1688,8 @@ protected getTopologyInfo_result getResult(I iface, getTopologyInfo_args args) t result.success = iface.getTopologyInfo(args.id); } catch (NotAliveException e) { result.e = e; + } catch (AuthorizationException aze) { + result.aze = aze; } return result; } @@ -1518,6 +1710,8 @@ protected getTopologyConf_result getResult(I iface, getTopologyConf_args args) t result.success = iface.getTopologyConf(args.id); } catch (NotAliveException e) { result.e = e; + } catch (AuthorizationException aze) { + result.aze = aze; } return result; } @@ -1538,6 +1732,8 @@ protected getTopology_result getResult(I iface, getTopology_args args) throws or result.success = iface.getTopology(args.id); } catch (NotAliveException e) { result.e = e; + } catch (AuthorizationException aze) { + result.aze = aze; } return result; } @@ -1558,6 +1754,8 @@ protected getUserTopology_result getResult(I iface, getUserTopology_args args) t result.success = iface.getUserTopology(args.id); } catch (NotAliveException e) { result.e = e; + } catch (AuthorizationException aze) { + result.aze = aze; } return result; } @@ -2151,14 +2349,17 @@ public static class submitTopology_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -2177,6 +2378,8 @@ public static _Fields findByThriftId(int fieldId) { return E; case 2: // ITE return ITE; + case 3: // AZE + return AZE; default: return null; } @@ -2225,6 +2428,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(submitTopology_result.class, metaDataMap); } @@ -2234,11 +2439,13 @@ public submitTopology_result() { public submitTopology_result( AlreadyAliveException e, - InvalidTopologyException ite) + InvalidTopologyException ite, + AuthorizationException aze) { this(); this.e = e; this.ite = ite; + this.aze = aze; } /** @@ -2251,6 +2458,9 @@ public submitTopology_result(submitTopology_result other) { if (other.is_set_ite()) { this.ite = new InvalidTopologyException(other.ite); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public submitTopology_result deepCopy() { @@ -2261,6 +2471,7 @@ public submitTopology_result deepCopy() { public void clear() { this.e = null; this.ite = null; + this.aze = null; } public AlreadyAliveException get_e() { @@ -2309,6 +2520,29 @@ public void set_ite_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case E: @@ -2327,6 +2561,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -2338,6 +2580,9 @@ public Object getFieldValue(_Fields field) { case ITE: return get_ite(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -2353,6 +2598,8 @@ public boolean isSet(_Fields field) { return is_set_e(); case ITE: return is_set_ite(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -2388,6 +2635,15 @@ public boolean equals(submitTopology_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -2405,6 +2661,11 @@ public int hashCode() { if (present_ite) builder.append(ite); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -2436,6 +2697,16 @@ public int compareTo(submitTopology_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -2469,6 +2740,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 3: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -2489,6 +2768,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(ITE_FIELD_DESC); this.ite.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -2514,6 +2797,14 @@ public String toString() { sb.append(this.ite); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -3220,14 +3511,17 @@ public static class submitTopologyWithOpts_result implements org.apache.thrift.T private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final org.apache.thrift.protocol.TField ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)3); private AlreadyAliveException e; // required private InvalidTopologyException ite; // required + private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { E((short)1, "e"), - ITE((short)2, "ite"); + ITE((short)2, "ite"), + AZE((short)3, "aze"); private static final Map byName = new HashMap(); @@ -3246,6 +3540,8 @@ public static _Fields findByThriftId(int fieldId) { return E; case 2: // ITE return ITE; + case 3: // AZE + return AZE; default: return null; } @@ -3294,6 +3590,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(submitTopologyWithOpts_result.class, metaDataMap); } @@ -3303,11 +3601,13 @@ public submitTopologyWithOpts_result() { public submitTopologyWithOpts_result( AlreadyAliveException e, - InvalidTopologyException ite) + InvalidTopologyException ite, + AuthorizationException aze) { this(); this.e = e; this.ite = ite; + this.aze = aze; } /** @@ -3320,6 +3620,9 @@ public submitTopologyWithOpts_result(submitTopologyWithOpts_result other) { if (other.is_set_ite()) { this.ite = new InvalidTopologyException(other.ite); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public submitTopologyWithOpts_result deepCopy() { @@ -3330,6 +3633,7 @@ public submitTopologyWithOpts_result deepCopy() { public void clear() { this.e = null; this.ite = null; + this.aze = null; } public AlreadyAliveException get_e() { @@ -3378,6 +3682,29 @@ public void set_ite_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case E: @@ -3396,6 +3723,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -3407,6 +3742,9 @@ public Object getFieldValue(_Fields field) { case ITE: return get_ite(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -3422,6 +3760,8 @@ public boolean isSet(_Fields field) { return is_set_e(); case ITE: return is_set_ite(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -3457,6 +3797,15 @@ public boolean equals(submitTopologyWithOpts_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -3474,6 +3823,11 @@ public int hashCode() { if (present_ite) builder.append(ite); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -3505,6 +3859,16 @@ public int compareTo(submitTopologyWithOpts_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -3538,6 +3902,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 3: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -3558,6 +3930,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(ITE_FIELD_DESC); this.ite.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -3583,6 +3959,14 @@ public String toString() { sb.append(this.ite); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -3914,12 +4298,15 @@ public static class killTopology_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -3936,6 +4323,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -3982,6 +4371,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(killTopology_result.class, metaDataMap); } @@ -3990,10 +4381,12 @@ public killTopology_result() { } public killTopology_result( - NotAliveException e) + NotAliveException e, + AuthorizationException aze) { this(); this.e = e; + this.aze = aze; } /** @@ -4003,6 +4396,9 @@ public killTopology_result(killTopology_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public killTopology_result deepCopy() { @@ -4012,6 +4408,7 @@ public killTopology_result deepCopy() { @Override public void clear() { this.e = null; + this.aze = null; } public NotAliveException get_e() { @@ -4037,6 +4434,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case E: @@ -4047,6 +4467,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -4055,6 +4483,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -4068,6 +4499,8 @@ public boolean isSet(_Fields field) { switch (field) { case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -4094,6 +4527,15 @@ public boolean equals(killTopology_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -4106,6 +4548,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -4127,6 +4574,16 @@ public int compareTo(killTopology_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -4152,6 +4609,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -4168,6 +4633,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -4185,6 +4654,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -4610,12 +5087,15 @@ public static class killTopologyWithOpts_result implements org.apache.thrift.TBa private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("killTopologyWithOpts_result"); private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); private NotAliveException e; // required + private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - E((short)1, "e"); + E((short)1, "e"), + AZE((short)2, "aze"); private static final Map byName = new HashMap(); @@ -4632,6 +5112,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -4678,6 +5160,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(killTopologyWithOpts_result.class, metaDataMap); } @@ -4686,10 +5170,12 @@ public killTopologyWithOpts_result() { } public killTopologyWithOpts_result( - NotAliveException e) + NotAliveException e, + AuthorizationException aze) { this(); this.e = e; + this.aze = aze; } /** @@ -4699,6 +5185,9 @@ public killTopologyWithOpts_result(killTopologyWithOpts_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public killTopologyWithOpts_result deepCopy() { @@ -4708,6 +5197,7 @@ public killTopologyWithOpts_result deepCopy() { @Override public void clear() { this.e = null; + this.aze = null; } public NotAliveException get_e() { @@ -4733,6 +5223,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case E: @@ -4743,6 +5256,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -4751,6 +5272,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -4764,6 +5288,8 @@ public boolean isSet(_Fields field) { switch (field) { case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -4790,6 +5316,15 @@ public boolean equals(killTopologyWithOpts_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -4802,6 +5337,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -4823,6 +5363,16 @@ public int compareTo(killTopologyWithOpts_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -4848,7 +5398,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; - default: + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } iprot.readFieldEnd(); @@ -4864,6 +5422,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -4881,6 +5443,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -5212,12 +5782,15 @@ public static class activate_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -5234,6 +5807,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -5280,6 +5855,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(activate_result.class, metaDataMap); } @@ -5288,10 +5865,12 @@ public activate_result() { } public activate_result( - NotAliveException e) + NotAliveException e, + AuthorizationException aze) { this(); this.e = e; + this.aze = aze; } /** @@ -5301,6 +5880,9 @@ public activate_result(activate_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public activate_result deepCopy() { @@ -5310,6 +5892,7 @@ public activate_result deepCopy() { @Override public void clear() { this.e = null; + this.aze = null; } public NotAliveException get_e() { @@ -5335,6 +5918,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case E: @@ -5345,6 +5951,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -5353,6 +5967,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -5366,6 +5983,8 @@ public boolean isSet(_Fields field) { switch (field) { case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -5392,6 +6011,15 @@ public boolean equals(activate_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -5404,6 +6032,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -5425,6 +6058,16 @@ public int compareTo(activate_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -5450,6 +6093,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -5466,6 +6117,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -5483,6 +6138,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -5814,12 +6477,15 @@ public static class deactivate_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -5836,6 +6502,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -5882,6 +6550,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deactivate_result.class, metaDataMap); } @@ -5890,10 +6560,12 @@ public deactivate_result() { } public deactivate_result( - NotAliveException e) + NotAliveException e, + AuthorizationException aze) { this(); this.e = e; + this.aze = aze; } /** @@ -5903,6 +6575,9 @@ public deactivate_result(deactivate_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public deactivate_result deepCopy() { @@ -5912,6 +6587,7 @@ public deactivate_result deepCopy() { @Override public void clear() { this.e = null; + this.aze = null; } public NotAliveException get_e() { @@ -5937,6 +6613,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case E: @@ -5947,6 +6646,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -5955,6 +6662,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -5968,6 +6678,8 @@ public boolean isSet(_Fields field) { switch (field) { case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -5994,6 +6706,15 @@ public boolean equals(deactivate_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -6006,6 +6727,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -6027,6 +6753,16 @@ public int compareTo(deactivate_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -6052,6 +6788,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -6068,6 +6812,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -6085,6 +6833,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -6511,14 +7267,17 @@ public static class rebalance_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -6537,6 +7296,8 @@ public static _Fields findByThriftId(int fieldId) { return E; case 2: // ITE return ITE; + case 3: // AZE + return AZE; default: return null; } @@ -6585,6 +7346,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(rebalance_result.class, metaDataMap); } @@ -6594,11 +7357,13 @@ public rebalance_result() { public rebalance_result( NotAliveException e, - InvalidTopologyException ite) + InvalidTopologyException ite, + AuthorizationException aze) { this(); this.e = e; this.ite = ite; + this.aze = aze; } /** @@ -6611,6 +7376,9 @@ public rebalance_result(rebalance_result other) { if (other.is_set_ite()) { this.ite = new InvalidTopologyException(other.ite); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public rebalance_result deepCopy() { @@ -6621,6 +7389,7 @@ public rebalance_result deepCopy() { public void clear() { this.e = null; this.ite = null; + this.aze = null; } public NotAliveException get_e() { @@ -6669,6 +7438,29 @@ public void set_ite_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case E: @@ -6687,6 +7479,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -6698,6 +7498,9 @@ public Object getFieldValue(_Fields field) { case ITE: return get_ite(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -6713,6 +7516,8 @@ public boolean isSet(_Fields field) { return is_set_e(); case ITE: return is_set_ite(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -6748,6 +7553,15 @@ public boolean equals(rebalance_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -6765,6 +7579,11 @@ public int hashCode() { if (present_ite) builder.append(ite); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -6796,6 +7615,16 @@ public int compareTo(rebalance_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -6829,6 +7658,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 3: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -6849,6 +7686,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(ITE_FIELD_DESC); this.ite.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -6867,11 +7708,901 @@ public String toString() { } first = false; if (!first) sb.append(", "); - sb.append("ite:"); - if (this.ite == null) { + sb.append("ite:"); + if (this.ite == null) { + sb.append("null"); + } else { + sb.append(this.ite); + } + first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + } + + public static class uploadNewCredentials_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadNewCredentials_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField CREDS_FIELD_DESC = new org.apache.thrift.protocol.TField("creds", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private String name; // required + private Credentials creds; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + CREDS((short)2, "creds"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // CREDS + return CREDS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Credentials.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(uploadNewCredentials_args.class, metaDataMap); + } + + public uploadNewCredentials_args() { + } + + public uploadNewCredentials_args( + String name, + Credentials creds) + { + this(); + this.name = name; + this.creds = creds; + } + + /** + * Performs a deep copy on other. + */ + public uploadNewCredentials_args(uploadNewCredentials_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_creds()) { + this.creds = new Credentials(other.creds); + } + } + + public uploadNewCredentials_args deepCopy() { + return new uploadNewCredentials_args(this); + } + + @Override + public void clear() { + this.name = null; + this.creds = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public Credentials get_creds() { + return this.creds; + } + + public void set_creds(Credentials creds) { + this.creds = creds; + } + + public void unset_creds() { + this.creds = null; + } + + /** Returns true if field creds is set (has been assigned a value) and false otherwise */ + public boolean is_set_creds() { + return this.creds != null; + } + + public void set_creds_isSet(boolean value) { + if (!value) { + this.creds = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case CREDS: + if (value == null) { + unset_creds(); + } else { + set_creds((Credentials)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case CREDS: + return get_creds(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case CREDS: + return is_set_creds(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof uploadNewCredentials_args) + return this.equals((uploadNewCredentials_args)that); + return false; + } + + public boolean equals(uploadNewCredentials_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_creds = true && this.is_set_creds(); + boolean that_present_creds = true && that.is_set_creds(); + if (this_present_creds || that_present_creds) { + if (!(this_present_creds && that_present_creds)) + return false; + if (!this.creds.equals(that.creds)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_name = true && (is_set_name()); + builder.append(present_name); + if (present_name) + builder.append(name); + + boolean present_creds = true && (is_set_creds()); + builder.append(present_creds); + if (present_creds) + builder.append(creds); + + return builder.toHashCode(); + } + + public int compareTo(uploadNewCredentials_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + uploadNewCredentials_args typedOther = (uploadNewCredentials_args)other; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_creds()).compareTo(typedOther.is_set_creds()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_creds()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, typedOther.creds); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField field; + iprot.readStructBegin(); + while (true) + { + field = iprot.readFieldBegin(); + if (field.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (field.id) { + case 1: // NAME + if (field.type == org.apache.thrift.protocol.TType.STRING) { + this.name = iprot.readString(); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + case 2: // CREDS + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.creds = new Credentials(); + this.creds.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (this.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(this.name); + oprot.writeFieldEnd(); + } + if (this.creds != null) { + oprot.writeFieldBegin(CREDS_FIELD_DESC); + this.creds.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("uploadNewCredentials_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("creds:"); + if (this.creds == null) { + sb.append("null"); + } else { + sb.append(this.creds); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + } + + public static class uploadNewCredentials_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadNewCredentials_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private NotAliveException e; // required + private InvalidTopologyException ite; // required + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"), + ITE((short)2, "ite"), + AZE((short)3, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + case 2: // ITE + return ITE; + case 3: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(uploadNewCredentials_result.class, metaDataMap); + } + + public uploadNewCredentials_result() { + } + + public uploadNewCredentials_result( + NotAliveException e, + InvalidTopologyException ite, + AuthorizationException aze) + { + this(); + this.e = e; + this.ite = ite; + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public uploadNewCredentials_result(uploadNewCredentials_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_ite()) { + this.ite = new InvalidTopologyException(other.ite); + } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public uploadNewCredentials_result deepCopy() { + return new uploadNewCredentials_result(this); + } + + @Override + public void clear() { + this.e = null; + this.ite = null; + this.aze = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException get_ite() { + return this.ite; + } + + public void set_ite(InvalidTopologyException ite) { + this.ite = ite; + } + + public void unset_ite() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean is_set_ite() { + return this.ite != null; + } + + public void set_ite_isSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case ITE: + if (value == null) { + unset_ite(); + } else { + set_ite((InvalidTopologyException)value); + } + break; + + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + case ITE: + return get_ite(); + + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + case ITE: + return is_set_ite(); + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof uploadNewCredentials_result) + return this.equals((uploadNewCredentials_result)that); + return false; + } + + public boolean equals(uploadNewCredentials_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_ite = true && this.is_set_ite(); + boolean that_present_ite = true && that.is_set_ite(); + if (this_present_ite || that_present_ite) { + if (!(this_present_ite && that_present_ite)) + return false; + if (!this.ite.equals(that.ite)) + return false; + } + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_e = true && (is_set_e()); + builder.append(present_e); + if (present_e) + builder.append(e); + + boolean present_ite = true && (is_set_ite()); + builder.append(present_ite); + if (present_ite) + builder.append(ite); + + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + + return builder.toHashCode(); + } + + public int compareTo(uploadNewCredentials_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + uploadNewCredentials_result typedOther = (uploadNewCredentials_result)other; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, typedOther.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_ite()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ite, typedOther.ite); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField field; + iprot.readStructBegin(); + while (true) + { + field = iprot.readFieldBegin(); + if (field.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (field.id) { + case 1: // E + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.e = new NotAliveException(); + this.e.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + case 2: // ITE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.ite = new InvalidTopologyException(); + this.ite.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + case 3: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + oprot.writeStructBegin(STRUCT_DESC); + + if (this.is_set_e()) { + oprot.writeFieldBegin(E_FIELD_DESC); + this.e.write(oprot); + oprot.writeFieldEnd(); + } else if (this.is_set_ite()) { + oprot.writeFieldBegin(ITE_FIELD_DESC); + this.ite.write(oprot); + oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("uploadNewCredentials_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("ite:"); + if (this.ite == null) { + sb.append("null"); + } else { + sb.append(this.ite); + } + first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { sb.append("null"); } else { - sb.append(this.ite); + sb.append(this.aze); } first = false; sb.append(")"); @@ -7106,12 +8837,15 @@ public static class beginFileUpload_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -7128,6 +8862,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; + case 1: // AZE + return AZE; default: return null; } @@ -7174,6 +8910,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFileUpload_result.class, metaDataMap); } @@ -7182,10 +8920,12 @@ public beginFileUpload_result() { } public beginFileUpload_result( - String success) + String success, + AuthorizationException aze) { this(); this.success = success; + this.aze = aze; } /** @@ -7195,6 +8935,9 @@ public beginFileUpload_result(beginFileUpload_result other) { if (other.is_set_success()) { this.success = other.success; } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public beginFileUpload_result deepCopy() { @@ -7204,6 +8947,7 @@ public beginFileUpload_result deepCopy() { @Override public void clear() { this.success = null; + this.aze = null; } public String get_success() { @@ -7229,6 +8973,29 @@ public void set_success_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -7239,6 +9006,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -7247,6 +9022,9 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -7260,6 +9038,8 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -7286,6 +9066,15 @@ public boolean equals(beginFileUpload_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -7298,6 +9087,11 @@ public int hashCode() { if (present_success) builder.append(success); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -7319,6 +9113,16 @@ public int compareTo(beginFileUpload_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -7343,6 +9147,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -7359,6 +9171,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeString(this.success); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -7376,6 +9192,14 @@ public String toString() { sb.append(this.success); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -7809,11 +9633,13 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException public static class uploadChunk_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadChunk_result"); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { -; + AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -7828,6 +9654,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { + case 1: // AZE + return AZE; default: return null; } @@ -7866,9 +9694,14 @@ public String getFieldName() { return _fieldName; } } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(uploadChunk_result.class, metaDataMap); } @@ -7876,10 +9709,20 @@ public String getFieldName() { public uploadChunk_result() { } + public uploadChunk_result( + AuthorizationException aze) + { + this(); + this.aze = aze; + } + /** * Performs a deep copy on other. */ public uploadChunk_result(uploadChunk_result other) { + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public uploadChunk_result deepCopy() { @@ -7888,15 +9731,50 @@ public uploadChunk_result deepCopy() { @Override public void clear() { + this.aze = null; + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } } public void setFieldValue(_Fields field, Object value) { switch (field) { + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } public Object getFieldValue(_Fields field) { switch (field) { + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -7908,6 +9786,8 @@ public boolean isSet(_Fields field) { } switch (field) { + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -7925,6 +9805,15 @@ public boolean equals(uploadChunk_result that) { if (that == null) return false; + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -7932,6 +9821,11 @@ public boolean equals(uploadChunk_result that) { public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -7943,6 +9837,16 @@ public int compareTo(uploadChunk_result other) { int lastComparison = 0; uploadChunk_result typedOther = (uploadChunk_result)other; + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -7960,6 +9864,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t break; } switch (field.id) { + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -7972,6 +9884,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { oprot.writeStructBegin(STRUCT_DESC); + if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -7981,6 +9898,13 @@ public String toString() { StringBuilder sb = new StringBuilder("uploadChunk_result("); boolean first = true; + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -8311,11 +10235,13 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException public static class finishFileUpload_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("finishFileUpload_result"); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { -; + AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -8330,6 +10256,8 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { + case 1: // AZE + return AZE; default: return null; } @@ -8368,9 +10296,14 @@ public String getFieldName() { return _fieldName; } } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(finishFileUpload_result.class, metaDataMap); } @@ -8378,10 +10311,20 @@ public String getFieldName() { public finishFileUpload_result() { } + public finishFileUpload_result( + AuthorizationException aze) + { + this(); + this.aze = aze; + } + /** * Performs a deep copy on other. */ public finishFileUpload_result(finishFileUpload_result other) { + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public finishFileUpload_result deepCopy() { @@ -8390,15 +10333,50 @@ public finishFileUpload_result deepCopy() { @Override public void clear() { + this.aze = null; + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } } public void setFieldValue(_Fields field, Object value) { switch (field) { + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } public Object getFieldValue(_Fields field) { switch (field) { + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -8410,6 +10388,8 @@ public boolean isSet(_Fields field) { } switch (field) { + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -8427,6 +10407,15 @@ public boolean equals(finishFileUpload_result that) { if (that == null) return false; + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -8434,6 +10423,11 @@ public boolean equals(finishFileUpload_result that) { public int hashCode() { HashCodeBuilder builder = new HashCodeBuilder(); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -8445,6 +10439,16 @@ public int compareTo(finishFileUpload_result other) { int lastComparison = 0; finishFileUpload_result typedOther = (finishFileUpload_result)other; + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -8462,6 +10466,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t break; } switch (field.id) { + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -8474,6 +10486,11 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { oprot.writeStructBegin(STRUCT_DESC); + if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -8483,6 +10500,13 @@ public String toString() { StringBuilder sb = new StringBuilder("finishFileUpload_result("); boolean first = true; + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -8814,12 +10838,15 @@ public static class beginFileDownload_result implements org.apache.thrift.TBase< private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFileDownload_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1); private String success; // required + private AuthorizationException aze; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); + SUCCESS((short)0, "success"), + AZE((short)1, "aze"); private static final Map byName = new HashMap(); @@ -8836,6 +10863,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; + case 1: // AZE + return AZE; default: return null; } @@ -8882,6 +10911,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFileDownload_result.class, metaDataMap); } @@ -8890,10 +10921,12 @@ public beginFileDownload_result() { } public beginFileDownload_result( - String success) + String success, + AuthorizationException aze) { this(); this.success = success; + this.aze = aze; } /** @@ -8903,6 +10936,9 @@ public beginFileDownload_result(beginFileDownload_result other) { if (other.is_set_success()) { this.success = other.success; } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public beginFileDownload_result deepCopy() { @@ -8912,6 +10948,7 @@ public beginFileDownload_result deepCopy() { @Override public void clear() { this.success = null; + this.aze = null; } public String get_success() { @@ -8937,6 +10974,29 @@ public void set_success_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -8947,6 +11007,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -8955,6 +11023,9 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -8968,6 +11039,8 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -8994,6 +11067,15 @@ public boolean equals(beginFileDownload_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -9006,6 +11088,11 @@ public int hashCode() { if (present_success) builder.append(success); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -9027,6 +11114,16 @@ public int compareTo(beginFileDownload_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -9051,6 +11148,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -9067,6 +11172,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeString(this.success); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -9084,6 +11193,14 @@ public String toString() { sb.append(this.success); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -9415,12 +11532,15 @@ public static class downloadChunk_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -9437,6 +11557,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; + case 1: // AZE + return AZE; default: return null; } @@ -9483,6 +11605,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(downloadChunk_result.class, metaDataMap); } @@ -9491,10 +11615,12 @@ public downloadChunk_result() { } public downloadChunk_result( - ByteBuffer success) + ByteBuffer success, + AuthorizationException aze) { this(); this.success = success; + this.aze = aze; } /** @@ -9505,6 +11631,9 @@ public downloadChunk_result(downloadChunk_result other) { this.success = org.apache.thrift.TBaseHelper.copyBinary(other.success); ; } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public downloadChunk_result deepCopy() { @@ -9514,6 +11643,7 @@ public downloadChunk_result deepCopy() { @Override public void clear() { this.success = null; + this.aze = null; } public byte[] get_success() { @@ -9548,6 +11678,29 @@ public void set_success_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -9558,6 +11711,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -9566,6 +11727,9 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -9579,6 +11743,8 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -9605,6 +11771,15 @@ public boolean equals(downloadChunk_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -9617,6 +11792,11 @@ public int hashCode() { if (present_success) builder.append(success); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -9638,6 +11818,16 @@ public int compareTo(downloadChunk_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -9662,6 +11852,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -9678,6 +11876,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeBinary(this.success); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -9695,6 +11897,14 @@ public String toString() { org.apache.thrift.TBaseHelper.toString(this.success, sb); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -9927,12 +12137,15 @@ public static class getNimbusConf_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -9949,6 +12162,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; + case 1: // AZE + return AZE; default: return null; } @@ -9995,6 +12210,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNimbusConf_result.class, metaDataMap); } @@ -10003,10 +12220,12 @@ public getNimbusConf_result() { } public getNimbusConf_result( - String success) + String success, + AuthorizationException aze) { this(); this.success = success; + this.aze = aze; } /** @@ -10016,6 +12235,9 @@ public getNimbusConf_result(getNimbusConf_result other) { if (other.is_set_success()) { this.success = other.success; } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public getNimbusConf_result deepCopy() { @@ -10025,6 +12247,7 @@ public getNimbusConf_result deepCopy() { @Override public void clear() { this.success = null; + this.aze = null; } public String get_success() { @@ -10044,9 +12267,32 @@ public boolean is_set_success() { return this.success != null; } - public void set_success_isSet(boolean value) { + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { if (!value) { - this.success = null; + this.aze = null; } } @@ -10060,6 +12306,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -10068,6 +12322,9 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -10081,6 +12338,8 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -10107,6 +12366,15 @@ public boolean equals(getNimbusConf_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -10119,6 +12387,11 @@ public int hashCode() { if (present_success) builder.append(success); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -10140,6 +12413,16 @@ public int compareTo(getNimbusConf_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -10164,6 +12447,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -10180,6 +12471,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(SUCCESS_FIELD_DESC); oprot.writeString(this.success); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -10197,6 +12492,14 @@ public String toString() { sb.append(this.success); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -10429,12 +12732,15 @@ public static class getClusterInfo_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -10451,6 +12757,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; + case 1: // AZE + return AZE; default: return null; } @@ -10497,6 +12805,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClusterSummary.class))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getClusterInfo_result.class, metaDataMap); } @@ -10505,10 +12815,12 @@ public getClusterInfo_result() { } public getClusterInfo_result( - ClusterSummary success) + ClusterSummary success, + AuthorizationException aze) { this(); this.success = success; + this.aze = aze; } /** @@ -10518,6 +12830,9 @@ public getClusterInfo_result(getClusterInfo_result other) { if (other.is_set_success()) { this.success = new ClusterSummary(other.success); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public getClusterInfo_result deepCopy() { @@ -10527,6 +12842,7 @@ public getClusterInfo_result deepCopy() { @Override public void clear() { this.success = null; + this.aze = null; } public ClusterSummary get_success() { @@ -10552,6 +12868,29 @@ public void set_success_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -10562,6 +12901,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -10570,6 +12917,9 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return get_success(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -10583,6 +12933,8 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return is_set_success(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -10609,6 +12961,15 @@ public boolean equals(getClusterInfo_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -10621,6 +12982,11 @@ public int hashCode() { if (present_success) builder.append(success); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -10642,6 +13008,16 @@ public int compareTo(getClusterInfo_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -10667,6 +13043,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 1: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -10683,6 +13067,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(SUCCESS_FIELD_DESC); this.success.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -10700,6 +13088,14 @@ public String toString() { sb.append(this.success); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -11032,14 +13428,17 @@ public static class getTopologyInfo_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -11058,6 +13457,8 @@ public static _Fields findByThriftId(int fieldId) { return SUCCESS; case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -11106,6 +13507,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyInfo.class))); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfo_result.class, metaDataMap); } @@ -11115,11 +13518,13 @@ public getTopologyInfo_result() { public getTopologyInfo_result( TopologyInfo success, - NotAliveException e) + NotAliveException e, + AuthorizationException aze) { this(); this.success = success; this.e = e; + this.aze = aze; } /** @@ -11132,6 +13537,9 @@ public getTopologyInfo_result(getTopologyInfo_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public getTopologyInfo_result deepCopy() { @@ -11142,6 +13550,7 @@ public getTopologyInfo_result deepCopy() { public void clear() { this.success = null; this.e = null; + this.aze = null; } public TopologyInfo get_success() { @@ -11190,6 +13599,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -11208,6 +13640,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -11219,6 +13659,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -11234,6 +13677,8 @@ public boolean isSet(_Fields field) { return is_set_success(); case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -11269,6 +13714,15 @@ public boolean equals(getTopologyInfo_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -11286,6 +13740,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -11317,6 +13776,16 @@ public int compareTo(getTopologyInfo_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -11350,6 +13819,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -11370,6 +13847,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -11395,6 +13876,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -11727,14 +14216,17 @@ public static class getTopologyConf_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -11753,6 +14245,8 @@ public static _Fields findByThriftId(int fieldId) { return SUCCESS; case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -11801,6 +14295,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_result.class, metaDataMap); } @@ -11810,11 +14306,13 @@ public getTopologyConf_result() { public getTopologyConf_result( String success, - NotAliveException e) + NotAliveException e, + AuthorizationException aze) { this(); this.success = success; this.e = e; + this.aze = aze; } /** @@ -11827,6 +14325,9 @@ public getTopologyConf_result(getTopologyConf_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public getTopologyConf_result deepCopy() { @@ -11837,6 +14338,7 @@ public getTopologyConf_result deepCopy() { public void clear() { this.success = null; this.e = null; + this.aze = null; } public String get_success() { @@ -11885,6 +14387,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -11903,6 +14428,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -11914,6 +14447,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -11929,6 +14465,8 @@ public boolean isSet(_Fields field) { return is_set_success(); case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -11964,6 +14502,15 @@ public boolean equals(getTopologyConf_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -11981,6 +14528,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -12012,6 +14564,16 @@ public int compareTo(getTopologyConf_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -12044,6 +14606,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -12064,6 +14634,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -12089,6 +14663,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -12421,14 +15003,17 @@ public static class getTopology_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -12447,6 +15032,8 @@ public static _Fields findByThriftId(int fieldId) { return SUCCESS; case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -12495,6 +15082,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_result.class, metaDataMap); } @@ -12504,11 +15093,13 @@ public getTopology_result() { public getTopology_result( StormTopology success, - NotAliveException e) + NotAliveException e, + AuthorizationException aze) { this(); this.success = success; this.e = e; + this.aze = aze; } /** @@ -12521,6 +15112,9 @@ public getTopology_result(getTopology_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public getTopology_result deepCopy() { @@ -12531,6 +15125,7 @@ public getTopology_result deepCopy() { public void clear() { this.success = null; this.e = null; + this.aze = null; } public StormTopology get_success() { @@ -12579,6 +15174,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -12597,6 +15215,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -12608,6 +15234,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -12623,6 +15252,8 @@ public boolean isSet(_Fields field) { return is_set_success(); case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -12658,6 +15289,15 @@ public boolean equals(getTopology_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -12675,6 +15315,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -12706,6 +15351,16 @@ public int compareTo(getTopology_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -12739,6 +15394,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -12759,6 +15422,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -12784,6 +15451,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } @@ -13116,14 +15791,17 @@ public static class getUserTopology_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -13142,6 +15820,8 @@ public static _Fields findByThriftId(int fieldId) { return SUCCESS; case 1: // E return E; + case 2: // AZE + return AZE; default: return null; } @@ -13190,6 +15870,8 @@ public String getFieldName() { new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_result.class, metaDataMap); } @@ -13199,11 +15881,13 @@ public getUserTopology_result() { public getUserTopology_result( StormTopology success, - NotAliveException e) + NotAliveException e, + AuthorizationException aze) { this(); this.success = success; this.e = e; + this.aze = aze; } /** @@ -13216,6 +15900,9 @@ public getUserTopology_result(getUserTopology_result other) { if (other.is_set_e()) { this.e = new NotAliveException(other.e); } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } } public getUserTopology_result deepCopy() { @@ -13226,6 +15913,7 @@ public getUserTopology_result deepCopy() { public void clear() { this.success = null; this.e = null; + this.aze = null; } public StormTopology get_success() { @@ -13274,6 +15962,29 @@ public void set_e_isSet(boolean value) { } } + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: @@ -13292,6 +16003,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + } } @@ -13303,6 +16022,9 @@ public Object getFieldValue(_Fields field) { case E: return get_e(); + case AZE: + return get_aze(); + } throw new IllegalStateException(); } @@ -13318,6 +16040,8 @@ public boolean isSet(_Fields field) { return is_set_success(); case E: return is_set_e(); + case AZE: + return is_set_aze(); } throw new IllegalStateException(); } @@ -13353,6 +16077,15 @@ public boolean equals(getUserTopology_result that) { return false; } + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + return true; } @@ -13370,6 +16103,11 @@ public int hashCode() { if (present_e) builder.append(e); + boolean present_aze = true && (is_set_aze()); + builder.append(present_aze); + if (present_aze) + builder.append(aze); + return builder.toHashCode(); } @@ -13401,6 +16139,16 @@ public int compareTo(getUserTopology_result other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(typedOther.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, typedOther.aze); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -13434,6 +16182,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // AZE + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.aze = new AuthorizationException(); + this.aze.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -13454,6 +16210,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeFieldBegin(E_FIELD_DESC); this.e.write(oprot); oprot.writeFieldEnd(); + } else if (this.is_set_aze()) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + this.aze.write(oprot); + oprot.writeFieldEnd(); } oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -13479,6 +16239,14 @@ public String toString() { sb.append(this.e); } first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; sb.append(")"); return sb.toString(); } diff --git a/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java b/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java index 78d66c910a9..f885f69ba0f 100644 --- a/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java +++ b/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java @@ -42,8 +42,10 @@ public class SubmitOptions implements org.apache.thrift.TBase byName = new HashMap(); @@ -68,6 +71,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // INITIAL_STATUS return INITIAL_STATUS; + case 2: // CREDS + return CREDS; default: return null; } @@ -114,6 +119,8 @@ public String getFieldName() { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.INITIAL_STATUS, new org.apache.thrift.meta_data.FieldMetaData("initial_status", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyInitialStatus.class))); + tmpMap.put(_Fields.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Credentials.class))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SubmitOptions.class, metaDataMap); } @@ -135,6 +142,9 @@ public SubmitOptions(SubmitOptions other) { if (other.is_set_initial_status()) { this.initial_status = other.initial_status; } + if (other.is_set_creds()) { + this.creds = new Credentials(other.creds); + } } public SubmitOptions deepCopy() { @@ -144,6 +154,7 @@ public SubmitOptions deepCopy() { @Override public void clear() { this.initial_status = null; + this.creds = null; } /** @@ -177,6 +188,29 @@ public void set_initial_status_isSet(boolean value) { } } + public Credentials get_creds() { + return this.creds; + } + + public void set_creds(Credentials creds) { + this.creds = creds; + } + + public void unset_creds() { + this.creds = null; + } + + /** Returns true if field creds is set (has been assigned a value) and false otherwise */ + public boolean is_set_creds() { + return this.creds != null; + } + + public void set_creds_isSet(boolean value) { + if (!value) { + this.creds = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case INITIAL_STATUS: @@ -187,6 +221,14 @@ public void setFieldValue(_Fields field, Object value) { } break; + case CREDS: + if (value == null) { + unset_creds(); + } else { + set_creds((Credentials)value); + } + break; + } } @@ -195,6 +237,9 @@ public Object getFieldValue(_Fields field) { case INITIAL_STATUS: return get_initial_status(); + case CREDS: + return get_creds(); + } throw new IllegalStateException(); } @@ -208,6 +253,8 @@ public boolean isSet(_Fields field) { switch (field) { case INITIAL_STATUS: return is_set_initial_status(); + case CREDS: + return is_set_creds(); } throw new IllegalStateException(); } @@ -234,6 +281,15 @@ public boolean equals(SubmitOptions that) { return false; } + boolean this_present_creds = true && this.is_set_creds(); + boolean that_present_creds = true && that.is_set_creds(); + if (this_present_creds || that_present_creds) { + if (!(this_present_creds && that_present_creds)) + return false; + if (!this.creds.equals(that.creds)) + return false; + } + return true; } @@ -246,6 +302,11 @@ public int hashCode() { if (present_initial_status) builder.append(initial_status.getValue()); + boolean present_creds = true && (is_set_creds()); + builder.append(present_creds); + if (present_creds) + builder.append(creds); + return builder.toHashCode(); } @@ -267,6 +328,16 @@ public int compareTo(SubmitOptions other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_creds()).compareTo(typedOther.is_set_creds()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_creds()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, typedOther.creds); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -291,6 +362,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 2: // CREDS + if (field.type == org.apache.thrift.protocol.TType.STRUCT) { + this.creds = new Credentials(); + this.creds.read(iprot); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -309,6 +388,13 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeI32(this.initial_status.getValue()); oprot.writeFieldEnd(); } + if (this.creds != null) { + if (is_set_creds()) { + oprot.writeFieldBegin(CREDS_FIELD_DESC); + this.creds.write(oprot); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -325,6 +411,16 @@ public String toString() { sb.append(this.initial_status); } first = false; + if (is_set_creds()) { + if (!first) sb.append(", "); + sb.append("creds:"); + if (this.creds == null) { + sb.append("null"); + } else { + sb.append(this.creds); + } + first = false; + } sb.append(")"); return sb.toString(); } diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java index 2ec9cb37b23..2c36d4e32f1 100644 --- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java +++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java @@ -47,6 +47,8 @@ public class TopologyInfo implements org.apache.thrift.TBase executors; // required private String status; // required private Map> errors; // required + private String sched_status; // required + private String owner; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -62,7 +66,9 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { UPTIME_SECS((short)3, "uptime_secs"), EXECUTORS((short)4, "executors"), STATUS((short)5, "status"), - ERRORS((short)6, "errors"); + ERRORS((short)6, "errors"), + SCHED_STATUS((short)513, "sched_status"), + OWNER((short)514, "owner"); private static final Map byName = new HashMap(); @@ -89,6 +95,10 @@ public static _Fields findByThriftId(int fieldId) { return STATUS; case 6: // ERRORS return ERRORS; + case 513: // SCHED_STATUS + return SCHED_STATUS; + case 514: // OWNER + return OWNER; default: return null; } @@ -151,6 +161,10 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class))))); + tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap); } @@ -217,6 +231,12 @@ public TopologyInfo(TopologyInfo other) { } this.errors = __this__errors; } + if (other.is_set_sched_status()) { + this.sched_status = other.sched_status; + } + if (other.is_set_owner()) { + this.owner = other.owner; + } } public TopologyInfo deepCopy() { @@ -232,6 +252,8 @@ public void clear() { this.executors = null; this.status = null; this.errors = null; + this.sched_status = null; + this.owner = null; } public String get_id() { @@ -397,6 +419,52 @@ public void set_errors_isSet(boolean value) { } } + public String get_sched_status() { + return this.sched_status; + } + + public void set_sched_status(String sched_status) { + this.sched_status = sched_status; + } + + public void unset_sched_status() { + this.sched_status = null; + } + + /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */ + public boolean is_set_sched_status() { + return this.sched_status != null; + } + + public void set_sched_status_isSet(boolean value) { + if (!value) { + this.sched_status = null; + } + } + + public String get_owner() { + return this.owner; + } + + public void set_owner(String owner) { + this.owner = owner; + } + + public void unset_owner() { + this.owner = null; + } + + /** Returns true if field owner is set (has been assigned a value) and false otherwise */ + public boolean is_set_owner() { + return this.owner != null; + } + + public void set_owner_isSet(boolean value) { + if (!value) { + this.owner = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: @@ -447,6 +515,22 @@ public void setFieldValue(_Fields field, Object value) { } break; + case SCHED_STATUS: + if (value == null) { + unset_sched_status(); + } else { + set_sched_status((String)value); + } + break; + + case OWNER: + if (value == null) { + unset_owner(); + } else { + set_owner((String)value); + } + break; + } } @@ -470,6 +554,12 @@ public Object getFieldValue(_Fields field) { case ERRORS: return get_errors(); + case SCHED_STATUS: + return get_sched_status(); + + case OWNER: + return get_owner(); + } throw new IllegalStateException(); } @@ -493,6 +583,10 @@ public boolean isSet(_Fields field) { return is_set_status(); case ERRORS: return is_set_errors(); + case SCHED_STATUS: + return is_set_sched_status(); + case OWNER: + return is_set_owner(); } throw new IllegalStateException(); } @@ -564,6 +658,24 @@ public boolean equals(TopologyInfo that) { return false; } + boolean this_present_sched_status = true && this.is_set_sched_status(); + boolean that_present_sched_status = true && that.is_set_sched_status(); + if (this_present_sched_status || that_present_sched_status) { + if (!(this_present_sched_status && that_present_sched_status)) + return false; + if (!this.sched_status.equals(that.sched_status)) + return false; + } + + boolean this_present_owner = true && this.is_set_owner(); + boolean that_present_owner = true && that.is_set_owner(); + if (this_present_owner || that_present_owner) { + if (!(this_present_owner && that_present_owner)) + return false; + if (!this.owner.equals(that.owner)) + return false; + } + return true; } @@ -601,6 +713,16 @@ public int hashCode() { if (present_errors) builder.append(errors); + boolean present_sched_status = true && (is_set_sched_status()); + builder.append(present_sched_status); + if (present_sched_status) + builder.append(sched_status); + + boolean present_owner = true && (is_set_owner()); + builder.append(present_owner); + if (present_owner) + builder.append(owner); + return builder.toHashCode(); } @@ -672,6 +794,26 @@ public int compareTo(TopologyInfo other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(typedOther.is_set_sched_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_sched_status()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, typedOther.sched_status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_owner()).compareTo(typedOther.is_set_owner()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_owner()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -766,6 +908,20 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 513: // SCHED_STATUS + if (field.type == org.apache.thrift.protocol.TType.STRING) { + this.sched_status = iprot.readString(); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + case 514: // OWNER + if (field.type == org.apache.thrift.protocol.TType.STRING) { + this.owner = iprot.readString(); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -829,6 +985,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. } oprot.writeFieldEnd(); } + if (this.sched_status != null) { + if (is_set_sched_status()) { + oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC); + oprot.writeString(this.sched_status); + oprot.writeFieldEnd(); + } + } + if (this.owner != null) { + if (is_set_owner()) { + oprot.writeFieldBegin(OWNER_FIELD_DESC); + oprot.writeString(this.owner); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -881,6 +1051,26 @@ public String toString() { sb.append(this.errors); } first = false; + if (is_set_sched_status()) { + if (!first) sb.append(", "); + sb.append("sched_status:"); + if (this.sched_status == null) { + sb.append("null"); + } else { + sb.append(this.sched_status); + } + first = false; + } + if (is_set_owner()) { + if (!first) sb.append(", "); + sb.append("owner:"); + if (this.owner == null) { + sb.append("null"); + } else { + sb.append(this.owner); + } + first = false; + } sb.append(")"); return sb.toString(); } diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java index 97ae6d7e466..fea21374dec 100644 --- a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java +++ b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java @@ -48,6 +48,8 @@ public class TopologySummary implements org.apache.thrift.TBase byName = new HashMap(); @@ -94,6 +100,10 @@ public static _Fields findByThriftId(int fieldId) { return UPTIME_SECS; case 7: // STATUS return STATUS; + case 513: // SCHED_STATUS + return SCHED_STATUS; + case 514: // OWNER + return OWNER; default: return null; } @@ -157,6 +167,10 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap); } @@ -206,6 +220,12 @@ public TopologySummary(TopologySummary other) { if (other.is_set_status()) { this.status = other.status; } + if (other.is_set_sched_status()) { + this.sched_status = other.sched_status; + } + if (other.is_set_owner()) { + this.owner = other.owner; + } } public TopologySummary deepCopy() { @@ -225,6 +245,8 @@ public void clear() { set_uptime_secs_isSet(false); this.uptime_secs = 0; this.status = null; + this.sched_status = null; + this.owner = null; } public String get_id() { @@ -384,6 +406,52 @@ public void set_status_isSet(boolean value) { } } + public String get_sched_status() { + return this.sched_status; + } + + public void set_sched_status(String sched_status) { + this.sched_status = sched_status; + } + + public void unset_sched_status() { + this.sched_status = null; + } + + /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */ + public boolean is_set_sched_status() { + return this.sched_status != null; + } + + public void set_sched_status_isSet(boolean value) { + if (!value) { + this.sched_status = null; + } + } + + public String get_owner() { + return this.owner; + } + + public void set_owner(String owner) { + this.owner = owner; + } + + public void unset_owner() { + this.owner = null; + } + + /** Returns true if field owner is set (has been assigned a value) and false otherwise */ + public boolean is_set_owner() { + return this.owner != null; + } + + public void set_owner_isSet(boolean value) { + if (!value) { + this.owner = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case ID: @@ -442,6 +510,22 @@ public void setFieldValue(_Fields field, Object value) { } break; + case SCHED_STATUS: + if (value == null) { + unset_sched_status(); + } else { + set_sched_status((String)value); + } + break; + + case OWNER: + if (value == null) { + unset_owner(); + } else { + set_owner((String)value); + } + break; + } } @@ -468,6 +552,12 @@ public Object getFieldValue(_Fields field) { case STATUS: return get_status(); + case SCHED_STATUS: + return get_sched_status(); + + case OWNER: + return get_owner(); + } throw new IllegalStateException(); } @@ -493,6 +583,10 @@ public boolean isSet(_Fields field) { return is_set_uptime_secs(); case STATUS: return is_set_status(); + case SCHED_STATUS: + return is_set_sched_status(); + case OWNER: + return is_set_owner(); } throw new IllegalStateException(); } @@ -573,6 +667,24 @@ public boolean equals(TopologySummary that) { return false; } + boolean this_present_sched_status = true && this.is_set_sched_status(); + boolean that_present_sched_status = true && that.is_set_sched_status(); + if (this_present_sched_status || that_present_sched_status) { + if (!(this_present_sched_status && that_present_sched_status)) + return false; + if (!this.sched_status.equals(that.sched_status)) + return false; + } + + boolean this_present_owner = true && this.is_set_owner(); + boolean that_present_owner = true && that.is_set_owner(); + if (this_present_owner || that_present_owner) { + if (!(this_present_owner && that_present_owner)) + return false; + if (!this.owner.equals(that.owner)) + return false; + } + return true; } @@ -615,6 +727,16 @@ public int hashCode() { if (present_status) builder.append(status); + boolean present_sched_status = true && (is_set_sched_status()); + builder.append(present_sched_status); + if (present_sched_status) + builder.append(sched_status); + + boolean present_owner = true && (is_set_owner()); + builder.append(present_owner); + if (present_owner) + builder.append(owner); + return builder.toHashCode(); } @@ -696,6 +818,26 @@ public int compareTo(TopologySummary other) { return lastComparison; } } + lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(typedOther.is_set_sched_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_sched_status()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, typedOther.sched_status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_owner()).compareTo(typedOther.is_set_owner()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_owner()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -766,6 +908,20 @@ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.t org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } break; + case 513: // SCHED_STATUS + if (field.type == org.apache.thrift.protocol.TType.STRING) { + this.sched_status = iprot.readString(); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; + case 514: // OWNER + if (field.type == org.apache.thrift.protocol.TType.STRING) { + this.owner = iprot.readString(); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); } @@ -806,6 +962,20 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. oprot.writeString(this.status); oprot.writeFieldEnd(); } + if (this.sched_status != null) { + if (is_set_sched_status()) { + oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC); + oprot.writeString(this.sched_status); + oprot.writeFieldEnd(); + } + } + if (this.owner != null) { + if (is_set_owner()) { + oprot.writeFieldBegin(OWNER_FIELD_DESC); + oprot.writeString(this.owner); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -854,6 +1024,26 @@ public String toString() { sb.append(this.status); } first = false; + if (is_set_sched_status()) { + if (!first) sb.append(", "); + sb.append("sched_status:"); + if (this.sched_status == null) { + sb.append("null"); + } else { + sb.append(this.sched_status); + } + first = false; + } + if (is_set_owner()) { + if (!first) sb.append(", "); + sb.append("owner:"); + if (this.owner == null) { + sb.append("null"); + } else { + sb.append(this.owner); + } + first = false; + } sb.append(")"); return sb.toString(); } diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java index 5e0b5af76ea..e0c7cc72945 100644 --- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java +++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java @@ -35,6 +35,10 @@ public class Cluster { * key: topologyId, value: topology's current assignments. */ private Map assignments; + /** + * key topologyId, Value: scheduler's status. + */ + private Map status; /** * a map from hostname to supervisor id. @@ -50,6 +54,7 @@ public Cluster(INimbus nimbus, Map supervisors, Map(assignments.size()); this.assignments.putAll(assignments); + this.status = new HashMap(); this.hostToId = new HashMap>(); for (String nodeId : supervisors.keySet()) { SupervisorDetails supervisor = supervisors.get(nodeId); @@ -432,4 +437,12 @@ public Map getAssignments() { public Map getSupervisors() { return this.supervisors; } + + public void setStatus(String topologyId, String status) { + this.status.put(topologyId, status); + } + + public Map getStatusMap() { + return this.status; + } } diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java new file mode 100644 index 00000000000..3053b5b8a9c --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java @@ -0,0 +1,219 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.scheduler.multitenant; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.scheduler.SchedulerAssignment; +import backtype.storm.scheduler.TopologyDetails; +import backtype.storm.scheduler.WorkerSlot; + +/** + * A pool of machines that anyone can use, but topologies are not isolated + */ +public class DefaultPool extends NodePool { + private static final Logger LOG = LoggerFactory.getLogger(DefaultPool.class); + private Set _nodes = new HashSet(); + private HashMap _tds = new HashMap(); + + @Override + public void addTopology(TopologyDetails td) { + String topId = td.getId(); + LOG.debug("Adding in Topology {}", topId); + _tds.put(topId, td); + SchedulerAssignment assignment = _cluster.getAssignmentById(topId); + if (assignment != null) { + for (WorkerSlot ws: assignment.getSlots()) { + Node n = _nodeIdToNode.get(ws.getNodeId()); + _nodes.add(n); + } + } + } + + @Override + public boolean canAdd(TopologyDetails td) { + return true; + } + + @Override + public Collection takeNodes(int nodesNeeded) { + HashSet ret = new HashSet(); + LinkedList sortedNodes = new LinkedList(_nodes); + Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC); + for (Node n: sortedNodes) { + if (nodesNeeded <= ret.size()) { + break; + } + if (n.isAlive()) { + n.freeAllSlots(_cluster); + _nodes.remove(n); + ret.add(n); + } + } + return ret; + } + + @Override + public int nodesAvailable() { + int total = 0; + for (Node n: _nodes) { + if (n.isAlive()) total++; + } + return total; + } + + @Override + public int slotsAvailable() { + return Node.countTotalSlotsAlive(_nodes); + } + + @Override + public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) { + int nodesFound = 0; + int slotsFound = 0; + LinkedList sortedNodes = new LinkedList(_nodes); + Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC); + for (Node n: sortedNodes) { + if (slotsNeeded <= 0) { + break; + } + if (n.isAlive()) { + nodesFound++; + int totalSlotsFree = n.totalSlots(); + slotsFound += totalSlotsFree; + slotsNeeded -= totalSlotsFree; + } + } + return new NodeAndSlotCounts(nodesFound, slotsFound); + } + + @Override + public Collection takeNodesBySlots(int slotsNeeded) { + HashSet ret = new HashSet(); + LinkedList sortedNodes = new LinkedList(_nodes); + Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC); + for (Node n: sortedNodes) { + if (slotsNeeded <= 0) { + break; + } + if (n.isAlive()) { + n.freeAllSlots(_cluster); + _nodes.remove(n); + ret.add(n); + slotsNeeded -= n.totalSlotsFree(); + } + } + return ret; + } + + @Override + public void scheduleAsNeeded(NodePool... lesserPools) { + for (TopologyDetails td : _tds.values()) { + String topId = td.getId(); + if (_cluster.needsScheduling(td)) { + LOG.debug("Scheduling topology {}",topId); + int totalTasks = td.getExecutors().size(); + int origRequest = td.getNumWorkers(); + int slotsRequested = Math.min(totalTasks, origRequest); + int slotsUsed = Node.countSlotsUsed(topId, _nodes); + int slotsFree = Node.countFreeSlotsAlive(_nodes); + //Check to see if we have enough slots before trying to get them + int slotsAvailable = 0; + if (slotsRequested > slotsFree) { + slotsAvailable = NodePool.slotsAvailable(lesserPools); + } + int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree + slotsAvailable); + int executorsNotRunning = _cluster.getUnassignedExecutors(td).size(); + LOG.debug("Slots... requested {} used {} free {} available {} to be used {}, executors not running {}", + new Object[] {slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse, executorsNotRunning}); + if (slotsToUse <= 0) { + if (executorsNotRunning > 0) { + _cluster.setStatus(topId,"Not fully scheduled (No free slots in default pool) "+executorsNotRunning+" executors not scheduled"); + } else { + if (slotsUsed < slotsRequested) { + _cluster.setStatus(topId,"Running with fewer slots than requested ("+slotsUsed+"/"+origRequest+")"); + } else { //slotsUsed < origRequest + _cluster.setStatus(topId,"Fully Scheduled (requested "+origRequest+" slots, but could only use "+slotsUsed+")"); + } + } + continue; + } + + int slotsNeeded = slotsToUse - slotsFree; + if (slotsNeeded > 0) { + _nodes.addAll(NodePool.takeNodesBySlot(slotsNeeded, lesserPools)); + } + + if (executorsNotRunning <= 0) { + //There are free slots that we can take advantage of now. + for (Node n: _nodes) { + n.freeTopology(topId, _cluster); + } + slotsFree = Node.countFreeSlotsAlive(_nodes); + slotsToUse = Math.min(slotsRequested, slotsFree); + } + + RoundRobinSlotScheduler slotSched = + new RoundRobinSlotScheduler(td, slotsToUse, _cluster); + + LinkedList nodes = new LinkedList(_nodes); + while (true) { + Node n = null; + do { + if (nodes.isEmpty()) { + throw new IllegalStateException("This should not happen, we" + + " messed up and did not get enough slots"); + } + n = nodes.peekFirst(); + if (n.totalSlotsFree() == 0) { + nodes.remove(); + n = null; + } + } while (n == null); + if (!slotSched.assignSlotTo(n)) { + break; + } + } + int afterSchedSlotsUsed = Node.countSlotsUsed(topId, _nodes); + if (afterSchedSlotsUsed < slotsRequested) { + _cluster.setStatus(topId,"Running with fewer slots than requested ("+afterSchedSlotsUsed+"/"+origRequest+")"); + } else if (afterSchedSlotsUsed < origRequest) { + _cluster.setStatus(topId,"Fully Scheduled (requested "+origRequest+" slots, but could only use "+afterSchedSlotsUsed+")"); + } else { + _cluster.setStatus(topId,"Fully Scheduled"); + } + } else { + _cluster.setStatus(topId,"Fully Scheduled"); + } + } + } + + @Override + public String toString() { + return "DefaultPool " + _nodes.size() + " nodes " + _tds.size() + " topologies"; + } +} diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java new file mode 100644 index 00000000000..c625895390b --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.scheduler.multitenant; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.scheduler.Cluster; +import backtype.storm.scheduler.TopologyDetails; + +/** + * All of the machines that currently have nothing assigned to them + */ +public class FreePool extends NodePool { + private static final Logger LOG = LoggerFactory.getLogger(FreePool.class); + private Set _nodes = new HashSet(); + private int _totalSlots = 0; + + @Override + public void init(Cluster cluster, Map nodeIdToNode) { + super.init(cluster, nodeIdToNode); + for (Node n: nodeIdToNode.values()) { + if(n.isTotallyFree() && n.isAlive()) { + _nodes.add(n); + _totalSlots += n.totalSlotsFree(); + } + } + LOG.debug("Found {} nodes with {} slots", _nodes.size(), _totalSlots); + } + + @Override + public void addTopology(TopologyDetails td) { + throw new IllegalArgumentException("The free pool cannot run any topologies"); + } + + @Override + public boolean canAdd(TopologyDetails td) { + // The free pool never has anything running + return false; + } + + @Override + public Collection takeNodes(int nodesNeeded) { + HashSet ret = new HashSet(); + Iterator it = _nodes.iterator(); + while (it.hasNext() && nodesNeeded > ret.size()) { + Node n = it.next(); + ret.add(n); + _totalSlots -= n.totalSlotsFree(); + it.remove(); + } + return ret; + } + + @Override + public int nodesAvailable() { + return _nodes.size(); + } + + @Override + public int slotsAvailable() { + return _totalSlots; + } + + @Override + public Collection takeNodesBySlots(int slotsNeeded) { + HashSet ret = new HashSet(); + Iterator it = _nodes.iterator(); + while (it.hasNext() && slotsNeeded > 0) { + Node n = it.next(); + ret.add(n); + _totalSlots -= n.totalSlotsFree(); + slotsNeeded -= n.totalSlotsFree(); + it.remove(); + } + return ret; + } + + @Override + public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) { + int slotsFound = 0; + int nodesFound = 0; + Iterator it = _nodes.iterator(); + while (it.hasNext() && slotsNeeded > 0) { + Node n = it.next(); + nodesFound++; + int totalSlots = n.totalSlots(); + slotsFound += totalSlots; + slotsNeeded -= totalSlots; + } + return new NodeAndSlotCounts(nodesFound, slotsFound); + } + + @Override + public void scheduleAsNeeded(NodePool... lesserPools) { + //No topologies running so NOOP + } + + @Override + public String toString() { + return "FreePool of "+_nodes.size()+" nodes with "+_totalSlots+" slots"; + } +} diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java new file mode 100644 index 00000000000..dc7eded4fee --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java @@ -0,0 +1,346 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.scheduler.multitenant; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.scheduler.SchedulerAssignment; +import backtype.storm.scheduler.TopologyDetails; +import backtype.storm.scheduler.WorkerSlot; + +/** + * A pool of machines that can be used to run isolated topologies + */ +public class IsolatedPool extends NodePool { + private static final Logger LOG = LoggerFactory.getLogger(IsolatedPool.class); + private Map> _topologyIdToNodes = new HashMap>(); + private HashMap _tds = new HashMap(); + private HashSet _isolated = new HashSet(); + private int _maxNodes; + private int _usedNodes; + + public IsolatedPool(int maxNodes) { + _maxNodes = maxNodes; + _usedNodes = 0; + } + + @Override + public void addTopology(TopologyDetails td) { + String topId = td.getId(); + LOG.debug("Adding in Topology {}", topId); + SchedulerAssignment assignment = _cluster.getAssignmentById(topId); + Set assignedNodes = new HashSet(); + if (assignment != null) { + for (WorkerSlot ws: assignment.getSlots()) { + Node n = _nodeIdToNode.get(ws.getNodeId()); + assignedNodes.add(n); + } + } + _usedNodes += assignedNodes.size(); + _topologyIdToNodes.put(topId, assignedNodes); + _tds.put(topId, td); + if (td.getConf().get(Config.TOPOLOGY_ISOLATED_MACHINES) != null) { + _isolated.add(topId); + } + } + + @Override + public boolean canAdd(TopologyDetails td) { + //Only add topologies that are not sharing nodes with other topologies + String topId = td.getId(); + SchedulerAssignment assignment = _cluster.getAssignmentById(topId); + if (assignment != null) { + for (WorkerSlot ws: assignment.getSlots()) { + Node n = _nodeIdToNode.get(ws.getNodeId()); + if (n.getRunningTopologies().size() > 1) { + return false; + } + } + } + return true; + } + + @Override + public void scheduleAsNeeded(NodePool ... lesserPools) { + for (String topId : _topologyIdToNodes.keySet()) { + TopologyDetails td = _tds.get(topId); + if (_cluster.needsScheduling(td)) { + LOG.debug("Scheduling topology {}",topId); + Set allNodes = _topologyIdToNodes.get(topId); + Number nodesRequested = (Number) td.getConf().get(Config.TOPOLOGY_ISOLATED_MACHINES); + int slotsToUse = 0; + if (nodesRequested == null) { + slotsToUse = getNodesForNotIsolatedTop(td, allNodes, lesserPools); + } else { + slotsToUse = getNodesForIsolatedTop(td, allNodes, lesserPools, + nodesRequested.intValue()); + } + //No slots to schedule for some reason, so skip it. + if (slotsToUse <= 0) { + continue; + } + + RoundRobinSlotScheduler slotSched = + new RoundRobinSlotScheduler(td, slotsToUse, _cluster); + + LinkedList sortedNodes = new LinkedList(allNodes); + Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC); + + LOG.debug("Nodes sorted by free space {}", sortedNodes); + while (true) { + Node n = sortedNodes.remove(); + if (!slotSched.assignSlotTo(n)) { + break; + } + int freeSlots = n.totalSlotsFree(); + for (int i = 0; i < sortedNodes.size(); i++) { + if (freeSlots >= sortedNodes.get(i).totalSlotsFree()) { + sortedNodes.add(i, n); + n = null; + break; + } + } + if (n != null) { + sortedNodes.add(n); + } + } + } + Set found = _topologyIdToNodes.get(topId); + int nc = found == null ? 0 : found.size(); + _cluster.setStatus(topId,"Scheduled Isolated on "+nc+" Nodes"); + } + } + + /** + * Get the nodes needed to schedule an isolated topology. + * @param td the topology to be scheduled + * @param allNodes the nodes already scheduled for this topology. + * This will be updated to include new nodes if needed. + * @param lesserPools node pools we can steal nodes from + * @return the number of additional slots that should be used for scheduling. + */ + private int getNodesForIsolatedTop(TopologyDetails td, Set allNodes, + NodePool[] lesserPools, int nodesRequested) { + String topId = td.getId(); + LOG.debug("Topology {} is isolated", topId); + int nodesFromUsAvailable = nodesAvailable(); + int nodesFromOthersAvailable = NodePool.nodesAvailable(lesserPools); + + int nodesUsed = _topologyIdToNodes.get(topId).size(); + int nodesNeeded = nodesRequested - nodesUsed; + LOG.debug("Nodes... requested {} used {} available from us {} " + + "avail from other {} needed {}", new Object[] {nodesRequested, + nodesUsed, nodesFromUsAvailable, nodesFromOthersAvailable, + nodesNeeded}); + if ((nodesNeeded - nodesFromUsAvailable) > (_maxNodes - _usedNodes)) { + _cluster.setStatus(topId,"Max Nodes("+_maxNodes+") for this user would be exceeded. " + + ((nodesNeeded - nodesFromUsAvailable) - (_maxNodes - _usedNodes)) + + " more nodes needed to run topology."); + return 0; + } + + //In order to avoid going over _maxNodes I may need to steal from + // myself even though other pools have free nodes. so figure out how + // much each group should provide + int nodesNeededFromOthers = Math.min(Math.min(_maxNodes - _usedNodes, + nodesFromOthersAvailable), nodesNeeded); + int nodesNeededFromUs = nodesNeeded - nodesNeededFromOthers; + LOG.debug("Nodes... needed from us {} needed from others {}", + nodesNeededFromUs, nodesNeededFromOthers); + + if (nodesNeededFromUs > nodesFromUsAvailable) { + _cluster.setStatus(topId, "Not Enough Nodes Available to Schedule Topology"); + return 0; + } + + //Get the nodes + Collection found = NodePool.takeNodes(nodesNeededFromOthers, lesserPools); + _usedNodes += found.size(); + allNodes.addAll(found); + Collection foundMore = takeNodes(nodesNeededFromUs); + _usedNodes += foundMore.size(); + allNodes.addAll(foundMore); + + int totalTasks = td.getExecutors().size(); + int origRequest = td.getNumWorkers(); + int slotsRequested = Math.min(totalTasks, origRequest); + int slotsUsed = Node.countSlotsUsed(allNodes); + int slotsFree = Node.countFreeSlotsAlive(allNodes); + int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree); + if (slotsToUse <= 0) { + _cluster.setStatus(topId, "Node has partially crashed, if this situation persists rebalance the topology."); + } + return slotsToUse; + } + + /** + * Get the nodes needed to schedule a non-isolated topology. + * @param td the topology to be scheduled + * @param allNodes the nodes already scheduled for this topology. + * This will be updated to include new nodes if needed. + * @param lesserPools node pools we can steal nodes from + * @return the number of additional slots that should be used for scheduling. + */ + private int getNodesForNotIsolatedTop(TopologyDetails td, Set allNodes, + NodePool[] lesserPools) { + String topId = td.getId(); + LOG.debug("Topology {} is not isolated",topId); + int totalTasks = td.getExecutors().size(); + int origRequest = td.getNumWorkers(); + int slotsRequested = Math.min(totalTasks, origRequest); + int slotsUsed = Node.countSlotsUsed(topId, allNodes); + int slotsFree = Node.countFreeSlotsAlive(allNodes); + //Check to see if we have enough slots before trying to get them + int slotsAvailable = 0; + if (slotsRequested > slotsFree) { + slotsAvailable = NodePool.slotsAvailable(lesserPools); + } + int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree + slotsAvailable); + LOG.debug("Slots... requested {} used {} free {} available {} to be used {}", + new Object[] {slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse}); + if (slotsToUse <= 0) { + _cluster.setStatus(topId, "Not Enough Slots Available to Schedule Topology"); + return 0; + } + int slotsNeeded = slotsToUse - slotsFree; + int numNewNodes = NodePool.getNodeCountIfSlotsWereTaken(slotsNeeded, lesserPools); + LOG.debug("Nodes... new {} used {} max {}", + new Object[]{numNewNodes, _usedNodes, _maxNodes}); + if ((numNewNodes + _usedNodes) > _maxNodes) { + _cluster.setStatus(topId,"Max Nodes("+_maxNodes+") for this user would be exceeded. " + + (numNewNodes - (_maxNodes - _usedNodes)) + " more nodes needed to run topology."); + return 0; + } + + Collection found = NodePool.takeNodesBySlot(slotsNeeded, lesserPools); + _usedNodes += found.size(); + allNodes.addAll(found); + return slotsToUse; + } + + @Override + public Collection takeNodes(int nodesNeeded) { + LOG.debug("Taking {} from {}", nodesNeeded, this); + HashSet ret = new HashSet(); + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + Iterator it = entry.getValue().iterator(); + while (it.hasNext()) { + if (nodesNeeded <= 0) { + return ret; + } + Node n = it.next(); + it.remove(); + n.freeAllSlots(_cluster); + ret.add(n); + nodesNeeded--; + _usedNodes--; + } + } + } + return ret; + } + + @Override + public int nodesAvailable() { + int total = 0; + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + total += entry.getValue().size(); + } + } + return total; + } + + @Override + public int slotsAvailable() { + int total = 0; + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + total += Node.countTotalSlotsAlive(entry.getValue()); + } + } + return total; + } + + @Override + public Collection takeNodesBySlots(int slotsNeeded) { + HashSet ret = new HashSet(); + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + Iterator it = entry.getValue().iterator(); + while (it.hasNext()) { + Node n = it.next(); + if (n.isAlive()) { + it.remove(); + _usedNodes--; + n.freeAllSlots(_cluster); + ret.add(n); + slotsNeeded -= n.totalSlots(); + if (slotsNeeded <= 0) { + return ret; + } + } + } + } + } + return ret; + } + + @Override + public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) { + int nodesFound = 0; + int slotsFound = 0; + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + Iterator it = entry.getValue().iterator(); + while (it.hasNext()) { + Node n = it.next(); + if (n.isAlive()) { + nodesFound++; + int totalSlotsFree = n.totalSlots(); + slotsFound += totalSlotsFree; + slotsNeeded -= totalSlotsFree; + if (slotsNeeded <= 0) { + return new NodeAndSlotCounts(nodesFound, slotsFound); + } + } + } + } + } + return new NodeAndSlotCounts(nodesFound, slotsFound); + } + + @Override + public String toString() { + return "IsolatedPool... "; + } +} diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java new file mode 100644 index 00000000000..320b388930b --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java @@ -0,0 +1,98 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.scheduler.multitenant; + +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.scheduler.Cluster; +import backtype.storm.scheduler.IScheduler; +import backtype.storm.scheduler.Topologies; +import backtype.storm.scheduler.TopologyDetails; +import backtype.storm.utils.Utils; + +public class MultitenantScheduler implements IScheduler { + private static final Logger LOG = LoggerFactory.getLogger(MultitenantScheduler.class); + @SuppressWarnings("rawtypes") + private Map _conf; + + @Override + public void prepare(@SuppressWarnings("rawtypes") Map conf) { + _conf = conf; + } + + private Map getUserConf() { + Map ret = (Map)_conf.get(Config.MULTITENANT_SCHEDULER_USER_POOLS); + if (ret == null) { + ret = new HashMap(); + } else { + ret = new HashMap(ret); + } + + Map fromFile = Utils.findAndReadConfigFile("multitenant-scheduler.yaml", false); + Map tmp = (Map)fromFile.get(Config.MULTITENANT_SCHEDULER_USER_POOLS); + if (tmp != null) { + ret.putAll(tmp); + } + return ret; + } + + + @Override + public void schedule(Topologies topologies, Cluster cluster) { + LOG.debug("Rerunning scheduling..."); + Map nodeIdToNode = Node.getAllNodesFrom(cluster); + + Map userConf = getUserConf(); + + Map userPools = new HashMap(); + for (Map.Entry entry : userConf.entrySet()) { + userPools.put(entry.getKey(), new IsolatedPool(entry.getValue().intValue())); + } + DefaultPool defaultPool = new DefaultPool(); + FreePool freePool = new FreePool(); + + freePool.init(cluster, nodeIdToNode); + for (IsolatedPool pool : userPools.values()) { + pool.init(cluster, nodeIdToNode); + } + defaultPool.init(cluster, nodeIdToNode); + + for (TopologyDetails td: topologies.getTopologies()) { + String user = (String)td.getConf().get(Config.TOPOLOGY_SUBMITTER_USER); + LOG.debug("Found top {} run by user {}",td.getId(), user); + NodePool pool = userPools.get(user); + if (pool == null || !pool.canAdd(td)) { + pool = defaultPool; + } + pool.addTopology(td); + } + + //Now schedule all of the topologies that need to be scheduled + for (IsolatedPool pool : userPools.values()) { + pool.scheduleAsNeeded(freePool, defaultPool); + } + defaultPool.scheduleAsNeeded(freePool); + LOG.debug("Scheduling done..."); + } +} diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java new file mode 100644 index 00000000000..2bc2ceece3d --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java @@ -0,0 +1,322 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.scheduler.multitenant; + +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.Map.Entry; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.scheduler.Cluster; +import backtype.storm.scheduler.ExecutorDetails; +import backtype.storm.scheduler.SchedulerAssignment; +import backtype.storm.scheduler.SupervisorDetails; +import backtype.storm.scheduler.WorkerSlot; + +/** + * Represents a single node in the cluster. + */ +public class Node { + private static final Logger LOG = LoggerFactory.getLogger(Node.class); + private Map> _topIdToUsedSlots = new HashMap>(); + private Set _freeSlots = new HashSet(); + private final String _nodeId; + private boolean _isAlive; + + public Node(String nodeId, Set allPorts, boolean isAlive) { + _nodeId = nodeId; + _isAlive = isAlive; + if (_isAlive && allPorts != null) { + for (int port: allPorts) { + _freeSlots.add(new WorkerSlot(_nodeId, port)); + } + } + } + + public String getId() { + return _nodeId; + } + + public boolean isAlive() { + return _isAlive; + } + + /** + * @return a collection of the topology ids currently running on this node + */ + public Collection getRunningTopologies() { + return _topIdToUsedSlots.keySet(); + } + + public boolean isTotallyFree() { + return _topIdToUsedSlots.isEmpty(); + } + + public int totalSlotsFree() { + return _freeSlots.size(); + } + + public int totalSlotsUsed() { + int total = 0; + for (Set slots: _topIdToUsedSlots.values()) { + total += slots.size(); + } + return total; + } + + public int totalSlots() { + return totalSlotsFree() + totalSlotsUsed(); + } + + public int totalSlotsUsed(String topId) { + int total = 0; + Set slots = _topIdToUsedSlots.get(topId); + if (slots != null) { + total = slots.size(); + } + return total; + } + + private void validateSlot(WorkerSlot ws) { + if (!_nodeId.equals(ws.getNodeId())) { + throw new IllegalArgumentException( + "Trying to add a slot to the wrong node " + ws + + " is not a part of " + _nodeId); + } + } + + private void addOrphanedSlot(WorkerSlot ws) { + if (_isAlive) { + throw new IllegalArgumentException("Orphaned Slots " + + "only are allowed on dead nodes."); + } + validateSlot(ws); + if (_freeSlots.contains(ws)) { + return; + } + for (Set used: _topIdToUsedSlots.values()) { + if (used.contains(ws)) { + return; + } + } + _freeSlots.add(ws); + } + + boolean assignInternal(WorkerSlot ws, String topId, boolean dontThrow) { + validateSlot(ws); + if (!_freeSlots.remove(ws)) { + if (dontThrow) { + return true; + } + throw new IllegalStateException("Assigning a slot that was not free " + ws); + } + Set usedSlots = _topIdToUsedSlots.get(topId); + if (usedSlots == null) { + usedSlots = new HashSet(); + _topIdToUsedSlots.put(topId, usedSlots); + } + usedSlots.add(ws); + return false; + } + + /** + * Free all slots on this node. This will update the Cluster too. + * @param cluster the cluster to be updated + */ + public void freeAllSlots(Cluster cluster) { + if (!_isAlive) { + LOG.warn("Freeing all slots on a dead node {} ",_nodeId); + } + for (Entry> entry : _topIdToUsedSlots.entrySet()) { + cluster.freeSlots(entry.getValue()); + if (_isAlive) { + _freeSlots.addAll(entry.getValue()); + } + } + _topIdToUsedSlots = new HashMap>(); + } + + /** + * Frees a single slot in this node + * @param ws the slot to free + * @param cluster the cluster to update + */ + public void free(WorkerSlot ws, Cluster cluster) { + if (_freeSlots.contains(ws)) return; + for (Entry> entry : _topIdToUsedSlots.entrySet()) { + Set slots = entry.getValue(); + if (slots.remove(ws)) { + cluster.freeSlot(ws); + if (_isAlive) { + _freeSlots.add(ws); + } + return; + } + } + throw new IllegalArgumentException("Tried to free a slot that was not" + + " part of this node " + _nodeId); + } + + /** + * Frees all the slots for a topology. + * @param topId the topology to free slots for + * @param cluster the cluster to update + */ + public void freeTopology(String topId, Cluster cluster) { + Set slots = _topIdToUsedSlots.get(topId); + if (slots == null || slots.isEmpty()) return; + for (WorkerSlot ws : slots) { + cluster.freeSlot(ws); + if (_isAlive) { + _freeSlots.add(ws); + } + } + _topIdToUsedSlots.remove(topId); + } + + /** + * Assign a free slot on the node to the following topology and executors. + * This will update the cluster too. + * @param topId the topology to assign a free slot to. + * @param executors the executors to run in that slot. + * @param cluster the cluster to be updated + */ + public void assign(String topId, Collection executors, + Cluster cluster) { + if (!_isAlive) { + throw new IllegalStateException("Trying to adding to a dead node " + _nodeId); + } + if (_freeSlots.isEmpty()) { + throw new IllegalStateException("Trying to assign to a full node " + _nodeId); + } + if (executors.size() == 0) { + LOG.warn("Trying to assign nothing from " + topId + " to " + _nodeId + " (Ignored)"); + } else { + WorkerSlot slot = _freeSlots.iterator().next(); + cluster.assign(slot, topId, executors); + assignInternal(slot, topId, false); + } + } + + @Override + public boolean equals(Object other) { + if (other instanceof Node) { + return _nodeId.equals(((Node)other)._nodeId); + } + return false; + } + + @Override + public int hashCode() { + return _nodeId.hashCode(); + } + + @Override + public String toString() { + return "Node: " + _nodeId; + } + + public static int countSlotsUsed(String topId, Collection nodes) { + int total = 0; + for (Node n: nodes) { + total += n.totalSlotsUsed(topId); + } + return total; + } + + public static int countSlotsUsed(Collection nodes) { + int total = 0; + for (Node n: nodes) { + total += n.totalSlotsUsed(); + } + return total; + } + + public static int countFreeSlotsAlive(Collection nodes) { + int total = 0; + for (Node n: nodes) { + if (n.isAlive()) { + total += n.totalSlotsFree(); + } + } + return total; + } + + public static int countTotalSlotsAlive(Collection nodes) { + int total = 0; + for (Node n: nodes) { + if (n.isAlive()) { + total += n.totalSlots(); + } + } + return total; + } + + public static Map getAllNodesFrom(Cluster cluster) { + Map nodeIdToNode = new HashMap(); + for (SupervisorDetails sup : cluster.getSupervisors().values()) { + //Node ID and supervisor ID are the same. + String id = sup.getId(); + boolean isAlive = !cluster.isBlackListed(id); + LOG.debug("Found a {} Node {} {}", + new Object[] {isAlive? "living":"dead", id, sup.getAllPorts()}); + nodeIdToNode.put(id, new Node(id, sup.getAllPorts(), isAlive)); + } + + for (Entry entry : cluster.getAssignments().entrySet()) { + String topId = entry.getValue().getTopologyId(); + for (WorkerSlot ws: entry.getValue().getSlots()) { + String id = ws.getNodeId(); + Node node = nodeIdToNode.get(id); + if (node == null) { + LOG.debug("Found an assigned slot on a dead supervisor {}", ws); + node = new Node(id, null, false); + nodeIdToNode.put(id, node); + } + if (!node.isAlive()) { + //The supervisor on the node down so add an orphaned slot to hold the unsupervised worker + node.addOrphanedSlot(ws); + } + if (node.assignInternal(ws, topId, true)) { + LOG.warn("Bad scheduling state, "+ws+" assigned multiple workers, unassigning everything..."); + node.free(ws, cluster); + } + } + } + + return nodeIdToNode; + } + + /** + * Used to sort a list of nodes so the node with the most free slots comes + * first. + */ + public static final Comparator FREE_NODE_COMPARATOR_DEC = new Comparator() { + @Override + public int compare(Node o1, Node o2) { + return o2.totalSlotsFree() - o1.totalSlotsFree(); + } + }; +} diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java new file mode 100644 index 00000000000..21d1577a47c --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java @@ -0,0 +1,296 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.scheduler.multitenant; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.scheduler.Cluster; +import backtype.storm.scheduler.ExecutorDetails; +import backtype.storm.scheduler.SchedulerAssignment; +import backtype.storm.scheduler.TopologyDetails; +import backtype.storm.scheduler.WorkerSlot; + +/** + * A pool of nodes that can be used to run topologies. + */ +public abstract class NodePool { + protected Cluster _cluster; + protected Map _nodeIdToNode; + + public static class NodeAndSlotCounts { + public final int _nodes; + public final int _slots; + + public NodeAndSlotCounts(int nodes, int slots) { + _nodes = nodes; + _slots = slots; + } + } + + /** + * Place executors into slots in a round robin way, taking into account + * component spreading among different hosts. + */ + public static class RoundRobinSlotScheduler { + private Map> _nodeToComps; + private HashMap> _spreadToSchedule; + private LinkedList> _slots; + private Set _lastSlot; + private Cluster _cluster; + private String _topId; + + /** + * Create a new scheduler for a given topology + * @param td the topology to schedule + * @param slotsToUse the number of slots to use for the executors left to + * schedule. + * @param cluster the cluster to schedule this on. + */ + public RoundRobinSlotScheduler(TopologyDetails td, int slotsToUse, + Cluster cluster) { + _topId = td.getId(); + _cluster = cluster; + + Map execToComp = td.getExecutorToComponent(); + SchedulerAssignment assignment = _cluster.getAssignmentById(_topId); + _nodeToComps = new HashMap>(); + + if (assignment != null) { + Map execToSlot = assignment.getExecutorToSlot(); + + for (Entry entry: execToSlot.entrySet()) { + String nodeId = entry.getValue().getNodeId(); + Set comps = _nodeToComps.get(nodeId); + if (comps == null) { + comps = new HashSet(); + _nodeToComps.put(nodeId, comps); + } + comps.add(execToComp.get(entry.getKey())); + } + } + + _spreadToSchedule = new HashMap>(); + List spreadComps = (List)td.getConf().get(Config.TOPOLOGY_SPREAD_COMPONENTS); + if (spreadComps != null) { + for (String comp: spreadComps) { + _spreadToSchedule.put(comp, new ArrayList()); + } + } + + _slots = new LinkedList>(); + for (int i = 0; i < slotsToUse; i++) { + _slots.add(new HashSet()); + } + + int at = 0; + for (Entry> entry: _cluster.getNeedsSchedulingComponentToExecutors(td).entrySet()) { + LOG.debug("Scheduling for {}", entry.getKey()); + if (_spreadToSchedule.containsKey(entry.getKey())) { + LOG.debug("Saving {} for spread...",entry.getKey()); + _spreadToSchedule.get(entry.getKey()).addAll(entry.getValue()); + } else { + for (ExecutorDetails ed: entry.getValue()) { + LOG.debug("Assigning {} {} to slot {}", new Object[]{entry.getKey(), ed, at}); + _slots.get(at).add(ed); + at++; + if (at >= _slots.size()) { + at = 0; + } + } + } + } + _lastSlot = _slots.get(_slots.size() - 1); + } + + /** + * Assign a slot to the given node. + * @param n the node to assign a slot to. + * @return true if there are more slots to assign else false. + */ + public boolean assignSlotTo(Node n) { + if (_slots.isEmpty()) { + return false; + } + Set slot = _slots.pop(); + if (slot == _lastSlot) { + //The last slot fill it up + for (Entry> entry: _spreadToSchedule.entrySet()) { + if (entry.getValue().size() > 0) { + slot.addAll(entry.getValue()); + } + } + } else { + String nodeId = n.getId(); + Set nodeComps = _nodeToComps.get(nodeId); + if (nodeComps == null) { + nodeComps = new HashSet(); + _nodeToComps.put(nodeId, nodeComps); + } + for (Entry> entry: _spreadToSchedule.entrySet()) { + if (entry.getValue().size() > 0) { + String comp = entry.getKey(); + if (!nodeComps.contains(comp)) { + nodeComps.add(comp); + slot.add(entry.getValue().remove(0)); + } + } + } + } + n.assign(_topId, slot, _cluster); + return !_slots.isEmpty(); + } + } + + private static final Logger LOG = LoggerFactory.getLogger(NodePool.class); + /** + * Initialize the pool. + * @param cluster the cluster + * @param nodeIdToNode the mapping of node id to nodes + */ + public void init(Cluster cluster, Map nodeIdToNode) { + _cluster = cluster; + _nodeIdToNode = nodeIdToNode; + } + + /** + * Add a topology to the pool + * @param td the topology to add. + */ + public abstract void addTopology(TopologyDetails td); + + /** + * Check if this topology can be added to this pool + * @param td the topology + * @return true if it can else false + */ + public abstract boolean canAdd(TopologyDetails td); + + /** + * @return the number of nodes that are available to be taken + */ + public abstract int slotsAvailable(); + + /** + * Take nodes from this pool that can fulfill possibly up to the + * slotsNeeded + * @param slotsNeeded the number of slots that are needed. + * @return a Collection of nodes with the removed nodes in it. + * This may be empty, but should not be null. + */ + public abstract Collection takeNodesBySlots(int slotsNeeded); + + /** + * Get the number of nodes and slots this would provide to get the slots needed + * @param slots the number of slots needed + * @return the number of nodes and slots that would be returned. + */ + public abstract NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slots); + + /** + * @return the number of nodes that are available to be taken + */ + public abstract int nodesAvailable(); + + /** + * Take up to nodesNeeded from this pool + * @param nodesNeeded the number of nodes that are needed. + * @return a Collection of nodes with the removed nodes in it. + * This may be empty, but should not be null. + */ + public abstract Collection takeNodes(int nodesNeeded); + + /** + * Reschedule any topologies as needed. + * @param lesserPools pools that may be used to steal nodes from. + */ + public abstract void scheduleAsNeeded(NodePool ... lesserPools); + + public static int slotsAvailable(NodePool[] pools) { + int slotsAvailable = 0; + for (NodePool pool: pools) { + slotsAvailable += pool.slotsAvailable(); + } + return slotsAvailable; + } + + public static int nodesAvailable(NodePool[] pools) { + int nodesAvailable = 0; + for (NodePool pool: pools) { + nodesAvailable += pool.nodesAvailable(); + } + return nodesAvailable; + } + + public static Collection takeNodesBySlot(int slotsNeeded,NodePool[] pools) { + LOG.debug("Trying to grab {} free slots from {}",slotsNeeded, pools); + HashSet ret = new HashSet(); + for (NodePool pool: pools) { + Collection got = pool.takeNodesBySlots(slotsNeeded); + ret.addAll(got); + slotsNeeded -= Node.countFreeSlotsAlive(got); + LOG.debug("Got {} nodes so far need {} more slots",ret.size(),slotsNeeded); + if (slotsNeeded <= 0) { + break; + } + } + return ret; + } + + public static Collection takeNodes(int nodesNeeded,NodePool[] pools) { + LOG.debug("Trying to grab {} free nodes from {}",nodesNeeded, pools); + HashSet ret = new HashSet(); + for (NodePool pool: pools) { + Collection got = pool.takeNodes(nodesNeeded); + ret.addAll(got); + nodesNeeded -= got.size(); + LOG.debug("Got {} nodes so far need {} more nodes", ret.size(), nodesNeeded); + if (nodesNeeded <= 0) { + break; + } + } + return ret; + } + + public static int getNodeCountIfSlotsWereTaken(int slots,NodePool[] pools) { + LOG.debug("How many nodes to get {} slots from {}",slots, pools); + int total = 0; + for (NodePool pool: pools) { + NodeAndSlotCounts ns = pool.getNodeAndSlotCountIfSlotsWereTaken(slots); + total += ns._nodes; + slots -= ns._slots; + LOG.debug("Found {} nodes so far {} more slots needed", total, slots); + if (slots <= 0) { + break; + } + } + return total; + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java index b0e48e53822..b5118c7f6f8 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java @@ -20,6 +20,7 @@ import backtype.storm.Config; import javax.security.auth.login.Configuration; import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.Subject; import java.security.NoSuchAlgorithmException; import java.security.URIParameter; import org.slf4j.Logger; @@ -28,7 +29,11 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; +import java.util.Collection; +import java.util.Set; +import java.util.HashSet; import java.util.Map; +import java.util.concurrent.ExecutorService; public class AuthUtils { private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class); @@ -47,39 +52,179 @@ public static Configuration GetConfiguration(Map storm_conf) { //find login file configuration from Storm configuration String loginConfigurationFile = (String)storm_conf.get("java.security.auth.login.config"); if ((loginConfigurationFile != null) && (loginConfigurationFile.length()>0)) { + File config_file = new File(loginConfigurationFile); + if (! config_file.canRead()) { + throw new RuntimeException("File " + loginConfigurationFile + + " cannot be read."); + } try { - URI config_uri = new File(loginConfigurationFile).toURI(); + URI config_uri = config_file.toURI(); login_conf = Configuration.getInstance("JavaLoginConfig", new URIParameter(config_uri)); - } catch (NoSuchAlgorithmException ex1) { - if (ex1.getCause() instanceof FileNotFoundException) - throw new RuntimeException("configuration file "+loginConfigurationFile+" could not be found"); - else throw new RuntimeException(ex1); - } catch (Exception ex2) { - throw new RuntimeException(ex2); + } catch (Exception ex) { + throw new RuntimeException(ex); } } return login_conf; } + /** + * Construct a principal to local plugin + * @param conf storm configuration + * @return the plugin + */ + public static IPrincipalToLocal GetPrincipalToLocalPlugin(Map storm_conf) { + IPrincipalToLocal ptol = null; + try { + String ptol_klassName = (String) storm_conf.get(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN); + Class klass = Class.forName(ptol_klassName); + ptol = (IPrincipalToLocal)klass.newInstance(); + ptol.prepare(storm_conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + return ptol; + } + + /** + * Get all of the configured Credential Renwer Plugins. + * @param storm_conf the storm configuration to use. + * @return the configured credential renewers. + */ + public static Collection GetCredentialRenewers(Map conf) { + try { + Set ret = new HashSet(); + Collection clazzes = (Collection)conf.get(Config.NIMBUS_CREDENTIAL_RENEWERS); + if (clazzes != null) { + for (String clazz : clazzes) { + ICredentialsRenewer inst = (ICredentialsRenewer)Class.forName(clazz).newInstance(); + inst.prepare(conf); + ret.add(inst); + } + } + return ret; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Get all of the configured AutoCredential Plugins. + * @param storm_conf the storm configuration to use. + * @return the configured auto credentials. + */ + public static Collection GetAutoCredentials(Map storm_conf) { + try { + Set autos = new HashSet(); + Collection clazzes = (Collection)storm_conf.get(Config.TOPOLOGY_AUTO_CREDENTIALS); + if (clazzes != null) { + for (String clazz : clazzes) { + IAutoCredentials a = (IAutoCredentials)Class.forName(clazz).newInstance(); + a.prepare(storm_conf); + autos.add(a); + } + } + LOG.info("Got AutoCreds "+autos); + return autos; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Populate a subject from credentials using the IAutoCredentials. + * @param subject the subject to populate or null if a new Subject should be created. + * @param autos the IAutoCredentials to call to populate the subject. + * @param credentials the credentials to pull from + * @return the populated subject. + */ + public static Subject populateSubject(Subject subject, Collection autos, Map credentials) { + try { + if (subject == null) { + subject = new Subject(); + } + for (IAutoCredentials autoCred : autos) { + autoCred.populateSubject(subject, credentials); + } + return subject; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Update a subject from credentials using the IAutoCredentials. + * @param subject the subject to update + * @param autos the IAutoCredentials to call to update the subject. + * @param credentials the credentials to pull from + */ + public static void updateSubject(Subject subject, Collection autos, Map credentials) { + if (subject == null) { + throw new RuntimeException("The subject cannot be null when updating a subject with credentials"); + } + + try { + for (IAutoCredentials autoCred : autos) { + autoCred.updateSubject(subject, credentials); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + /** * Construct a transport plugin per storm configuration * @param conf storm configuration * @return */ - public static ITransportPlugin GetTransportPlugin(Map storm_conf, Configuration login_conf) { + public static ITransportPlugin GetTransportPlugin(ThriftConnectionType type, Map storm_conf, Configuration login_conf) { ITransportPlugin transportPlugin = null; try { - String transport_plugin_klassName = (String) storm_conf.get(Config.STORM_THRIFT_TRANSPORT_PLUGIN); + String transport_plugin_klassName = type.getTransportPlugin(storm_conf); Class klass = Class.forName(transport_plugin_klassName); transportPlugin = (ITransportPlugin)klass.newInstance(); - transportPlugin.prepare(storm_conf, login_conf); + transportPlugin.prepare(type, storm_conf, login_conf); } catch(Exception e) { throw new RuntimeException(e); } return transportPlugin; } + private static IHttpCredentialsPlugin GetHttpCredentialsPlugin(Map conf, + String klassName) { + IHttpCredentialsPlugin plugin = null; + try { + Class klass = Class.forName(klassName); + plugin = (IHttpCredentialsPlugin)klass.newInstance(); + plugin.prepare(conf); + } catch(Exception e) { + throw new RuntimeException(e); + } + return plugin; + } + + /** + * Construct an HttpServletRequest credential plugin specified by the UI + * storm configuration + * @param conf storm configuration + * @return the plugin + */ + public static IHttpCredentialsPlugin GetUiHttpCredentialsPlugin(Map conf) { + String klassName = (String)conf.get(Config.UI_HTTP_CREDS_PLUGIN); + return AuthUtils.GetHttpCredentialsPlugin(conf, klassName); + } + + /** + * Construct an HttpServletRequest credential plugin specified by the DRPC + * storm configuration + * @param conf storm configuration + * @return the plugin + */ + public static IHttpCredentialsPlugin GetDrpcHttpCredentialsPlugin(Map conf) { + String klassName = (String)conf.get(Config.DRPC_HTTP_CREDS_PLUGIN); + return AuthUtils.GetHttpCredentialsPlugin(conf, klassName); + } + public static String get(Configuration configuration, String section, String key) throws IOException { AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(section); if (configurationEntries == null) { diff --git a/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java new file mode 100644 index 00000000000..7d8e0fb5d14 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.security.Principal; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.security.auth.Subject; +import javax.servlet.http.HttpServletRequest; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.ReqContext; + +public class DefaultHttpCredentialsPlugin implements IHttpCredentialsPlugin { + private static final Logger LOG = + LoggerFactory.getLogger(DefaultHttpCredentialsPlugin.class); + + /** + * No-op + * @param storm_conf Storm configuration + */ + @Override + public void prepare(Map storm_conf) { + // Do nothing. + } + + /** + * Gets the user name from the request principal. + * @param req the servlet request + * @return the authenticated user, or null if none is authenticated + */ + @Override + public String getUserName(HttpServletRequest req) { + Principal princ = null; + if (req != null && (princ = req.getUserPrincipal()) != null) { + String userName = princ.getName(); + if (userName != null && !userName.isEmpty()) { + LOG.debug("HTTP request had user ("+userName+")"); + return userName; + } + } + return null; + } + + /** + * Populates a given context with a new Subject derived from the + * credentials in a servlet request. + * @param context the context to be populated + * @param req the servlet request + * @return the context + */ + @Override + public ReqContext populateContext(ReqContext context, + HttpServletRequest req) { + String userName = getUserName(req); + if (userName != null) { + Set principals = new HashSet(1); + principals.add(new SingleUserPrincipal(userName)); + Subject s = new Subject(true, principals, new HashSet(), new HashSet()); + context.setSubject(s); + } + return context; + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java new file mode 100644 index 00000000000..729d744d410 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.util.Map; +import java.security.Principal; + +/** + * Storm can be configured to launch worker processed as a given user. + * Some transports need to map the Principal to a local user name. + */ +public class DefaultPrincipalToLocal implements IPrincipalToLocal { + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + public void prepare(Map storm_conf) {} + + /** + * Convert a Principal to a local user name. + * @param principal the principal to convert + * @return The local user name. + */ + public String toLocal(Principal principal) { + return principal == null ? null : principal.getName(); + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IAutoCredentials.java b/storm-core/src/jvm/backtype/storm/security/auth/IAutoCredentials.java new file mode 100644 index 00000000000..b3886da8cbf --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/IAutoCredentials.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.util.Map; + +import javax.security.auth.Subject; + +/** + * Provides a way to automatically push credentials to a topology and to + * retreave them in the worker. + */ +public interface IAutoCredentials { + + public void prepare(Map conf); + + /** + * Called to populate the credentials on the client side. + * @param credentials the credentials to be populated. + */ + public void populateCredentials(Map credentials); + + /** + * Called to initially populate the subject on the worker side with credentials passed in. + * @param subject the subject to optionally put credentials in. + * @param credentials the credentials to be used. + */ + public void populateSubject(Subject subject, Map credentials); + + + /** + * Called to update the subject on the worker side when new credentials are recieved. + * This means that populateSubject has already been called on this subject. + * @param subject the subject to optionally put credentials in. + * @param credentials the credentials to be used. + */ + public void updateSubject(Subject subject, Map credentials); + +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java b/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java new file mode 100644 index 00000000000..a84a9e853c9 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.util.Collection; +import java.util.Map; + +/** + * Provides a way to renew credentials on behelf of a user. + */ +public interface ICredentialsRenewer { + + /** + * Called when initializing the service. + * @param conf the storm cluster configuration. + */ + public void prepare(Map conf); + + /** + * Renew any credentials that need to be renewed. (Update the credentials if needed) + * @param credentials the credentials that may have something to renew. + */ + public void renew(Map credentials); +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java new file mode 100644 index 00000000000..a012ce41844 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.util.Map; +import javax.servlet.http.HttpServletRequest; + +import backtype.storm.security.auth.ReqContext; + +/** + * Interface for handling credentials in an HttpServletRequest + */ +public interface IHttpCredentialsPlugin { + /** + * Invoked once immediately after construction + * @param storm_conf Storm configuration + */ + void prepare(Map storm_conf); + + /** + * Gets the user name from the request. + * @param req the servlet request + * @return the authenticated user, or null if none is authenticated. + */ + String getUserName(HttpServletRequest req); + + /** + * Populates a given context with credentials information from an HTTP + * request. + * @param req the servlet request + * @return the context + */ + ReqContext populateContext(ReqContext context, HttpServletRequest req); +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java new file mode 100644 index 00000000000..fca3d37e6c2 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.util.Map; +import java.security.Principal; + +/** + * Storm can be configured to launch worker processed as a given user. + * Some transports need to map the Principal to a local user name. + */ +public interface IPrincipalToLocal { + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + void prepare(Map storm_conf); + + /** + * Convert a Principal to a local user name. + * @param principal the principal to convert + * @return The local user name. + */ + public String toLocal(Principal principal); +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java index a44d1156242..7575d71cbde 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java @@ -18,7 +18,9 @@ package backtype.storm.security.auth; import java.io.IOException; +import java.security.Principal; import java.util.Map; +import java.util.concurrent.ExecutorService; import javax.security.auth.login.Configuration; @@ -27,24 +29,26 @@ import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; +import backtype.storm.security.auth.ThriftConnectionType; + /** * Interface for Thrift Transport plugin */ public interface ITransportPlugin { /** * Invoked once immediately after construction + * @param type the type of connection this will process. * @param storm_conf Storm configuration * @param login_conf login configuration */ - void prepare(Map storm_conf, Configuration login_conf); + void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf); /** - * Create a server associated with a given port and service handler - * @param port listening port + * Create a server associated with a given port, service handler, and purpose * @param processor service handler - * @return server to be binded + * @return server */ - public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException; + public TServer getServer(TProcessor processor) throws IOException, TTransportException; /** * Connect to the specified server via framed transport diff --git a/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java new file mode 100644 index 00000000000..35c778869a7 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.util.Map; +import java.security.Principal; + +/** + * Map a kerberos principal to a local user + */ +public class KerberosPrincipalToLocal implements IPrincipalToLocal { + + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + public void prepare(Map storm_conf) {} + + /** + * Convert a Principal to a local user name. + * @param principal the principal to convert + * @return The local user name. + */ + public String toLocal(Principal principal) { + //This technically does not conform with rfc1964, but should work so + // long as you don't have any really odd names in your KDC. + return principal == null ? null : principal.getName().split("[/@]")[0]; + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java b/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java index 68d8493e118..4033f1870b5 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java @@ -59,11 +59,19 @@ protected ReqContext initialValue() { //private constructor @VisibleForTesting - ReqContext(AccessControlContext acl_ctxt) { + public ReqContext(AccessControlContext acl_ctxt) { _subject = Subject.getSubject(acl_ctxt); _reqID = uniqueId.incrementAndGet(); } + //private constructor + @VisibleForTesting + public ReqContext(Subject sub) { + _subject = sub; + _reqID = uniqueId.incrementAndGet(); + } + + /** * client address */ diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java index 44ab754792f..348fae3025b 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java @@ -21,6 +21,9 @@ import java.net.Socket; import java.security.Principal; import java.util.Map; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.TimeUnit; import javax.security.auth.Subject; import javax.security.auth.login.Configuration; @@ -40,38 +43,48 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import backtype.storm.security.auth.ThriftConnectionType; + /** * Base class for SASL authentication plugin. */ public abstract class SaslTransportPlugin implements ITransportPlugin { + protected ThriftConnectionType type; + protected Map storm_conf; protected Configuration login_conf; private static final Logger LOG = LoggerFactory.getLogger(SaslTransportPlugin.class); - /** - * Invoked once immediately after construction - * @param conf Storm configuration - * @param login_conf login configuration - */ - public void prepare(Map storm_conf, Configuration login_conf) { + @Override + public void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf) { + this.type = type; + this.storm_conf = storm_conf; this.login_conf = login_conf; } - public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException { + @Override + public TServer getServer(TProcessor processor) throws IOException, TTransportException { + int port = type.getPort(storm_conf); TTransportFactory serverTransportFactory = getServerTransportFactory(); - - //define THsHaServer args - //original: THsHaServer + TNonblockingServerSocket - //option: TThreadPoolServer + TServerSocket TServerSocket serverTransport = new TServerSocket(port); + int numWorkerThreads = type.getNumThreads(storm_conf); + int maxBufferSize = type.getMaxBufferSize(storm_conf); + Integer queueSize = type.getQueueSize(storm_conf); + TThreadPoolServer.Args server_args = new TThreadPoolServer.Args(serverTransport). processor(new TUGIWrapProcessor(processor)). - minWorkerThreads(64). - maxWorkerThreads(64). - protocolFactory(new TBinaryProtocol.Factory()); - if (serverTransportFactory != null) + minWorkerThreads(numWorkerThreads). + maxWorkerThreads(numWorkerThreads). + protocolFactory(new TBinaryProtocol.Factory(false, true, maxBufferSize)); + + if (serverTransportFactory != null) { server_args.transportFactory(serverTransportFactory); + } + + //if (queueSize != null) { + // server_args.executorService(new ThreadPoolExecutor(numWorkerThreads, numWorkerThreads, + // 60, TimeUnit.SECONDS, new ArrayBlockingQueue(queueSize))); + //} - //construct THsHaServer return new TThreadPoolServer(server_args); } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java index 5a817466e45..f0905e2bb6e 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java @@ -18,12 +18,18 @@ package backtype.storm.security.auth; import java.io.IOException; +import java.security.Principal; import java.net.InetAddress; import java.net.Socket; import java.net.UnknownHostException; import java.util.Map; +import java.util.HashSet; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.TimeUnit; import javax.security.auth.login.Configuration; +import javax.security.auth.Subject; import org.apache.thrift.TException; import org.apache.thrift.TProcessor; import org.apache.thrift.protocol.TBinaryProtocol; @@ -39,33 +45,43 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import backtype.storm.security.auth.ThriftConnectionType; + /** * Simple transport for Thrift plugin. * * This plugin is designed to be backward compatible with existing Storm code. */ public class SimpleTransportPlugin implements ITransportPlugin { + protected ThriftConnectionType type; + protected Map storm_conf; protected Configuration login_conf; private static final Logger LOG = LoggerFactory.getLogger(SimpleTransportPlugin.class); - /** - * Invoked once immediately after construction - * @param conf Storm configuration - * @param login_conf login configuration - */ - public void prepare(Map storm_conf, Configuration login_conf) { + @Override + public void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf) { + this.type = type; + this.storm_conf = storm_conf; this.login_conf = login_conf; } - /** - * We will let Thrift to apply default transport factory - */ - public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException { + @Override + public TServer getServer(TProcessor processor) throws IOException, TTransportException { + int port = type.getPort(storm_conf); TNonblockingServerSocket serverTransport = new TNonblockingServerSocket(port); + int numWorkerThreads = type.getNumThreads(storm_conf); + int maxBufferSize = type.getMaxBufferSize(storm_conf); + Integer queueSize = type.getQueueSize(storm_conf); + THsHaServer.Args server_args = new THsHaServer.Args(serverTransport). processor(new SimpleWrapProcessor(processor)). - workerThreads(64). - protocolFactory(new TBinaryProtocol.Factory()); + workerThreads(numWorkerThreads). + protocolFactory(new TBinaryProtocol.Factory(false, true, maxBufferSize)); + + if (queueSize != null) { + server_args.executorService(new ThreadPoolExecutor(numWorkerThreads, numWorkerThreads, + 60, TimeUnit.SECONDS, new ArrayBlockingQueue(queueSize))); + } //construct THsHaServer return new THsHaServer(server_args); @@ -86,6 +102,13 @@ public TTransport connect(TTransport transport, String serverHost) throws TTrans return conn; } + /** + * @return the subject that will be used for all connections + */ + protected Subject getDefaultSubject() { + return null; + } + /** * Processor that populate simple transport info into ReqContext, and then invoke a service handler */ @@ -115,7 +138,19 @@ public boolean process(final TProtocol inProt, final TProtocol outProt) throws T } //anonymous user - req_context.setSubject(null); + Subject s = getDefaultSubject(); + if (s == null) { + final String user = (String)storm_conf.get("debug.simple.transport.user"); + if (user != null) { + HashSet principals = new HashSet(); + principals.add(new Principal() { + public String getName() { return user; } + public String toString() { return user; } + }); + s = new Subject(true, principals, new HashSet(), new HashSet()); + } + } + req_context.setSubject(s); //invoke service handler return wrapped.process(inProt, outProt); diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java b/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java new file mode 100644 index 00000000000..6af17faf67b --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.security.Principal; + +/** + * A Principal that represents a user. + */ +public class SingleUserPrincipal implements Principal { + + private final String _userName; + + public SingleUserPrincipal(String userName) { + _userName = userName; + } + + @Override + public boolean equals(Object another) { + if (another instanceof SingleUserPrincipal) { + return _userName.equals(((SingleUserPrincipal)another)._userName); + } + return false; + } + + @Override + public String getName() { + return _userName; + } + + @Override + public String toString() { + return this.getName(); + } + + @Override + public int hashCode() { + return _userName.hashCode(); + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java b/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java new file mode 100644 index 00000000000..b50ae6e3999 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.io.IOException; +import java.util.Random; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import backtype.storm.utils.Utils.BoundedExponentialBackoffRetry; + +public class TBackoffConnect { + private static final Logger LOG = LoggerFactory.getLogger(TBackoffConnect.class); + private int _completedRetries = 0; + private int _retryTimes; + private BoundedExponentialBackoffRetry waitGrabber; + + public TBackoffConnect(int retryTimes, int retryInterval, int retryIntervalCeiling) { + + _retryTimes = retryTimes; + waitGrabber = new BoundedExponentialBackoffRetry(retryInterval, + retryTimes, + retryIntervalCeiling); + } + + public TTransport doConnectWithRetry(ITransportPlugin transportPlugin, TTransport underlyingTransport, String host) throws IOException { + boolean connected = false; + TTransport transportResult = null; + while(!connected) { + try { + transportResult = transportPlugin.connect(underlyingTransport, host); + connected = true; + } catch (TTransportException ex) { + retryNext(ex); + } + } + return transportResult; + } + + private void retryNext(TTransportException ex) { + if(!canRetry()) { + throw new RuntimeException(ex); + } + try { + int sleeptime = waitGrabber.getSleepTimeMs(_completedRetries, 0); + + LOG.debug("Failed to connect. Retrying... (" + Integer.toString( _completedRetries) + ") in " + Integer.toString(sleeptime) + "ms"); + + Thread.sleep(sleeptime); + } catch (InterruptedException e) { + LOG.info("Nimbus connection retry interrupted."); + } + + _completedRetries++; + } + + private boolean canRetry() { + return (_completedRetries < _retryTimes); + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java b/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java index d3971270e7a..ec337d99062 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java @@ -24,56 +24,91 @@ import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; -import org.apache.thrift.transport.TTransportException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import backtype.storm.utils.Utils; +import backtype.storm.Config; +import backtype.storm.security.auth.TBackoffConnect; public class ThriftClient { private static final Logger LOG = LoggerFactory.getLogger(ThriftClient.class); private TTransport _transport; protected TProtocol _protocol; + private String _host; + private Integer _port; + private Integer _timeout; + private Map _conf; + private ThriftConnectionType _type; - public ThriftClient(Map storm_conf, String host, int port) throws TTransportException { - this(storm_conf, host, port, null); + public ThriftClient(Map storm_conf, ThriftConnectionType type, String host) { + this(storm_conf, type, host, null, null); } - public ThriftClient(Map storm_conf, String host, int port, Integer timeout) throws TTransportException { + public ThriftClient(Map storm_conf, ThriftConnectionType type, String host, Integer port, Integer timeout) { + //create a socket with server + if (host==null) { + throw new IllegalArgumentException("host is not set"); + } + + if (port == null) { + port = type.getPort(storm_conf); + } + + if (port<=0) { + throw new IllegalArgumentException("invalid port: "+port); + } + + _host = host; + _port = port; + _timeout = timeout; + _conf = storm_conf; + _type = type; + reconnect(); + } + + public synchronized TTransport transport() { + return _transport; + } + + public synchronized void reconnect() { + close(); try { + TSocket socket = new TSocket(_host, _port); + if(_timeout!=null) { + socket.setTimeout(_timeout); + } + //locate login configuration - Configuration login_conf = AuthUtils.GetConfiguration(storm_conf); + Configuration login_conf = AuthUtils.GetConfiguration(_conf); //construct a transport plugin - ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(storm_conf, login_conf); + ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(_type, _conf, login_conf); - //create a socket with server - if(host==null) { - throw new IllegalArgumentException("host is not set"); - } - if(port<=0) { - throw new IllegalArgumentException("invalid port: "+port); - } - TSocket socket = new TSocket(host, port); - if(timeout!=null) { - socket.setTimeout(timeout); - } final TTransport underlyingTransport = socket; + //TODO get this from type instead of hardcoding to Nimbus. //establish client-server transport via plugin - _transport = transportPlugin.connect(underlyingTransport, host); + //do retries if the connect fails + TBackoffConnect connectionRetry + = new TBackoffConnect( + Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_TIMES)), + Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL)), + Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL_CEILING))); + _transport = connectionRetry.doConnectWithRetry(transportPlugin, underlyingTransport, _host); } catch (IOException ex) { throw new RuntimeException(ex); } _protocol = null; - if (_transport != null) + if (_transport != null) { _protocol = new TBinaryProtocol(_transport); + } } - public TTransport transport() { - return _transport; - } - - public void close() { - _transport.close(); + public synchronized void close() { + if (_transport != null) { + _transport.close(); + _transport = null; + _protocol = null; + } } } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ThriftConnectionType.java b/storm-core/src/jvm/backtype/storm/security/auth/ThriftConnectionType.java new file mode 100644 index 00000000000..f9be7ae5f81 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/ThriftConnectionType.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.security.auth; + +import backtype.storm.utils.Utils; +import backtype.storm.Config; + +import java.util.Map; + +/** + * The purpose for which the Thrift server is created. + */ +public enum ThriftConnectionType { + NIMBUS(Config.NIMBUS_THRIFT_TRANSPORT_PLUGIN, Config.NIMBUS_THRIFT_PORT, null, + Config.NIMBUS_THRIFT_THREADS, Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE), + DRPC(Config.DRPC_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_PORT, Config.DRPC_QUEUE_SIZE, + Config.DRPC_WORKER_THREADS, Config.DRPC_MAX_BUFFER_SIZE), + DRPC_INVOCATIONS(Config.DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_INVOCATIONS_PORT, null, + Config.DRPC_INVOCATIONS_THREADS, Config.DRPC_MAX_BUFFER_SIZE); + + private final String _transConf; + private final String _portConf; + private final String _qConf; + private final String _threadsConf; + private final String _buffConf; + + ThriftConnectionType(String transConf, String portConf, String qConf, + String threadsConf, String buffConf) { + _transConf = transConf; + _portConf = portConf; + _qConf = qConf; + _threadsConf = threadsConf; + _buffConf = buffConf; + } + + public String getTransportPlugin(Map conf) { + String ret = (String)conf.get(_transConf); + if (ret == null) { + ret = (String)conf.get(Config.STORM_THRIFT_TRANSPORT_PLUGIN); + } + return ret; + } + + public int getPort(Map conf) { + return Utils.getInt(conf.get(_portConf)); + } + + public Integer getQueueSize(Map conf) { + if (_qConf == null) { + return null; + } + return (Integer)conf.get(_qConf); + } + + public int getNumThreads(Map conf) { + return Utils.getInt(conf.get(_threadsConf)); + } + + public int getMaxBufferSize(Map conf) { + return Utils.getInt(conf.get(_buffConf)); + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java b/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java index ec6e4ad8564..64243cee604 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java @@ -18,27 +18,28 @@ package backtype.storm.security.auth; import java.util.Map; + import javax.security.auth.login.Configuration; + import org.apache.thrift.TProcessor; import org.apache.thrift.server.TServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import backtype.storm.utils.Utils; public class ThriftServer { private static final Logger LOG = LoggerFactory.getLogger(ThriftServer.class); private Map _storm_conf; //storm configuration protected TProcessor _processor = null; - private int _port = 0; + private final ThriftConnectionType _type; private TServer _server = null; private Configuration _login_conf; - public ThriftServer(Map storm_conf, TProcessor processor, int port) { + public ThriftServer(Map storm_conf, TProcessor processor, ThriftConnectionType type) { + _storm_conf = storm_conf; + _processor = processor; + _type = type; + try { - _storm_conf = storm_conf; - _processor = processor; - _port = port; - //retrieve authentication configuration _login_conf = AuthUtils.GetConfiguration(_storm_conf); } catch (Exception x) { @@ -63,10 +64,10 @@ public boolean isServing() { public void serve() { try { //locate our thrift transport plugin - ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(_storm_conf, _login_conf); + ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(_type, _storm_conf, _login_conf); //server - _server = transportPlugin.getServer(_port, _processor); + _server = transportPlugin.getServer(_processor); //start accepting requests _server.serve(); diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java new file mode 100644 index 00000000000..8951edd6806 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java @@ -0,0 +1,46 @@ +package backtype.storm.security.auth.authorizer; + +import java.util.Map; + +import backtype.storm.security.auth.IAuthorizer; +import backtype.storm.security.auth.ReqContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class DRPCAuthorizerBase implements IAuthorizer { + public static Logger LOG = LoggerFactory.getLogger(DRPCAuthorizerBase.class); + + /** + * A key name for the function requested to be executed by a user. + */ + public static final String FUNCTION_NAME = "function.name"; + + @Override + public abstract void prepare(Map conf); + + abstract protected boolean permitClientRequest(ReqContext context, String operation, Map params); + + abstract protected boolean permitInvocationRequest(ReqContext context, String operation, Map params); + + /** + * Authorizes request from to the DRPC server. + * @param context the client request context + * @param operation the operation requested by the DRPC server + * @param params a Map with any key-value entries of use to the authorization implementation + */ + @Override + public boolean permit(ReqContext context, String operation, Map params) { + if ("execute".equals(operation)) { + return permitClientRequest(context, operation, params); + } else if ("failRequest".equals(operation) || + "fetchRequest".equals(operation) || + "result".equals(operation)) { + return permitInvocationRequest(context, operation, params); + } + // Deny unsupported operations. + LOG.warn("Denying unsupported operation \""+operation+"\" from "+ + context.remoteAddress()); + return false; + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java new file mode 100644 index 00000000000..45eaea5477b --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java @@ -0,0 +1,157 @@ +package backtype.storm.security.auth.authorizer; + +import java.lang.reflect.Field; +import java.security.Principal; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import backtype.storm.Config; +import backtype.storm.security.auth.ReqContext; +import backtype.storm.security.auth.authorizer.DRPCAuthorizerBase; +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.security.auth.IPrincipalToLocal; +import backtype.storm.utils.Utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DRPCSimpleACLAuthorizer extends DRPCAuthorizerBase { + public static Logger LOG = + LoggerFactory.getLogger(DRPCSimpleACLAuthorizer.class); + + public static final String CLIENT_USERS_KEY = "client.users"; + public static final String INVOCATION_USER_KEY = "invocation.user"; + public static final String FUNCTION_KEY = "function.name"; + + protected String _aclFileName = ""; + protected IPrincipalToLocal _ptol; + protected boolean _permitWhenMissingFunctionEntry = false; + + protected class AclFunctionEntry { + final public Set clientUsers; + final public String invocationUser; + public AclFunctionEntry(Collection clientUsers, + String invocationUser) { + this.clientUsers = (clientUsers != null) ? + new HashSet(clientUsers) : new HashSet(); + this.invocationUser = invocationUser; + } + } + + private volatile Map _acl = null; + private volatile long _lastUpdate = 0; + + protected Map readAclFromConfig() { + //Thread safety is mostly around _acl. If _acl needs to be updated it is changed atomically + //More then one thread may be trying to update it at a time, but that is OK, because the + //change is atomic + long now = System.currentTimeMillis(); + if ((now - 5000) > _lastUpdate || _acl == null) { + Map acl = new HashMap(); + Map conf = Utils.findAndReadConfigFile(_aclFileName); + if (conf.containsKey(Config.DRPC_AUTHORIZER_ACL)) { + Map> confAcl = + (Map>) + conf.get(Config.DRPC_AUTHORIZER_ACL); + + for (String function : confAcl.keySet()) { + Map val = confAcl.get(function); + Collection clientUsers = + val.containsKey(CLIENT_USERS_KEY) ? + (Collection) val.get(CLIENT_USERS_KEY) : null; + String invocationUser = + val.containsKey(INVOCATION_USER_KEY) ? + (String) val.get(INVOCATION_USER_KEY) : null; + acl.put(function, + new AclFunctionEntry(clientUsers, invocationUser)); + } + } else if (!_permitWhenMissingFunctionEntry) { + LOG.warn("Requiring explicit ACL entries, but none given. " + + "Therefore, all operiations will be denied."); + } + _acl = acl; + _lastUpdate = System.currentTimeMillis(); + } + return _acl; + } + + @Override + public void prepare(Map conf) { + Boolean isStrict = + (Boolean) conf.get(Config.DRPC_AUTHORIZER_ACL_STRICT); + _permitWhenMissingFunctionEntry = + (isStrict != null && !isStrict) ? true : false; + _aclFileName = (String) conf.get(Config.DRPC_AUTHORIZER_ACL_FILENAME); + _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); + } + + private String getUserFromContext(ReqContext context) { + if (context != null) { + Principal princ = context.principal(); + if (princ != null) { + return princ.getName(); + } + } + return null; + } + + private String getLocalUserFromContext(ReqContext context) { + if (context != null) { + return _ptol.toLocal(context.principal()); + } + return null; + } + + protected boolean permitClientOrInvocationRequest(ReqContext context, Map params, + String fieldName) { + Map acl = readAclFromConfig(); + String function = (String) params.get(FUNCTION_KEY); + if (function != null && ! function.isEmpty()) { + AclFunctionEntry entry = acl.get(function); + if (entry == null && _permitWhenMissingFunctionEntry) { + return true; + } + if (entry != null) { + Object value; + try { + Field field = AclFunctionEntry.class.getDeclaredField(fieldName); + value = field.get(entry); + } catch (Exception ex) { + LOG.warn("Caught Exception while accessing ACL", ex); + return false; + } + String principal = getUserFromContext(context); + String user = getLocalUserFromContext(context); + if (value == null) { + LOG.warn("Configuration for function '"+function+"' is "+ + "invalid: it should have both an invocation user "+ + "and a list of client users defined."); + } else if (value instanceof Set && + (((Set)value).contains(principal) || + ((Set)value).contains(user))) { + return true; + } else if (value instanceof String && + (value.equals(principal) || + value.equals(user))) { + return true; + } + } + } + return false; + } + + @Override + protected boolean permitClientRequest(ReqContext context, String operation, + Map params) { + return permitClientOrInvocationRequest(context, params, "clientUsers"); + } + + @Override + protected boolean permitInvocationRequest(ReqContext context, String operation, + Map params) { + return permitClientOrInvocationRequest(context, params, "invocationUser"); + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java index 3af5e3c2587..5e84b382f89 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java @@ -49,9 +49,9 @@ public void prepare(Map conf) { public boolean permit(ReqContext context, String operation, Map topology_conf) { LOG.info("[req "+ context.requestID()+ "] Access " + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) - + " principal:"+ (context.principal() == null? "null" : context.principal()) + + (context.principal() == null? "" : (" principal:"+ context.principal())) +" op:"+operation - + " topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)); + + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)))); return false; } } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java index ef9568395b3..9af44d334df 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java @@ -49,9 +49,9 @@ public void prepare(Map conf) { public boolean permit(ReqContext context, String operation, Map topology_conf) { LOG.info("[req "+ context.requestID()+ "] Access " + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) - + " principal:"+(context.principal() == null? "null" : context.principal()) - +" op:"+ operation - + " topoology:"+ topology_conf.get(Config.TOPOLOGY_NAME)); + + (context.principal() == null? "" : (" principal:"+ context.principal())) + +" op:"+operation + + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)))); return true; } } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java new file mode 100644 index 00000000000..ef13750f6f4 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.authorizer; + +import java.util.Arrays; +import java.util.Map; +import java.util.Set; +import java.util.HashSet; +import java.util.Collection; + +import backtype.storm.Config; +import backtype.storm.security.auth.IAuthorizer; +import backtype.storm.security.auth.ReqContext; +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.security.auth.IPrincipalToLocal; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An authorization implementation that simply checks if a user is allowed to perform specific + * operations. + */ +public class SimpleACLAuthorizer implements IAuthorizer { + private static final Logger LOG = LoggerFactory.getLogger(SimpleACLAuthorizer.class); + + protected Set _userCommands = new HashSet(Arrays.asList("submitTopology", "fileUpload", "getNimbusConf", "getClusterInfo")); + protected Set _supervisorCommands = new HashSet(Arrays.asList("fileDownload")); + protected Set _topoCommands = new HashSet(Arrays.asList("killTopology","rebalance","activate","deactivate","getTopologyConf","getTopology","getUserTopology","getTopologyInfo","uploadNewCredentials")); + + protected Set _admins; + protected Set _supervisors; + protected IPrincipalToLocal _ptol; + + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + @Override + public void prepare(Map conf) { + _admins = new HashSet(); + _supervisors = new HashSet(); + + if (conf.containsKey(Config.NIMBUS_ADMINS)) { + _admins.addAll((Collection)conf.get(Config.NIMBUS_ADMINS)); + } + if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) { + _supervisors.addAll((Collection)conf.get(Config.NIMBUS_SUPERVISOR_USERS)); + } + _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); + } + + /** + * permit() method is invoked for each incoming Thrift request + * @param context request context includes info about + * @param operation operation name + * @param topology_storm configuration of targeted topology + * @return true if the request is authorized, false if reject + */ + @Override + public boolean permit(ReqContext context, String operation, Map topology_conf) { + + LOG.info("[req "+ context.requestID()+ "] Access " + + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) + + (context.principal() == null? "" : (" principal:"+ context.principal())) + +" op:"+operation + + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)))); + + String principal = context.principal().getName(); + String user = _ptol.toLocal(context.principal()); + + if (_admins.contains(principal) || _admins.contains(user)) { + return true; + } + + if (_supervisors.contains(principal) || _supervisors.contains(user)) { + return _supervisorCommands.contains(operation); + } + + if (_userCommands.contains(operation)) { + return true; + } + + if (_topoCommands.contains(operation)) { + Set topoUsers = new HashSet(); + if (topology_conf.containsKey(Config.TOPOLOGY_USERS)) { + topoUsers.addAll((Collection)topology_conf.get(Config.TOPOLOGY_USERS)); + } + + if (topoUsers.contains(principal) || topoUsers.contains(user)) { + return true; + } + } + + return false; + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java new file mode 100644 index 00000000000..55109f9dab1 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.authorizer; + +import java.util.Map; +import java.util.Set; +import java.util.HashSet; +import java.util.Collection; + +import backtype.storm.Config; +import backtype.storm.security.auth.IAuthorizer; +import backtype.storm.security.auth.ReqContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An authorization implementation that simply checks a whitelist of users that + * are allowed to use the cluster. + */ +public class SimpleWhitelistAuthorizer implements IAuthorizer { + private static final Logger LOG = LoggerFactory.getLogger(SimpleWhitelistAuthorizer.class); + public static String WHITELIST_USERS_CONF = "storm.auth.simple-white-list.users"; + protected Set users; + + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + @Override + public void prepare(Map conf) { + users = new HashSet(); + if (conf.containsKey(WHITELIST_USERS_CONF)) { + users.addAll((Collection)conf.get(WHITELIST_USERS_CONF)); + } + } + + /** + * permit() method is invoked for each incoming Thrift request + * @param context request context includes info about + * @param operation operation name + * @param topology_storm configuration of targeted topology + * @return true if the request is authorized, false if reject + */ + @Override + public boolean permit(ReqContext context, String operation, Map topology_conf) { + LOG.info("[req "+ context.requestID()+ "] Access " + + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) + + (context.principal() == null? "" : (" principal:"+ context.principal())) + +" op:"+operation + + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)))); + return context.principal() != null ? users.contains(context.principal().getName()) : false; + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java index 4abf98bba6e..cb68579060b 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java @@ -50,6 +50,7 @@ protected TTransportFactory getServerTransportFactory() throws IOException { return factory; } + @Override public TTransport connect(TTransport transport, String serverHost) throws TTransportException, IOException { ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf); TSaslClientTransport wrapper_transport = new TSaslClientTransport(DIGEST, diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java new file mode 100644 index 00000000000..52bf540e679 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java @@ -0,0 +1,267 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import backtype.storm.security.auth.IAutoCredentials; +import backtype.storm.security.auth.ICredentialsRenewer; +import backtype.storm.security.auth.AuthUtils; + +import java.util.Date; +import java.util.Map; +import java.util.Set; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.lang.reflect.Method; +import java.lang.reflect.Constructor; +import java.security.Principal; +import java.util.concurrent.atomic.AtomicReference; + +import javax.security.auth.kerberos.KerberosTicket; +import javax.security.auth.kerberos.KerberosPrincipal; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginException; +import javax.security.auth.login.LoginContext; +import javax.security.auth.RefreshFailedException; +import javax.security.auth.Subject; +import javax.xml.bind.DatatypeConverter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Automatically take a user's TGT, and push it, and renew it in Nimbus. + */ +public class AutoTGT implements IAutoCredentials, ICredentialsRenewer { + private static final Logger LOG = LoggerFactory.getLogger(AutoTGT.class); + private static final float TICKET_RENEW_WINDOW = 0.80f; + protected static AtomicReference kerbTicket = new AtomicReference(); + private Map conf; + + public void prepare(Map conf) { + this.conf = conf; + } + + private static KerberosTicket getTGT(Subject subject) { + Set tickets = subject.getPrivateCredentials(KerberosTicket.class); + for(KerberosTicket ticket: tickets) { + KerberosPrincipal server = ticket.getServer(); + if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) { + tickets = null; + return ticket; + } + } + tickets = null; + return null; + } + + @Override + public void populateCredentials(Map credentials) { + //Log the user in and get the TGT + try { + Configuration login_conf = AuthUtils.GetConfiguration(conf); + ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf); + + //login our user + Configuration.setConfiguration(login_conf); + LoginContext lc = new LoginContext(AuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler); + try { + lc.login(); + final Subject subject = lc.getSubject(); + KerberosTicket tgt = getTGT(subject); + + if (tgt == null) { //error + throw new RuntimeException("Fail to verify user principal with section \"" + +AuthUtils.LOGIN_CONTEXT_CLIENT+"\" in login configuration file "+ login_conf); + } + + if (!tgt.isForwardable()) { + throw new RuntimeException("The TGT found is not forwardable"); + } + + if (!tgt.isRenewable()) { + throw new RuntimeException("The TGT found is not renewable"); + } + + LOG.info("Pushing TGT for "+tgt.getClient()+" to topology."); + saveTGT(tgt, credentials); + } finally { + lc.logout(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static void saveTGT(KerberosTicket tgt, Map credentials) { + try { + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bao); + out.writeObject(tgt); + out.flush(); + out.close(); + credentials.put("TGT", DatatypeConverter.printBase64Binary(bao.toByteArray())); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static KerberosTicket getTGT(Map credentials) { + KerberosTicket ret = null; + if (credentials != null && credentials.containsKey("TGT")) { + try { + ByteArrayInputStream bin = new ByteArrayInputStream(DatatypeConverter.parseBase64Binary(credentials.get("TGT"))); + ObjectInputStream in = new ObjectInputStream(bin); + ret = (KerberosTicket)in.readObject(); + in.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + return ret; + } + + @Override + public void updateSubject(Subject subject, Map credentials) { + populateSubjectWithTGT(subject, credentials); + } + + @Override + public void populateSubject(Subject subject, Map credentials) { + populateSubjectWithTGT(subject, credentials); + loginHadoopUser(subject); + } + + private void populateSubjectWithTGT(Subject subject, Map credentials) { + KerberosTicket tgt = getTGT(credentials); + if (tgt != null) { + KerberosTicket oldTGT = getTGT(subject); + subject.getPrivateCredentials().add(tgt); + if (oldTGT != null && !oldTGT.equals(tgt)) { + subject.getPrivateCredentials().remove(oldTGT); + } + subject.getPrincipals().add(tgt.getClient()); + kerbTicket.set(tgt); + } else { + LOG.info("No TGT found in credentials"); + } + } + + /** + * Hadoop does not just go off of a TGT, it needs a bit more. This + * should fill in the rest. + * @param subject the subject that should have a TGT in it. + */ + private void loginHadoopUser(Subject subject) { + Class ugi = null; + try { + ugi = Class.forName("org.apache.hadoop.security.UserGroupInformation"); + } catch (ClassNotFoundException e) { + LOG.info("Hadoop was not found on the class path"); + return; + } + try { + Method isSecEnabled = ugi.getMethod("isSecurityEnabled"); + if (!((Boolean)isSecEnabled.invoke(null))) { + LOG.warn("Hadoop is on the classpath but not configured for " + + "security, if you want security you need to be sure that " + + "hadoop.security.authentication=kerberos in core-site.xml " + + "in your jar"); + return; + } + + try { + Method login = ugi.getMethod("loginUserFromSubject", Subject.class); + login.invoke(null, subject); + } catch (NoSuchMethodException me) { + //The version of Hadoop does not have the needed client changes. + // So don't look now, but do something really ugly to work around it. + // This is because we are reaching into the hidden bits of Hadoop security, and it works for now, but may stop at any point in time. + + //We are just trying to do the following + // Configuration conf = new Configuration(); + // HadoopKerberosName.setConfiguration(conf); + // subject.getPrincipals().add(new User(tgt.getClient().toString(), AuthenticationMethod.KERBEROS, null)); + String name = getTGT(subject).getClient().toString(); + + LOG.warn("The Hadoop client does not have loginUserFromSubject, Trying to hack around it. This may not work..."); + Class confClass = Class.forName("org.apache.hadoop.conf.Configuration"); + Constructor confCons = confClass.getConstructor(); + Object conf = confCons.newInstance(); + Class hknClass = Class.forName("org.apache.hadoop.security.HadoopKerberosName"); + Method hknSetConf = hknClass.getMethod("setConfiguration",confClass); + hknSetConf.invoke(null, conf); + + Class authMethodClass = Class.forName("org.apache.hadoop.security.UserGroupInformation$AuthenticationMethod"); + Object kerbAuthMethod = null; + for (Object authMethod : authMethodClass.getEnumConstants()) { + if ("KERBEROS".equals(authMethod.toString())) { + kerbAuthMethod = authMethod; + break; + } + } + + Class userClass = Class.forName("org.apache.hadoop.security.User"); + Constructor userCons = userClass.getConstructor(String.class, authMethodClass, LoginContext.class); + userCons.setAccessible(true); + Object user = userCons.newInstance(name, kerbAuthMethod, null); + subject.getPrincipals().add((Principal)user); + } + } catch (Exception e) { + LOG.warn("Something went wrong while trying to initialize Hadoop through reflection. This version of hadoop may not be compatible.", e); + } + } + + private long getRefreshTime(KerberosTicket tgt) { + long start = tgt.getStartTime().getTime(); + long end = tgt.getEndTime().getTime(); + return start + (long) ((end - start) * TICKET_RENEW_WINDOW); + } + + @Override + public void renew(Map credentials) { + KerberosTicket tgt = getTGT(credentials); + if (tgt != null) { + long refreshTime = getRefreshTime(tgt); + long now = System.currentTimeMillis(); + if (now >= refreshTime) { + try { + LOG.info("Renewing TGT for "+tgt.getClient()); + tgt.refresh(); + saveTGT(tgt, credentials); + } catch (RefreshFailedException e) { + LOG.warn("Failed to refresh TGT", e); + } + } + } + } + + public static void main(String[] args) throws Exception { + AutoTGT at = new AutoTGT(); + Map conf = new java.util.HashMap(); + conf.put("java.security.auth.login.config", args[0]); + at.prepare(conf); + Map creds = new java.util.HashMap(); + at.populateCredentials(creds); + Subject s = new Subject(); + at.populateSubject(s, creds); + LOG.info("Got a Subject "+s); + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java new file mode 100644 index 00000000000..807abe304ab --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.security.Principal; +import java.util.Map; +import javax.security.auth.Subject; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.kerberos.KerberosTicket; +import javax.security.auth.login.LoginException; +import javax.security.auth.spi.LoginModule; + + +/** + * Custom LoginModule to enable Auto Login based on cached ticket + */ +public class AutoTGTKrb5LoginModule implements LoginModule { + private static final Logger LOG = LoggerFactory.getLogger(AutoTGTKrb5LoginModule.class); + + // initial state + private Subject subject; + + protected KerberosTicket kerbTicket = null; + + public void initialize(Subject subject, + CallbackHandler callbackHandler, + Map sharedState, + Map options) { + + this.subject = subject; + } + + public boolean login() throws LoginException { + LOG.debug("Acquire TGT from Cache"); + getKerbTicketFromCache(); + if (kerbTicket != null) { + return true; + } else { + throw new LoginException("Authentication failed, the TGT not found."); + } + } + + protected void getKerbTicketFromCache() { + kerbTicket = AutoTGT.kerbTicket.get(); + } + + protected Principal getKerbTicketClient() { + if (kerbTicket != null) { + return kerbTicket.getClient(); + } + return null; + } + + public boolean commit() throws LoginException { + if (isSucceeded() == false) { + return false; + } + if (subject == null || subject.isReadOnly()) { + kerbTicket = null; + throw new LoginException("Authentication failed because the Subject is invalid."); + } + // Let us add the kerbClientPrinc and kerbTicket + subject.getPrivateCredentials().add(kerbTicket); + subject.getPrincipals().add(getKerbTicketClient()); + LOG.debug("Commit Succeeded."); + return true; + } + + public boolean abort() throws LoginException { + if (isSucceeded() == false) { + return false; + } else { + return logout(); + } + } + + public boolean logout() throws LoginException { + if (subject != null && !subject.isReadOnly() && kerbTicket != null) { + subject.getPrincipals().remove(kerbTicket.getClient()); + subject.getPrivateCredentials().remove(kerbTicket); + } + kerbTicket = null; + return true; + } + + private boolean isSucceeded() { + return kerbTicket != null; + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java new file mode 100644 index 00000000000..ba34fc9bfde --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import java.security.Principal; +import javax.security.auth.kerberos.KerberosTicket; + +/** + * Custom LoginModule extended for testing. + */ +public class AutoTGTKrb5LoginModuleTest extends AutoTGTKrb5LoginModule { + + public Principal client = null; + + public void setKerbTicket(KerberosTicket ticket) { + this.kerbTicket = ticket; + } + + @Override + protected void getKerbTicketFromCache() { + // Do nothing. + } + + @Override + protected Principal getKerbTicketClient() { + return this.client; + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java new file mode 100644 index 00000000000..d46aa8b5dad --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import java.io.IOException; +import java.util.Map; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.AuthUtils; + +/** + * SASL client side callback handler. + */ +public class ClientCallbackHandler implements CallbackHandler { + private static final Logger LOG = LoggerFactory.getLogger(ClientCallbackHandler.class); + + /** + * Constructor based on a JAAS configuration + * + * For digest, you should have a pair of user name and password defined in this figgure. + * + * @param configuration + * @throws IOException + */ + public ClientCallbackHandler(Configuration configuration) throws IOException { + if (configuration == null) return; + AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_CLIENT); + if (configurationEntries == null) { + String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_CLIENT + + "' entry in this configuration: Client cannot start."; + LOG.error(errorMessage); + throw new IOException(errorMessage); + } + } + + /** + * This method is invoked by SASL for authentication challenges + * @param callbacks a collection of challenge callbacks + */ + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + for (Callback c : callbacks) { + if (c instanceof NameCallback) { + LOG.debug("name callback"); + } else if (c instanceof PasswordCallback) { + LOG.debug("password callback"); + LOG.warn("Could not login: the client is being asked for a password, but the " + + " client code does not currently support obtaining a password from the user." + + " Make sure that the client is configured to use a ticket cache (using" + + " the JAAS configuration setting 'useTicketCache=true)' and restart the client. If" + + " you still get this message after that, the TGT in the ticket cache has expired and must" + + " be manually refreshed. To do so, first determine if you are using a password or a" + + " keytab. If the former, run kinit in a Unix shell in the environment of the user who" + + " is running this client using the command" + + " 'kinit ' (where is the name of the client's Kerberos principal)." + + " If the latter, do" + + " 'kinit -k -t ' (where is the name of the Kerberos principal, and" + + " is the location of the keytab file). After manually refreshing your cache," + + " restart this client. If you continue to see this message after manually refreshing" + + " your cache, ensure that your KDC host's clock is in sync with this host's clock."); + } else if (c instanceof AuthorizeCallback) { + LOG.debug("authorization callback"); + AuthorizeCallback ac = (AuthorizeCallback) c; + String authid = ac.getAuthenticationID(); + String authzid = ac.getAuthorizationID(); + if (authid.equals(authzid)) { + ac.setAuthorized(true); + } else { + ac.setAuthorized(false); + } + if (ac.isAuthorized()) { + ac.setAuthorizedID(authzid); + } + } else { + throw new UnsupportedCallbackException(c); + } + } + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java new file mode 100644 index 00000000000..451f87bb490 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java @@ -0,0 +1,206 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import java.io.IOException; +import java.security.Principal; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import javax.security.auth.Subject; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.kerberos.KerberosTicket; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginException; +import javax.security.sasl.Sasl; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.apache.zookeeper.Login; +import org.apache.zookeeper.server.auth.KerberosName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.security.auth.SaslTransportPlugin; + +public class KerberosSaslTransportPlugin extends SaslTransportPlugin { + public static final String KERBEROS = "GSSAPI"; + private static final Logger LOG = LoggerFactory.getLogger(KerberosSaslTransportPlugin.class); + + public TTransportFactory getServerTransportFactory() throws IOException { + //create an authentication callback handler + CallbackHandler server_callback_handler = new ServerCallbackHandler(login_conf); + + //login our principal + Subject subject = null; + try { + //specify a configuration object to be used + Configuration.setConfiguration(login_conf); + //now login + Login login = new Login(AuthUtils.LOGIN_CONTEXT_SERVER, server_callback_handler); + subject = login.getSubject(); + } catch (LoginException ex) { + LOG.error("Server failed to login in principal:" + ex, ex); + throw new RuntimeException(ex); + } + + //check the credential of our principal + if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) { + throw new RuntimeException("Fail to verify user principal with section \"" + +AuthUtils.LOGIN_CONTEXT_SERVER+"\" in login configuration file "+ login_conf); + } + + String principal = AuthUtils.get(login_conf, AuthUtils.LOGIN_CONTEXT_SERVER, "principal"); + LOG.debug("principal:"+principal); + KerberosName serviceKerberosName = new KerberosName(principal); + String serviceName = serviceKerberosName.getServiceName(); + String hostName = serviceKerberosName.getHostName(); + Map props = new TreeMap(); + props.put(Sasl.QOP, "auth"); + props.put(Sasl.SERVER_AUTH, "false"); + + //create a transport factory that will invoke our auth callback for digest + TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory(); + factory.addServerDefinition(KERBEROS, serviceName, hostName, props, server_callback_handler); + + //create a wrap transport factory so that we could apply user credential during connections + TUGIAssumingTransportFactory wrapFactory = new TUGIAssumingTransportFactory(factory, subject); + + LOG.info("SASL GSSAPI transport factory will be used"); + return wrapFactory; + } + + public TTransport connect(TTransport transport, String serverHost) throws TTransportException, IOException { + //create an authentication callback handler + ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf); + + //login our user + Login login = null; + try { + //specify a configuration object to be used + Configuration.setConfiguration(login_conf); + //now login + login = new Login(AuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler); + } catch (LoginException ex) { + LOG.error("Server failed to login in principal:" + ex, ex); + throw new RuntimeException(ex); + } + + final Subject subject = login.getSubject(); + if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) { //error + throw new RuntimeException("Fail to verify user principal with section \"" + +AuthUtils.LOGIN_CONTEXT_CLIENT+"\" in login configuration file "+ login_conf); + } + + final String principal = getPrincipal(subject); + String serviceName = AuthUtils.get(login_conf, AuthUtils.LOGIN_CONTEXT_CLIENT, "serviceName"); + if (serviceName == null) { + serviceName = AuthUtils.SERVICE; + } + Map props = new TreeMap(); + props.put(Sasl.QOP, "auth"); + props.put(Sasl.SERVER_AUTH, "false"); + + LOG.debug("SASL GSSAPI client transport is being established"); + final TTransport sasalTransport = new TSaslClientTransport(KERBEROS, + principal, + serviceName, + serverHost, + props, + null, + transport); + + //open Sasl transport with the login credential + try { + Subject.doAs(subject, + new PrivilegedExceptionAction() { + public Void run() { + try { + LOG.debug("do as:"+ principal); + sasalTransport.open(); + } + catch (Exception e) { + LOG.error("Client failed to open SaslClientTransport to interact with a server during session initiation: " + e, e); + } + return null; + } + }); + } catch (PrivilegedActionException e) { + throw new RuntimeException(e); + } + + return sasalTransport; + } + + private String getPrincipal(Subject subject) { + Set principals = (Set)subject.getPrincipals(); + if (principals==null || principals.size()<1) { + LOG.info("No principal found in login subject"); + return null; + } + return ((Principal)(principals.toArray()[0])).getName(); + } + + /** A TransportFactory that wraps another one, but assumes a specified UGI + * before calling through. + * + * This is used on the server side to assume the server's Principal when accepting + * clients. + */ + static class TUGIAssumingTransportFactory extends TTransportFactory { + private final Subject subject; + private final TTransportFactory wrapped; + + public TUGIAssumingTransportFactory(TTransportFactory wrapped, Subject subject) { + this.wrapped = wrapped; + this.subject = subject; + + Set principals = (Set)subject.getPrincipals(); + if (principals.size()>0) + LOG.info("Service principal:"+ ((Principal)(principals.toArray()[0])).getName()); + } + + @Override + public TTransport getTransport(final TTransport trans) { + try { + return Subject.doAs(subject, + new PrivilegedExceptionAction() { + public TTransport run() { + try { + return wrapped.getTransport(trans); + } + catch (Exception e) { + LOG.error("Storm server failed to open transport to interact with a client during session initiation: " + e, e); + return null; + } + } + }); + } catch (PrivilegedActionException e) { + LOG.error("Storm server experienced a PrivilegedActionException exception while creating a transport using a JAAS principal context:" + e, e); + return null; + } + } + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java new file mode 100644 index 00000000000..9dc75c4db37 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + +import backtype.storm.security.auth.AuthUtils; + +/** + * SASL server side callback handler + */ +public class ServerCallbackHandler implements CallbackHandler { + private static final Logger LOG = LoggerFactory.getLogger(ServerCallbackHandler.class); + + private String userName; + + public ServerCallbackHandler(Configuration configuration) throws IOException { + if (configuration==null) return; + + AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_SERVER); + if (configurationEntries == null) { + String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_SERVER+"' entry in this configuration: Server cannot start."; + LOG.error(errorMessage); + throw new IOException(errorMessage); + } + } + + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + handleNameCallback((NameCallback) callback); + } else if (callback instanceof PasswordCallback) { + handlePasswordCallback((PasswordCallback) callback); + } else if (callback instanceof AuthorizeCallback) { + handleAuthorizeCallback((AuthorizeCallback) callback); + } + } + } + + private void handleNameCallback(NameCallback nc) { + LOG.debug("handleNameCallback"); + userName = nc.getDefaultName(); + nc.setName(nc.getDefaultName()); + } + + private void handlePasswordCallback(PasswordCallback pc) { + LOG.warn("No password found for user: " + userName); + } + + private void handleAuthorizeCallback(AuthorizeCallback ac) { + String authenticationID = ac.getAuthenticationID(); + LOG.debug("Successfully authenticated client: authenticationID=" + authenticationID); + ac.setAuthorized(true); + + ac.setAuthorizedID(authenticationID); + } +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf new file mode 100644 index 00000000000..92a1399ac63 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf @@ -0,0 +1,31 @@ +/* +This is a sample JAAS configuration for Storm servers to handle Kerberos authentication +*/ + +/* + StormServer section should contains the info about server keytab file and server principal. + In Storm, we have 2 thrift servers: Nimbus and DRPC. These servers could be assigned with + different principals. +*/ +StormServer { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + keyTab="/etc/storm_server.keytab" + storeKey=true + useTicketCache=false + principal="storm_service/carcloth.corp.acme.com@STORM.CORP.ACME.COM"; +}; + +/* +StormClient section should contains the info about client keytab file and client principal. +For example, Supervisors are clients of Nimbus, and we should assign keytab/principal for supervisors. +*/ +StormClient { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + keyTab="/etc/storm_client.keytab" + storeKey=true + useTicketCache=false + serviceName="storm_service"; +}; + diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf new file mode 100644 index 00000000000..138e1f3f2e8 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf @@ -0,0 +1,12 @@ +/* + This is a sample JAAS configuration for Storm topology launcher/submitter. + Since launcher machines are typically accessible by many folks, we + encourage you to leverage "kinit", instead of keytab. +*/ +StormClient { + com.sun.security.auth.module.Krb5LoginModule required + doNotPrompt=true + useTicketCache=true + serviceName="storm_service"; +}; + diff --git a/storm-core/src/jvm/backtype/storm/testing/SingleUserSimpleTransport.java b/storm-core/src/jvm/backtype/storm/testing/SingleUserSimpleTransport.java new file mode 100644 index 00000000000..4d25ac747dc --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/testing/SingleUserSimpleTransport.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.testing; + +import backtype.storm.security.auth.SimpleTransportPlugin; +import javax.security.auth.Subject; +import java.security.Principal; +import java.util.HashSet; + + +public class SingleUserSimpleTransport extends SimpleTransportPlugin { + @Override + protected Subject getDefaultSubject() { + HashSet principals = new HashSet(); + principals.add(new Principal() { + public String getName() { return "user"; } + public String toString() { return "user"; } + }); + return new Subject(true, principals, new HashSet(), new HashSet()); + } +} diff --git a/storm-core/src/jvm/backtype/storm/transactional/state/TestTransactionalState.java b/storm-core/src/jvm/backtype/storm/transactional/state/TestTransactionalState.java new file mode 100644 index 00000000000..3d4a4632390 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/transactional/state/TestTransactionalState.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.transactional.state; + +import java.util.List; +import java.util.Map; + +import backtype.storm.utils.ZookeeperAuthInfo; +import org.apache.curator.framework.CuratorFramework; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.data.ACL; + +/** + * Facilitates testing of non-public methods in the parent class. + */ +public class TestTransactionalState extends TransactionalState { + + /** + * Matching constructor in absence of a default constructor in the parent + * class. + */ + protected TestTransactionalState(Map conf, String id, Map componentConf, String subroot) { + super(conf, id, componentConf, subroot); + } + + public static void createNode(CuratorFramework curator, + String rootDir, byte[] data, List acls, CreateMode mode) + throws Exception { + TransactionalState.createNode(curator, rootDir, data, acls, mode); + } +} diff --git a/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java b/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java index 91fc6669623..5afcd0a2e5c 100644 --- a/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java +++ b/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java @@ -21,18 +21,25 @@ import backtype.storm.serialization.KryoValuesDeserializer; import backtype.storm.serialization.KryoValuesSerializer; import backtype.storm.utils.Utils; +import backtype.storm.utils.ZookeeperAuthInfo; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.ProtectACLCreateModePathAndBytesable; +import org.apache.curator.framework.api.PathAndBytesable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; public class TransactionalState { CuratorFramework _curator; KryoValuesSerializer _ser; KryoValuesDeserializer _des; + List _zkAcls = null; public static TransactionalState newUserState(Map conf, String id, Map componentConf) { return new TransactionalState(conf, id, componentConf, "user"); @@ -51,26 +58,55 @@ protected TransactionalState(Map conf, String id, Map componentConf, String subr componentConf .get(Config.TOPOLOGY_KRYO_REGISTER)); } - String rootDir = conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) + "/" + id + "/" + subroot; + String transactionalRoot = (String)conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT); + String rootDir = transactionalRoot + "/" + id + "/" + subroot; List servers = (List) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS); Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT); - CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port); + ZookeeperAuthInfo auth = new ZookeeperAuthInfo(conf); + CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port, auth); + _zkAcls = Utils.getWorkerACL(conf); try { - initter.create().creatingParentsIfNeeded().forPath(rootDir); - } catch(KeeperException.NodeExistsException e) { - + TransactionalState.createNode(initter, transactionalRoot, null, null, null); + } catch (KeeperException.NodeExistsException e) { + } + try { + TransactionalState.createNode(initter, rootDir, null, _zkAcls, null); + } catch (KeeperException.NodeExistsException e) { } - initter.close(); - _curator = Utils.newCuratorStarted(conf, servers, port, rootDir); + _curator = Utils.newCuratorStarted(conf, servers, port, rootDir, auth); _ser = new KryoValuesSerializer(conf); _des = new KryoValuesDeserializer(conf); } catch (Exception e) { throw new RuntimeException(e); } } + + protected static String forPath(PathAndBytesable builder, + String path, byte[] data) throws Exception { + return (data == null) + ? builder.forPath(path) + : builder.forPath(path, data); + } + + protected static void createNode(CuratorFramework curator, String path, + byte[] data, List acls, CreateMode mode) throws Exception { + ProtectACLCreateModePathAndBytesable builder = + curator.create().creatingParentsIfNeeded(); + if (acls == null) { + if (mode == null ) { + TransactionalState.forPath(builder, path, data); + } else { + TransactionalState.forPath(builder.withMode(mode), path, data); + } + return; + } + + TransactionalState.forPath(builder.withACL(acls), path, data); + } + public void setData(String path, Object obj) { path = "/" + path; byte[] ser = _ser.serializeObject(obj); @@ -78,10 +114,8 @@ public void setData(String path, Object obj) { if(_curator.checkExists().forPath(path)!=null) { _curator.setData().forPath(path, ser); } else { - _curator.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(path, ser); + TransactionalState.createNode(_curator, path, ser, _zkAcls, + CreateMode.PERSISTENT); } } catch(Exception e) { throw new RuntimeException(e); diff --git a/storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java b/storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java new file mode 100644 index 00000000000..9d0ee92a41f --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java @@ -0,0 +1,20 @@ +package backtype.storm.ui; + +public class InvalidRequestException extends Exception { + + public InvalidRequestException() { + super(); + } + + public InvalidRequestException(String msg) { + super(msg); + } + + public InvalidRequestException(String msg, Throwable cause) { + super(msg, cause); + } + + public InvalidRequestException(Throwable cause) { + super(cause); + } +} diff --git a/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java b/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java index cf38fb821fb..3218e49940a 100644 --- a/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java +++ b/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java @@ -17,68 +17,49 @@ */ package backtype.storm.utils; +import backtype.storm.Config; import backtype.storm.generated.DRPCExecutionException; import backtype.storm.generated.DistributedRPC; +import backtype.storm.generated.AuthorizationException; import org.apache.thrift.TException; -import org.apache.thrift.protocol.TBinaryProtocol; -import org.apache.thrift.transport.TFramedTransport; -import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; +import backtype.storm.security.auth.ThriftClient; +import backtype.storm.security.auth.ThriftConnectionType; +import org.apache.thrift.transport.TTransportException; -public class DRPCClient implements DistributedRPC.Iface { +import java.util.Map; + +public class DRPCClient extends ThriftClient implements DistributedRPC.Iface { private TTransport conn; private DistributedRPC.Client client; private String host; private int port; private Integer timeout; - public DRPCClient(String host, int port, Integer timeout) { - try { - this.host = host; - this.port = port; - this.timeout = timeout; - connect(); - } catch(TException e) { - throw new RuntimeException(e); - } - } - - public DRPCClient(String host, int port) { - this(host, port, null); + public DRPCClient(Map conf, String host, int port) throws TTransportException { + this(conf, host, port, null); } - - private void connect() throws TException { - TSocket socket = new TSocket(host, port); - if(timeout!=null) { - socket.setTimeout(timeout); - } - conn = new TFramedTransport(socket); - client = new DistributedRPC.Client(new TBinaryProtocol(conn)); - conn.open(); + + public DRPCClient(Map conf, String host, int port, Integer timeout) throws TTransportException { + super(conf, ThriftConnectionType.DRPC, host, port, timeout); + this.host = host; + this.port = port; + this.client = new DistributedRPC.Client(_protocol); } - + public String getHost() { return host; } public int getPort() { return port; - } + } - public String execute(String func, String args) throws TException, DRPCExecutionException { - try { - if(client==null) connect(); - return client.execute(func, args); - } catch(TException e) { - client = null; - throw e; - } catch(DRPCExecutionException e) { - client = null; - throw e; - } + public String execute(String func, String args) throws TException, DRPCExecutionException, AuthorizationException { + return client.execute(func, args); } - public void close() { - conn.close(); + public DistributedRPC.Client getClient() { + return client; } } diff --git a/storm-core/src/jvm/backtype/storm/utils/LocalState.java b/storm-core/src/jvm/backtype/storm/utils/LocalState.java index 0d0ae0754de..f412ff3b87a 100644 --- a/storm-core/src/jvm/backtype/storm/utils/LocalState.java +++ b/storm-core/src/jvm/backtype/storm/utils/LocalState.java @@ -23,16 +23,19 @@ import java.util.Map; import java.util.HashMap; import java.io.IOException; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A simple, durable, atomic K/V database. *Very inefficient*, should only be used for occasional reads/writes. * Every read/write hits disk. */ public class LocalState { + public static Logger LOG = LoggerFactory.getLogger(LocalState.class); private VersionedStore _vs; public LocalState(String backingDir) throws IOException { + LOG.debug("New Local State for {}", backingDir); _vs = new VersionedStore(backingDir); } @@ -83,8 +86,14 @@ public synchronized void cleanup(int keepVersions) throws IOException { private void persist(Map val, boolean cleanup) throws IOException { byte[] toWrite = Utils.serialize(val); String newPath = _vs.createVersion(); - FileUtils.writeByteArrayToFile(new File(newPath), toWrite); + File file = new File(newPath); + FileUtils.writeByteArrayToFile(file, toWrite); + if (toWrite.length != file.length()) { + throw new IOException("Tried to serialize " + toWrite.length + + " bytes to " + file.getCanonicalPath() + ", but " + + file.length() + " bytes were written."); + } _vs.succeedVersion(newPath); if(cleanup) _vs.cleanup(4); } -} \ No newline at end of file +} diff --git a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java index e93acc8acc7..273e232fb32 100644 --- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java +++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java @@ -19,6 +19,7 @@ import backtype.storm.Config; import backtype.storm.security.auth.ThriftClient; +import backtype.storm.security.auth.ThriftConnectionType; import backtype.storm.generated.Nimbus; import java.util.Map; import org.apache.thrift.transport.TTransportException; @@ -32,8 +33,7 @@ public class NimbusClient extends ThriftClient { public static NimbusClient getConfiguredClient(Map conf) { try { String nimbusHost = (String) conf.get(Config.NIMBUS_HOST); - int nimbusPort = Utils.getInt(conf.get(Config.NIMBUS_THRIFT_PORT)); - return new NimbusClient(conf, nimbusHost, nimbusPort); + return new NimbusClient(conf, nimbusHost); } catch (TTransportException ex) { throw new RuntimeException(ex); } @@ -44,7 +44,12 @@ public NimbusClient(Map conf, String host, int port) throws TTransportException } public NimbusClient(Map conf, String host, int port, Integer timeout) throws TTransportException { - super(conf, host, port, timeout); + super(conf, ThriftConnectionType.NIMBUS, host, port, timeout); + _client = new Nimbus.Client(_protocol); + } + + public NimbusClient(Map conf, String host) throws TTransportException { + super(conf, ThriftConnectionType.NIMBUS, host, null, null); _client = new Nimbus.Client(_protocol); } diff --git a/storm-core/src/jvm/backtype/storm/utils/TestUtils.java b/storm-core/src/jvm/backtype/storm/utils/TestUtils.java new file mode 100644 index 00000000000..276559c4192 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/utils/TestUtils.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.utils; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import java.util.Map; + +public class TestUtils extends Utils { + + public static void testSetupBuilder(CuratorFrameworkFactory.Builder + builder, String zkStr, Map conf, ZookeeperAuthInfo auth) + { + setupBuilder(builder, zkStr, conf, auth); + } + +} diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java index a1fed963a76..c28d93ae5c7 100644 --- a/storm-core/src/jvm/backtype/storm/utils/Utils.java +++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java @@ -17,6 +17,7 @@ */ package backtype.storm.utils; +import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.FileOutputStream; @@ -45,7 +46,12 @@ import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.commons.lang.StringUtils; import org.apache.thrift.TException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; import org.json.simple.JSONValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.yaml.snakeyaml.Yaml; import org.yaml.snakeyaml.constructor.SafeConstructor; @@ -53,10 +59,13 @@ import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.ComponentObject; import backtype.storm.generated.StormTopology; +import backtype.storm.generated.AuthorizationException; + import clojure.lang.IFn; import clojure.lang.RT; public class Utils { + public static Logger LOG = LoggerFactory.getLogger(Utils.class); public static final String DEFAULT_STREAM_ID = "default"; public static Object newInstance(String klass) { @@ -254,7 +263,7 @@ public static List tuple(Object... values) { return ret; } - public static void downloadFromMaster(Map conf, String file, String localFile) throws IOException, TException { + public static void downloadFromMaster(Map conf, String file, String localFile) throws AuthorizationException, IOException, TException { NimbusClient client = NimbusClient.getConfiguredClient(conf); String id = client.getClient().beginFileDownload(file); WritableByteChannel out = Channels.newChannel(new FileOutputStream(localFile)); @@ -307,6 +316,8 @@ public static Integer getInt(Object o) { return (Integer) o; } else if (o instanceof Short) { return ((Short) o).intValue(); + } else if (o instanceof String) { + return Integer.parseInt((String) o); } else { throw new IllegalArgumentException("Don't know how to convert " + o + " + to int"); } @@ -316,11 +327,6 @@ public static long secureRandomLong() { return UUID.randomUUID().getLeastSignificantBits(); } - - public static CuratorFramework newCurator(Map conf, List servers, Object port, String root) { - return newCurator(conf, servers, port, root, null); - } - public static class BoundedExponentialBackoffRetry extends ExponentialBackoffRetry { protected final int maxRetryInterval; @@ -350,32 +356,39 @@ public static CuratorFramework newCurator(Map conf, List servers, Object serverPorts.add(zkServer + ":" + Utils.getInt(port)); } String zkStr = StringUtils.join(serverPorts, ",") + root; - CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder() - .connectString(zkStr) - .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT))) - .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT))) - .retryPolicy(new BoundedExponentialBackoffRetry( + CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder(); + + setupBuilder(builder, zkStr, conf, auth); + + return builder.build(); + } + + protected static void setupBuilder(CuratorFrameworkFactory.Builder builder, String zkStr, Map conf, ZookeeperAuthInfo auth) + { + builder.connectString(zkStr) + .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT))) + .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT))) + .retryPolicy(new BoundedExponentialBackoffRetry( Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)), Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)), Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)))); - if(auth!=null && auth.scheme!=null) { + if(auth!=null && auth.scheme!=null && auth.payload!=null) { builder = builder.authorization(auth.scheme, auth.payload); } - return builder.build(); } - public static CuratorFramework newCurator(Map conf, List servers, Object port) { - return newCurator(conf, servers, port, ""); + public static CuratorFramework newCurator(Map conf, List servers, Object port, ZookeeperAuthInfo auth) { + return newCurator(conf, servers, port, "", auth); } - public static CuratorFramework newCuratorStarted(Map conf, List servers, Object port, String root) { - CuratorFramework ret = newCurator(conf, servers, port, root); + public static CuratorFramework newCuratorStarted(Map conf, List servers, Object port, String root, ZookeeperAuthInfo auth) { + CuratorFramework ret = newCurator(conf, servers, port, root, auth); ret.start(); return ret; } - public static CuratorFramework newCuratorStarted(Map conf, List servers, Object port) { - CuratorFramework ret = newCurator(conf, servers, port); + public static CuratorFramework newCuratorStarted(Map conf, List servers, Object port, ZookeeperAuthInfo auth) { + CuratorFramework ret = newCurator(conf, servers, port, auth); ret.start(); return ret; } @@ -413,6 +426,18 @@ public static byte[] toByteArray(ByteBuffer buffer) { return ret; } + public static void readAndLogStream(String prefix, InputStream in) { + try { + BufferedReader r = new BufferedReader(new InputStreamReader(in)); + String line = null; + while ((line = r.readLine())!= null) { + LOG.info("{}:{}", prefix, line); + } + } catch (IOException e) { + LOG.warn("Error whiel trying to log stream", e); + } + } + public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwable) { Throwable t = throwable; while(t != null) { @@ -423,4 +448,67 @@ public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwabl } return false; } + + /** + * Is the cluster configured to interact with ZooKeeper in a secure way? + * This only works when called from within Nimbus or a Supervisor process. + * @param conf the storm configuration, not the topology configuration + * @return true if it is configured else false. + */ + public static boolean isZkAuthenticationConfiguredStormServer(Map conf) { + return null != System.getProperty("java.security.auth.login.config") + || (conf != null + && conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME) != null + && ! ((String)conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME)).isEmpty()); + } + + /** + * Is the topology configured to have ZooKeeper authentication. + * @param conf the topology configuration + * @return true if ZK is configured else false + */ + public static boolean isZkAuthenticationConfiguredTopology(Map conf) { + return (conf != null + && conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME) != null + && ! ((String)conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME)).isEmpty()); + } + + public static List getWorkerACL(Map conf) { + //This is a work around to an issue with ZK where a sasl super user is not super unless there is an open SASL ACL so we are trying to give the correct perms + if (!isZkAuthenticationConfiguredTopology(conf)) { + return null; + } + String stormZKUser = (String)conf.get(Config.STORM_ZOOKEEPER_SUPERACL); + if (stormZKUser == null) { + throw new IllegalArgumentException("Authentication is enabled but "+Config.STORM_ZOOKEEPER_SUPERACL+" is not set"); + } + String[] split = stormZKUser.split(":",2); + if (split.length != 2) { + throw new IllegalArgumentException(Config.STORM_ZOOKEEPER_SUPERACL+" does not appear to be in the form scheme:acl, i.e. sasl:storm-user"); + } + ArrayList ret = new ArrayList(ZooDefs.Ids.CREATOR_ALL_ACL); + ret.add(new ACL(ZooDefs.Perms.ALL, new Id(split[0], split[1]))); + return ret; + } + + public static String threadDump() { + final StringBuilder dump = new StringBuilder(); + final java.lang.management.ThreadMXBean threadMXBean = java.lang.management.ManagementFactory.getThreadMXBean(); + final java.lang.management.ThreadInfo[] threadInfos = threadMXBean.getThreadInfo(threadMXBean.getAllThreadIds(), 100); + for (java.lang.management.ThreadInfo threadInfo : threadInfos) { + dump.append('"'); + dump.append(threadInfo.getThreadName()); + dump.append("\" "); + final Thread.State state = threadInfo.getThreadState(); + dump.append("\n java.lang.Thread.State: "); + dump.append(state); + final StackTraceElement[] stackTraceElements = threadInfo.getStackTrace(); + for (final StackTraceElement stackTraceElement : stackTraceElements) { + dump.append("\n at "); + dump.append(stackTraceElement); + } + dump.append("\n\n"); + } + return dump.toString(); + } } diff --git a/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java b/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java index a5a2e9a8c54..d972135a707 100644 --- a/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java +++ b/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java @@ -27,8 +27,13 @@ public class ZookeeperAuthInfo { public byte[] payload = null; public ZookeeperAuthInfo(Map conf) { - String scheme = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME); - String payload = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD); + String scheme = (String) conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME); + String payload = (String) conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD); + + if (scheme == null || payload == null) { + scheme = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME); + payload = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD); + } if(scheme!=null) { this.scheme = scheme; if(payload != null) { diff --git a/storm-core/src/jvm/backtype/storm/utils/ZookeeperServerCnxnFactory.java b/storm-core/src/jvm/backtype/storm/utils/ZookeeperServerCnxnFactory.java new file mode 100644 index 00000000000..08a763aaeaf --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/utils/ZookeeperServerCnxnFactory.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.utils; + +import java.io.IOException; +import java.net.BindException; +import java.net.InetSocketAddress; + +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ZookeeperServerCnxnFactory { + private static final Logger LOG = LoggerFactory.getLogger(ZookeeperServerCnxnFactory.class); + int _port; + NIOServerCnxnFactory _factory; + + public ZookeeperServerCnxnFactory(int port, int maxClientCnxns) { + //port range + int max; + if (port <= 0) { + _port = 2000; + max = 65535; + } else { + _port = port; + max = port; + } + + try { + _factory = new NIOServerCnxnFactory(); + } catch (IOException e) { + _port = 0; + _factory = null; + e.printStackTrace(); + throw new RuntimeException(e.getMessage()); + } + + //look for available port + for (; _port <= max; _port++) { + try { + _factory.configure(new InetSocketAddress(_port), maxClientCnxns); + LOG.debug("Zookeeper server successfully binded at port "+_port); + break; + } catch (BindException e1) { + } catch (IOException e2) { + _port = 0; + _factory = null; + e2.printStackTrace(); + throw new RuntimeException(e2.getMessage()); + } + } + + if (_port > max) { + _port = 0; + _factory = null; + LOG.error("Failed to find a port for Zookeeper"); + throw new RuntimeException("No port is available to launch an inprocess zookeeper."); + } + } + + public int port() { + return _port; + } + + public NIOServerCnxnFactory factory() { + return _factory; + } +} diff --git a/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java b/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java index 4ebb667b27e..d49371a6110 100644 --- a/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java +++ b/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java @@ -20,6 +20,7 @@ import backtype.storm.Config; import backtype.storm.drpc.DRPCInvocationsClient; import backtype.storm.generated.DistributedRPCInvocations; +import backtype.storm.generated.AuthorizationException; import backtype.storm.utils.ServiceRegistry; import backtype.storm.utils.Utils; import java.util.ArrayList; @@ -28,6 +29,7 @@ import java.util.Map; import org.apache.commons.lang.builder.ToStringBuilder; import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; import org.json.simple.JSONValue; import storm.trident.drpc.ReturnResultsReducer.ReturnResultsState; import storm.trident.operation.MultiReducer; @@ -47,12 +49,13 @@ public String toString() { } } boolean local; - + Map conf; Map _clients = new HashMap(); @Override public void prepare(Map conf, TridentMultiReducerContext context) { + this.conf = conf; local = conf.get(Config.STORM_CLUSTER_MODE).equals("local"); } @@ -89,7 +92,11 @@ public void complete(ReturnResultsState state, TridentCollector collector) { }}; if(!_clients.containsKey(server)) { - _clients.put(server, new DRPCInvocationsClient(host, port)); + try { + _clients.put(server, new DRPCInvocationsClient(conf, host, port)); + } catch (TTransportException ex) { + throw new RuntimeException(ex); + } } client = _clients.get(server); } @@ -98,6 +105,8 @@ public void complete(ReturnResultsState state, TridentCollector collector) { client.result(id, result); } catch(TException e) { collector.reportError(e); + } catch (AuthorizationException aze) { + collector.reportError(aze); } } } diff --git a/storm-core/src/jvm/storm/trident/topology/state/TestTransactionalState.java b/storm-core/src/jvm/storm/trident/topology/state/TestTransactionalState.java new file mode 100644 index 00000000000..ff3edb61ab8 --- /dev/null +++ b/storm-core/src/jvm/storm/trident/topology/state/TestTransactionalState.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package storm.trident.topology.state; + +import java.util.List; +import java.util.Map; + +import backtype.storm.utils.ZookeeperAuthInfo; +import org.apache.curator.framework.CuratorFramework; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.data.ACL; + +/** + * Facilitates testing of non-public methods in the parent class. + */ +public class TestTransactionalState extends TransactionalState { + + /** + * Matching constructor in absence of a default constructor in the parent + * class. + */ + protected TestTransactionalState(Map conf, String id, String subroot) { + super(conf, id, subroot); + } + + public static void createNode(CuratorFramework curator, + String rootDir, byte[] data, List acls, CreateMode mode) + throws Exception { + TransactionalState.createNode(curator, rootDir, data, acls, mode); + } +} diff --git a/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java b/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java index 532513741ec..5fba1a22d78 100644 --- a/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java +++ b/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java @@ -20,7 +20,12 @@ import backtype.storm.Config; import backtype.storm.utils.Utils; +import backtype.storm.utils.ZookeeperAuthInfo; + import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.ProtectACLCreateModePathAndBytesable; +import org.apache.curator.framework.api.PathAndBytesable; + import java.io.UnsupportedEncodingException; import java.util.ArrayList; import java.util.HashMap; @@ -28,10 +33,14 @@ import java.util.Map; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; import org.json.simple.JSONValue; public class TransactionalState { CuratorFramework _curator; + List _zkAcls = null; public static TransactionalState newUserState(Map conf, String id) { return new TransactionalState(conf, id, "user"); @@ -44,24 +53,53 @@ public static TransactionalState newCoordinatorState(Map conf, String id) { protected TransactionalState(Map conf, String id, String subroot) { try { conf = new HashMap(conf); - String rootDir = conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) + "/" + id + "/" + subroot; + String transactionalRoot = (String)conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT); + String rootDir = transactionalRoot + "/" + id + "/" + subroot; List servers = (List) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS); Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT); - CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port); + ZookeeperAuthInfo auth = new ZookeeperAuthInfo(conf); + CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port, auth); + _zkAcls = Utils.getWorkerACL(conf); + try { + TransactionalState.createNode(initter, transactionalRoot, null, null, null); + } catch (KeeperException.NodeExistsException e) { + } try { - initter.create().creatingParentsIfNeeded().forPath(rootDir); - } catch(KeeperException.NodeExistsException e) { - + TransactionalState.createNode(initter, rootDir, null, _zkAcls, null); + } catch (KeeperException.NodeExistsException e) { } - initter.close(); - _curator = Utils.newCuratorStarted(conf, servers, port, rootDir); + _curator = Utils.newCuratorStarted(conf, servers, port, rootDir, auth); } catch (Exception e) { throw new RuntimeException(e); } } + + protected static String forPath(PathAndBytesable builder, + String path, byte[] data) throws Exception { + return (data == null) + ? builder.forPath(path) + : builder.forPath(path, data); + } + + protected static void createNode(CuratorFramework curator, String path, + byte[] data, List acls, CreateMode mode) throws Exception { + ProtectACLCreateModePathAndBytesable builder = + curator.create().creatingParentsIfNeeded(); + if (acls == null) { + if (mode == null ) { + TransactionalState.forPath(builder, path, data); + } else { + TransactionalState.forPath(builder.withMode(mode), path, data); + } + return; + } + + TransactionalState.forPath(builder.withACL(acls), path, data); + } + public void setData(String path, Object obj) { path = "/" + path; byte[] ser; @@ -74,10 +112,8 @@ public void setData(String path, Object obj) { if(_curator.checkExists().forPath(path)!=null) { _curator.setData().forPath(path, ser); } else { - _curator.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(path, ser); + TransactionalState.createNode(_curator, path, ser, _zkAcls, + CreateMode.PERSISTENT); } } catch(Exception e) { throw new RuntimeException(e); diff --git a/storm-core/src/native/worker-launcher/.autom4te.cfg b/storm-core/src/native/worker-launcher/.autom4te.cfg new file mode 100644 index 00000000000..1ec584f4f87 --- /dev/null +++ b/storm-core/src/native/worker-launcher/.autom4te.cfg @@ -0,0 +1,42 @@ +# +# 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. +# + +# +# autom4te configuration for worker-launcher +# + +begin-language: "Autoheader-preselections" +args: --no-cache +end-language: "Autoheader-preselections" + +begin-language: "Automake-preselections" +args: --no-cache +end-language: "Automake-preselections" + +begin-language: "Autoreconf-preselections" +args: --no-cache +end-language: "Autoreconf-preselections" + +begin-language: "Autoconf-without-aclocal-m4" +args: --no-cache +end-language: "Autoconf-without-aclocal-m4" + +begin-language: "Autoconf" +args: --no-cache +end-language: "Autoconf" + diff --git a/storm-core/src/native/worker-launcher/.deps/worker-launcher.Po b/storm-core/src/native/worker-launcher/.deps/worker-launcher.Po new file mode 100644 index 00000000000..9ce06a81ea4 --- /dev/null +++ b/storm-core/src/native/worker-launcher/.deps/worker-launcher.Po @@ -0,0 +1 @@ +# dummy diff --git a/storm-core/src/native/worker-launcher/Makefile.am b/storm-core/src/native/worker-launcher/Makefile.am new file mode 100644 index 00000000000..c9183c05279 --- /dev/null +++ b/storm-core/src/native/worker-launcher/Makefile.am @@ -0,0 +1,32 @@ +# 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. + +AM_CFLAGS=-I$(srcdir)/impl -Wall -g -Werror + +# Define the programs that need to be built +bin_PROGRAMS = worker-launcher +check_PROGRAMS = test-worker-launcher + +TESTS = test-worker-launcher + +# Define the sources for the common files +common_SOURCES = impl/configuration.c impl/worker-launcher.c + +# Define the sources for the real executable +worker_launcher_SOURCES = $(common_SOURCES) impl/main.c + +# Define the sources for the test executable +test_worker_launcher_SOURCES = $(common_SOURCES) test/test-worker-launcher.c diff --git a/storm-core/src/native/worker-launcher/configure.ac b/storm-core/src/native/worker-launcher/configure.ac new file mode 100644 index 00000000000..ab1ef49b378 --- /dev/null +++ b/storm-core/src/native/worker-launcher/configure.ac @@ -0,0 +1,50 @@ +# 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. +# +# -*- Autoconf -*- +# Process this file with autoconf to produce a configure script. + +AC_PREREQ(2.59) +AC_INIT(worker-launcher, 1.0.0, storm-user@googlegroups.com) +AC_GNU_SOURCE + +AM_INIT_AUTOMAKE([subdir-objects foreign no-dist]) + +AC_CONFIG_SRCDIR([impl/worker-launcher.c]) +AC_CONFIG_FILES([Makefile]) + +AC_PREFIX_DEFAULT(`pwd`/../install) + +# Checks for programs. +AC_PROG_CC +AM_PROG_CC_C_O + +# Checks for libraries. + +# Checks for header files. +AC_LANG(C) +AC_CHECK_HEADERS([unistd.h]) + +# Checks for typedefs, structures, and compiler characteristics. +AC_HEADER_STDBOOL +AC_C_CONST +AC_TYPE_OFF_T +AC_TYPE_SIZE_T +AC_FUNC_STRERROR_R + +# Checks for library functions. +AC_CHECK_FUNCS([mkdir uname]) +AC_OUTPUT diff --git a/storm-core/src/native/worker-launcher/impl/configuration.c b/storm-core/src/native/worker-launcher/impl/configuration.c new file mode 100644 index 00000000000..7b7a3c1c3ac --- /dev/null +++ b/storm-core/src/native/worker-launcher/impl/configuration.c @@ -0,0 +1,340 @@ +/** + * 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. + */ + +// ensure we get the posix version of dirname by including this first +#include + +#include "configuration.h" +#include "worker-launcher.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#define MAX_SIZE 10 + +struct confentry { + const char *key; + const char *value; +}; + +struct configuration { + int size; + struct confentry **confdetails; +}; + +struct configuration config={.size=0, .confdetails=NULL}; + +//clean up method for freeing configuration +void free_configurations() { + int i = 0; + for (i = 0; i < config.size; i++) { + if (config.confdetails[i]->key != NULL) { + free((void *)config.confdetails[i]->key); + config.confdetails[i]->key = NULL; + } + if (config.confdetails[i]->value != NULL) { + free((void *)config.confdetails[i]->value); + config.confdetails[i]->value = NULL; + } + free(config.confdetails[i]); + config.confdetails[i] = NULL; + } + if (config.size > 0) { + free(config.confdetails); + config.confdetails = NULL; + } + config.size = 0; +} + +/** + * Is the file/directory only writable by root. + * Returns 1 if true + */ +static int is_only_root_writable(const char *file) { + struct stat file_stat; + if (stat(file, &file_stat) != 0) { + fprintf(ERRORFILE, "Can't stat file %s - %s\n", file, strerror(errno)); + return 0; + } + if (file_stat.st_uid != 0) { + fprintf(ERRORFILE, "File %s must be owned by root, but is owned by %d\n", + file, file_stat.st_uid); + return 0; + } + if ((file_stat.st_mode & (S_IWGRP | S_IWOTH)) != 0) { + fprintf(ERRORFILE, + "File %s must not be world or group writable, but is %03o\n", + file, file_stat.st_mode & (~S_IFMT)); + return 0; + } + return 1; +} + +/** + * Ensure that the configuration file and all of the containing directories + * are only writable by root. Otherwise, an attacker can change the + * configuration and potentially cause damage. + * returns 0 if permissions are ok + */ +int check_configuration_permissions(const char* file_name) { + // copy the input so that we can modify it with dirname + char* dir = strndup(file_name, PATH_MAX); + char* buffer = dir; + do { + if (!is_only_root_writable(dir)) { + free(buffer); + buffer = NULL; + return -1; + } + dir = dirname(dir); + } while (strcmp(dir, "/") != 0); + free(buffer); + buffer = NULL; + return 0; +} + +//function used to load the configurations present in the secure config +void read_config(const char* file_name) { + FILE *conf_file; + char *line; + char *equaltok; + char *temp_equaltok; + size_t linesize = 1000; + int size_read = 0; + + if (file_name == NULL) { + fprintf(ERRORFILE, "Null configuration filename passed in\n"); + exit(INVALID_CONFIG_FILE); + } + + #ifdef DEBUG + fprintf(LOGFILE, "read_config :Conf file name is : %s \n", file_name); + #endif + + //allocate space for ten configuration items. + config.confdetails = (struct confentry **) malloc(sizeof(struct confentry *) + * MAX_SIZE); + if (config.confdetails == NULL) { + fprintf(ERRORFILE, "malloc failed while reading configuration file.\n"); + exit(OUT_OF_MEMORY); + } + config.size = 0; + conf_file = fopen(file_name, "r"); + if (conf_file == NULL) { + fprintf(ERRORFILE, "Invalid conf file provided : %s \n", file_name); + exit(INVALID_CONFIG_FILE); + } + while(!feof(conf_file)) { + line = (char *) malloc(linesize); + if(line == NULL) { + fprintf(ERRORFILE, "malloc failed while reading configuration file.\n"); + exit(OUT_OF_MEMORY); + } + size_read = getline(&line,&linesize,conf_file); + + //feof returns true only after we read past EOF. + //so a file with no new line, at last can reach this place + //if size_read returns negative check for eof condition + if (size_read == -1) { + free(line); + line = NULL; + if(!feof(conf_file)){ + exit(INVALID_CONFIG_FILE); + } else { + break; + } + } + int eol = strlen(line) - 1; + if(line[eol] == '\n') { + //trim the ending new line + line[eol] = '\0'; + } + //comment line + if(line[0] == '#') { + free(line); + line = NULL; + continue; + } + //tokenize first to get key and list of values. + //if no equals is found ignore this line, can be an empty line also + equaltok = strtok_r(line, "=", &temp_equaltok); + if(equaltok == NULL) { + free(line); + line = NULL; + continue; + } + config.confdetails[config.size] = (struct confentry *) malloc( + sizeof(struct confentry)); + if(config.confdetails[config.size] == NULL) { + fprintf(LOGFILE, + "Failed allocating memory for single configuration item\n"); + goto cleanup; + } + + #ifdef DEBUG + fprintf(LOGFILE, "read_config : Adding conf key : %s \n", equaltok); + #endif + + memset(config.confdetails[config.size], 0, sizeof(struct confentry)); + const size_t key_tok_len = strlen(equaltok); + config.confdetails[config.size]->key = (char *) malloc( + sizeof(char) * (key_tok_len+1)); + if (config.confdetails[config.size]->key == NULL) { + fprintf(LOGFILE, + "Failed allocating memory for single configuration item\n"); + goto cleanup; + } + memset((void*)config.confdetails[config.size]->key, '\0', key_tok_len+1); + strncpy((char *)config.confdetails[config.size]->key, equaltok, key_tok_len); + equaltok = strtok_r(NULL, "=", &temp_equaltok); + if (equaltok == NULL) { + fprintf(LOGFILE, "configuration tokenization failed \n"); + goto cleanup; + } + //means value is commented so don't store the key + if(equaltok[0] == '#') { + free(line); + line = NULL; + free((void *)config.confdetails[config.size]->key); + config.confdetails[config.size]->key = NULL; + free(config.confdetails[config.size]); + config.confdetails[config.size] = NULL; + continue; + } + + #ifdef DEBUG + fprintf(LOGFILE, "read_config : Adding conf value : %s \n", equaltok); + #endif + + const size_t val_tok_len = strlen(equaltok); + config.confdetails[config.size]->value = (char *) malloc( + sizeof(char) * (val_tok_len+1)); + if (config.confdetails[config.size]->value == NULL) { + fprintf(LOGFILE, + "Failed allocating memory for single configuration item\n"); + goto cleanup; + } + memset((void *)config.confdetails[config.size]->value, '\0', val_tok_len+1); + strncpy((char *)config.confdetails[config.size]->value, equaltok, val_tok_len); + if((config.size + 1) % MAX_SIZE == 0) { + config.confdetails = (struct confentry **) realloc(config.confdetails, + sizeof(struct confentry **) * (MAX_SIZE + config.size)); + if (config.confdetails == NULL) { + fprintf(LOGFILE, + "Failed re-allocating memory for configuration items\n"); + goto cleanup; + } + } + if(config.confdetails[config.size] ) + config.size++; + free(line); + line = NULL; + } + + //close the file + fclose(conf_file); + + if (config.size == 0) { + fprintf(ERRORFILE, "Invalid configuration provided in %s\n", file_name); + exit(INVALID_CONFIG_FILE); + } + + //clean up allocated file name + return; + //free spaces alloced. + cleanup: + if (line != NULL) { + free(line); + line = NULL; + } + fclose(conf_file); + free_configurations(); + return; +} + +/* + * function used to get a configuration value. + * The function for the first time populates the configuration details into + * array, next time onwards used the populated array. + * + */ +char * get_value(const char* key) { + int count; + for (count = 0; count < config.size; count++) { + if (strcmp(config.confdetails[count]->key, key) == 0) { + return strdup(config.confdetails[count]->value); + } + } + return NULL; +} + +/** + * Function to return an array of values for a key. + * Value delimiter is assumed to be a comma. + */ +char ** get_values(const char * key) { + char *value = get_value(key); + return extract_values(value); +} + +/** + * Extracts array of values from the comma separated list of values. + */ +char ** extract_values(char *value) { + char ** toPass = NULL; + char *tempTok = NULL; + char *tempstr = NULL; + int size = 0; + int toPassSize = MAX_SIZE; + + //first allocate any array of 10 + if(value != NULL) { + toPass = (char **) malloc(sizeof(char *) * toPassSize); + tempTok = strtok_r((char *)value, ",", &tempstr); + while (tempTok != NULL) { + toPass[size++] = tempTok; + if(size == toPassSize) { + toPassSize += MAX_SIZE; + toPass = (char **) realloc(toPass,(sizeof(char *) * toPassSize)); + } + tempTok = strtok_r(NULL, ",", &tempstr); + } + } + if (size > 0) { + toPass[size] = NULL; + } + return toPass; +} + +// free an entry set of values +void free_values(char** values) { + if (*values != NULL) { + free(*values); + *values = NULL; + } + if (values != NULL) { + free(values); + values = NULL; + } +} diff --git a/storm-core/src/native/worker-launcher/impl/configuration.h b/storm-core/src/native/worker-launcher/impl/configuration.h new file mode 100644 index 00000000000..b0d4814b310 --- /dev/null +++ b/storm-core/src/native/worker-launcher/impl/configuration.h @@ -0,0 +1,45 @@ +/** + * 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. + */ + +/** + * Ensure that the configuration file and all of the containing directories + * are only writable by root. Otherwise, an attacker can change the + * configuration and potentially cause damage. + * returns 0 if permissions are ok + */ +int check_configuration_permissions(const char* file_name); + +// read the given configuration file +void read_config(const char* config_file); + +//method exposed to get the configurations +char *get_value(const char* key); + +//function to return array of values pointing to the key. Values are +//comma seperated strings. +char ** get_values(const char* key); + +// Extracts array of values from the comma separated list of values. +char ** extract_values(char *value); + +// free the memory returned by get_values +void free_values(char** values); + +//method to free allocated configuration +void free_configurations(); + diff --git a/storm-core/src/native/worker-launcher/impl/main.c b/storm-core/src/native/worker-launcher/impl/main.c new file mode 100644 index 00000000000..7067cf922cd --- /dev/null +++ b/storm-core/src/native/worker-launcher/impl/main.c @@ -0,0 +1,210 @@ +/** + * 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. + */ + +#include "configuration.h" +#include "worker-launcher.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#define _STRINGIFY(X) #X +#define STRINGIFY(X) _STRINGIFY(X) +#define CONF_FILENAME "worker-launcher.cfg" + +#ifndef EXEC_CONF_DIR + #error EXEC_CONF_DIR must be defined +#endif + +void display_usage(FILE *stream) { + fprintf(stream, + "Usage: worker-launcher --checksetup\n"); + fprintf(stream, + "Usage: worker-launcher user command command-args\n"); + fprintf(stream, "Commands:\n"); + fprintf(stream, " initialize stormdist dir: code-dir \n"); + fprintf(stream, " remove a file/directory: rmr \n"); + fprintf(stream, " launch a worker: worker \n"); + fprintf(stream, " signal a worker: signal \n"); +} + +int main(int argc, char **argv) { + int invalid_args = 0; + int do_check_setup = 0; + + LOGFILE = stdout; + ERRORFILE = stderr; + + // Minimum number of arguments required to run + // the std. worker-launcher commands is 3 + // 3 args not needed for checksetup option + if (argc < 3) { + invalid_args = 1; + if (argc == 2) { + const char *arg1 = argv[1]; + if (strcmp("--checksetup", arg1) == 0) { + invalid_args = 0; + do_check_setup = 1; + } + } + } + + if (invalid_args != 0) { + display_usage(stdout); + return INVALID_ARGUMENT_NUMBER; + } + + const char * command = NULL; + const char * working_dir = NULL; + + int exit_code = 0; + + char *executable_file = get_executable(); + + char *orig_conf_file = STRINGIFY(EXEC_CONF_DIR) "/" CONF_FILENAME; + char *conf_file = realpath(orig_conf_file, NULL); + + if (conf_file == NULL) { + fprintf(ERRORFILE, "Configuration file %s not found.\n", orig_conf_file); + exit(INVALID_CONFIG_FILE); + } + if (check_configuration_permissions(conf_file) != 0) { + exit(INVALID_CONFIG_FILE); + } + read_config(conf_file); + free(conf_file); + conf_file = NULL; + + // look up the node manager group in the config file + char *nm_group = get_value(LAUNCHER_GROUP_KEY); + if (nm_group == NULL) { + fprintf(ERRORFILE, "Can't get configured value for %s.\n", LAUNCHER_GROUP_KEY); + exit(INVALID_CONFIG_FILE); + } + struct group *group_info = getgrnam(nm_group); + if (group_info == NULL) { + fprintf(ERRORFILE, "Can't get group information for %s - %s.\n", nm_group, + strerror(errno)); + fflush(LOGFILE); + exit(INVALID_CONFIG_FILE); + } + + set_launcher_uid(getuid(), group_info->gr_gid); + // if we are running from a setuid executable, make the real uid root + setuid(0); + // set the real and effective group id to the node manager group + setgid(group_info->gr_gid); + + if (check_executor_permissions(executable_file) != 0) { + fprintf(ERRORFILE, "Invalid permissions on worker-launcher binary.\n"); + return INVALID_CONTAINER_EXEC_PERMISSIONS; + } + + if (do_check_setup != 0) { + // basic setup checks done + // verified configs available and valid + // verified executor permissions + return 0; + } + + //checks done for user name + if (argv[optind] == NULL) { + fprintf(ERRORFILE, "Invalid user name.\n"); + return INVALID_USER_NAME; + } + + int ret = set_user(argv[optind]); + if (ret != 0) { + return ret; + } + + optind = optind + 1; + command = argv[optind++]; + + fprintf(LOGFILE, "main : command provided %s\n",command); + fprintf(LOGFILE, "main : user is %s\n", user_detail->pw_name); + fflush(LOGFILE); + + if (strcasecmp("code-dir", command) == 0) { + if (argc != 4) { + fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 4) for code-dir\n", + argc); + fflush(ERRORFILE); + return INVALID_ARGUMENT_NUMBER; + } + exit_code = setup_stormdist_dir(argv[optind]); + } else if (strcasecmp("rmr", command) == 0) { + if (argc != 4) { + fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 4) for rmr\n", + argc); + fflush(ERRORFILE); + return INVALID_ARGUMENT_NUMBER; + } + exit_code= delete_as_user(user_detail->pw_name, argv[optind], + NULL); + } else if (strcasecmp("worker", command) == 0) { + if (argc != 5) { + fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 5) for worker\n", + argc); + fflush(ERRORFILE); + return INVALID_ARGUMENT_NUMBER; + } + working_dir = argv[optind++]; + exit_code = setup_stormdist_dir(working_dir); + if (exit_code == 0) { + exit_code = exec_as_user(working_dir, argv[optind]); + } + } else if (strcasecmp("signal", command) == 0) { + if (argc != 5) { + fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 5) for signal\n", + argc); + fflush(ERRORFILE); + return INVALID_ARGUMENT_NUMBER; + } + char* end_ptr = NULL; + char* option = argv[optind++]; + int container_pid = strtol(option, &end_ptr, 10); + if (option == end_ptr || *end_ptr != '\0') { + fprintf(ERRORFILE, "Illegal argument for container pid %s\n", option); + fflush(ERRORFILE); + return INVALID_ARGUMENT_NUMBER; + } + option = argv[optind++]; + int signal = strtol(option, &end_ptr, 10); + if (option == end_ptr || *end_ptr != '\0') { + fprintf(ERRORFILE, "Illegal argument for signal %s\n", option); + fflush(ERRORFILE); + return INVALID_ARGUMENT_NUMBER; + } + exit_code = signal_container_as_user(user_detail->pw_name, container_pid, signal); + } else { + fprintf(ERRORFILE, "Invalid command %s not supported.",command); + fflush(ERRORFILE); + exit_code = INVALID_COMMAND_PROVIDED; + } + fclose(LOGFILE); + fclose(ERRORFILE); + return exit_code; +} diff --git a/storm-core/src/native/worker-launcher/impl/worker-launcher.c b/storm-core/src/native/worker-launcher/impl/worker-launcher.c new file mode 100644 index 00000000000..81d7075f76d --- /dev/null +++ b/storm-core/src/native/worker-launcher/impl/worker-launcher.c @@ -0,0 +1,779 @@ +/** + * 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. + */ + +#include "configuration.h" +#include "worker-launcher.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +static const int DEFAULT_MIN_USERID = 1000; + +static const char* DEFAULT_BANNED_USERS[] = {"bin", 0}; + +//struct to store the user details +struct passwd *user_detail = NULL; + +FILE* LOGFILE = NULL; +FILE* ERRORFILE = NULL; + +static uid_t launcher_uid = -1; +static gid_t launcher_gid = -1; + +char *concatenate(char *concat_pattern, char *return_path_name, + int numArgs, ...); + +void set_launcher_uid(uid_t user, gid_t group) { + launcher_uid = user; + launcher_gid = group; +} + +/** + * get the executable filename. + */ +char* get_executable() { + char buffer[PATH_MAX]; + snprintf(buffer, PATH_MAX, "/proc/%u/exe", getpid()); + char *filename = malloc(PATH_MAX); + if (NULL == filename) { + fprintf(ERRORFILE, "malloc failed in get_executable\n"); + exit(-1); + } + ssize_t len = readlink(buffer, filename, PATH_MAX); + if (len == -1) { + fprintf(ERRORFILE, "Can't get executable name from %s - %s\n", buffer, + strerror(errno)); + exit(-1); + } else if (len >= PATH_MAX) { + fprintf(ERRORFILE, "Executable name %.*s is longer than %d characters.\n", + PATH_MAX, filename, PATH_MAX); + exit(-1); + } + filename[len] = '\0'; + return filename; +} + +int check_executor_permissions(char *executable_file) { + errno = 0; + char * resolved_path = realpath(executable_file, NULL); + if (resolved_path == NULL) { + fprintf(ERRORFILE, + "Error resolving the canonical name for the executable : %s!", + strerror(errno)); + return -1; + } + + struct stat filestat; + errno = 0; + if (stat(resolved_path, &filestat) != 0) { + fprintf(ERRORFILE, + "Could not stat the executable : %s!.\n", strerror(errno)); + return -1; + } + + uid_t binary_euid = filestat.st_uid; // Binary's user owner + gid_t binary_gid = filestat.st_gid; // Binary's group owner + + // Effective uid should be root + if (binary_euid != 0) { + fprintf(LOGFILE, + "The worker-launcher binary should be user-owned by root.\n"); + return -1; + } + + if (binary_gid != getgid()) { + fprintf(LOGFILE, "The configured nodemanager group %d is different from" + " the group of the executable %d\n", getgid(), binary_gid); + return -1; + } + + // check others do not have read/write/execute permissions + if ((filestat.st_mode & S_IROTH) == S_IROTH || (filestat.st_mode & S_IWOTH) + == S_IWOTH || (filestat.st_mode & S_IXOTH) == S_IXOTH) { + fprintf(LOGFILE, + "The worker-launcher binary should not have read or write or" + " execute for others.\n"); + return -1; + } + + // Binary should be setuid/setgid executable + if ((filestat.st_mode & S_ISUID) == 0) { + fprintf(LOGFILE, "The worker-launcher binary should be set setuid.\n"); + return -1; + } + + return 0; +} + +/** + * Change the effective user id to limit damage. + */ +static int change_effective_user(uid_t user, gid_t group) { + if (geteuid() == user && getegid() == group) { + return 0; + } + if (seteuid(0) != 0) { + return -1; + } + if (setegid(group) != 0) { + fprintf(LOGFILE, "Failed to set effective group id %d - %s\n", group, + strerror(errno)); + return -1; + } + if (seteuid(user) != 0) { + fprintf(LOGFILE, "Failed to set effective user id %d - %s\n", user, + strerror(errno)); + return -1; + } + return 0; +} + +/** + * Change the real and effective user and group to abandon the super user + * priviledges. + */ +int change_user(uid_t user, gid_t group) { + if (user == getuid() && user == geteuid() && + group == getgid() && group == getegid()) { + return 0; + } + + if (seteuid(0) != 0) { + fprintf(LOGFILE, "unable to reacquire root - %s\n", strerror(errno)); + fprintf(LOGFILE, "Real: %d:%d; Effective: %d:%d\n", + getuid(), getgid(), geteuid(), getegid()); + return SETUID_OPER_FAILED; + } + if (setgid(group) != 0) { + fprintf(LOGFILE, "unable to set group to %d - %s\n", group, + strerror(errno)); + fprintf(LOGFILE, "Real: %d:%d; Effective: %d:%d\n", + getuid(), getgid(), geteuid(), getegid()); + return SETUID_OPER_FAILED; + } + if (setuid(user) != 0) { + fprintf(LOGFILE, "unable to set user to %d - %s\n", user, strerror(errno)); + fprintf(LOGFILE, "Real: %d:%d; Effective: %d:%d\n", + getuid(), getgid(), geteuid(), getegid()); + return SETUID_OPER_FAILED; + } + + return 0; +} + +/** + * Utility function to concatenate argB to argA using the concat_pattern. + */ +char *concatenate(char *concat_pattern, char *return_path_name, + int numArgs, ...) { + va_list ap; + va_start(ap, numArgs); + int strlen_args = 0; + char *arg = NULL; + int j; + for (j = 0; j < numArgs; j++) { + arg = va_arg(ap, char*); + if (arg == NULL) { + fprintf(LOGFILE, "One of the arguments passed for %s in null.\n", + return_path_name); + return NULL; + } + strlen_args += strlen(arg); + } + va_end(ap); + + char *return_path = NULL; + int str_len = strlen(concat_pattern) + strlen_args + 1; + + return_path = (char *) malloc(str_len); + if (return_path == NULL) { + fprintf(LOGFILE, "Unable to allocate memory for %s.\n", return_path_name); + return NULL; + } + va_start(ap, numArgs); + vsnprintf(return_path, str_len, concat_pattern, ap); + va_end(ap); + return return_path; +} + +char *get_container_launcher_file(const char* work_dir) { + return concatenate("%s/%s", "container launcher", 2, work_dir, CONTAINER_SCRIPT); +} + +/** + * Get the tmp directory under the working directory + */ +char *get_tmp_directory(const char *work_dir) { + return concatenate("%s/%s", "tmp dir", 2, work_dir, TMP_DIR); +} + +/** + * Load the user information for a given user name. + */ +static struct passwd* get_user_info(const char* user) { + int string_size = sysconf(_SC_GETPW_R_SIZE_MAX); + void* buffer = malloc(string_size + sizeof(struct passwd)); + if (buffer == NULL) { + fprintf(LOGFILE, "Malloc failed in get_user_info\n"); + return NULL; + } + struct passwd *result = NULL; + if (getpwnam_r(user, buffer, buffer + sizeof(struct passwd), string_size, + &result) != 0) { + free(buffer); + buffer = NULL; + fprintf(LOGFILE, "Can't get user information %s - %s\n", user, + strerror(errno)); + return NULL; + } + return result; +} + +/** + * Is the user a real user account? + * Checks: + * 1. Not root + * 2. UID is above the minimum configured. + * 3. Not in banned user list + * Returns NULL on failure + */ +struct passwd* check_user(const char *user) { + if (strcmp(user, "root") == 0) { + fprintf(LOGFILE, "Running as root is not allowed\n"); + fflush(LOGFILE); + return NULL; + } + char *min_uid_str = get_value(MIN_USERID_KEY); + int min_uid = DEFAULT_MIN_USERID; + if (min_uid_str != NULL) { + char *end_ptr = NULL; + min_uid = strtol(min_uid_str, &end_ptr, 10); + if (min_uid_str == end_ptr || *end_ptr != '\0') { + fprintf(LOGFILE, "Illegal value of %s for %s in configuration\n", + min_uid_str, MIN_USERID_KEY); + fflush(LOGFILE); + free(min_uid_str); + min_uid_str = NULL; + return NULL; + } + free(min_uid_str); + min_uid_str = NULL; + } + struct passwd *user_info = get_user_info(user); + if (NULL == user_info) { + fprintf(LOGFILE, "User %s not found\n", user); + fflush(LOGFILE); + return NULL; + } + if (user_info->pw_uid < min_uid) { + fprintf(LOGFILE, "Requested user %s has id %d, which is below the " + "minimum allowed %d\n", user, user_info->pw_uid, min_uid); + fflush(LOGFILE); + free(user_info); + user_info = NULL; + return NULL; + } + char **banned_users = get_values(BANNED_USERS_KEY); + char **banned_user = (banned_users == NULL) ? + (char**) DEFAULT_BANNED_USERS : banned_users; + for(; *banned_user; ++banned_user) { + if (strcmp(*banned_user, user) == 0) { + free(user_info); + user_info = NULL; + if (banned_users != (char**)DEFAULT_BANNED_USERS) { + free_values(banned_users); + banned_users = NULL; + } + fprintf(LOGFILE, "Requested user %s is banned\n", user); + return NULL; + } + } + if (banned_users != NULL && banned_users != (char**)DEFAULT_BANNED_USERS) { + free_values(banned_users); + banned_users = NULL; + } + return user_info; +} + +/** + * function used to populate and user_details structure. + */ +int set_user(const char *user) { + // free any old user + if (user_detail != NULL) { + free(user_detail); + user_detail = NULL; + } + user_detail = check_user(user); + if (user_detail == NULL) { + return -1; + } + + if (geteuid() == user_detail->pw_uid) { + return 0; + } + + if (initgroups(user, user_detail->pw_gid) != 0) { + fprintf(LOGFILE, "Error setting supplementary groups for user %s: %s\n", + user, strerror(errno)); + return -1; + } + + return change_effective_user(user_detail->pw_uid, user_detail->pw_gid); +} + +/** + * Open a file as the node manager and return a file descriptor for it. + * Returns -1 on error + */ +static int open_file_as_nm(const char* filename) { + uid_t user = geteuid(); + gid_t group = getegid(); + if (change_effective_user(launcher_uid, launcher_gid) != 0) { + return -1; + } + int result = open(filename, O_RDONLY); + if (result == -1) { + fprintf(LOGFILE, "Can't open file %s as node manager - %s\n", filename, + strerror(errno)); + } + if (change_effective_user(user, group)) { + result = -1; + } + return result; +} + +/** + * Copy a file from a fd to a given filename. + * The new file must not exist and it is created with permissions perm. + * The input stream is closed. + * Return 0 if everything is ok. + */ +static int copy_file(int input, const char* in_filename, + const char* out_filename, mode_t perm) { + const int buffer_size = 128*1024; + char buffer[buffer_size]; + int out_fd = open(out_filename, O_WRONLY|O_CREAT|O_EXCL|O_NOFOLLOW, perm); + if (out_fd == -1) { + fprintf(LOGFILE, "Can't open %s for output - %s\n", out_filename, + strerror(errno)); + return -1; + } + ssize_t len = read(input, buffer, buffer_size); + while (len > 0) { + ssize_t pos = 0; + while (pos < len) { + ssize_t write_result = write(out_fd, buffer + pos, len - pos); + if (write_result <= 0) { + fprintf(LOGFILE, "Error writing to %s - %s\n", out_filename, + strerror(errno)); + close(out_fd); + return -1; + } + pos += write_result; + } + len = read(input, buffer, buffer_size); + } + if (len < 0) { + fprintf(LOGFILE, "Failed to read file %s - %s\n", in_filename, + strerror(errno)); + close(out_fd); + return -1; + } + if (close(out_fd) != 0) { + fprintf(LOGFILE, "Failed to close file %s - %s\n", out_filename, + strerror(errno)); + return -1; + } + close(input); + return 0; +} + +int setup_stormdist(FTSENT* entry, uid_t euser) { + if (lchown(entry->fts_path, euser, launcher_gid) != 0) { + fprintf(ERRORFILE, "Failure to exec app initialization process - %s\n", + strerror(errno)); + return -1; + } + mode_t mode = entry->fts_statp->st_mode; + mode_t new_mode = (mode & (S_IRWXU)) | S_IRGRP | S_IWGRP; + if ((mode & S_IXUSR) == S_IXUSR) { + new_mode = new_mode | S_IXGRP; + } + if ((mode & S_IFDIR) == S_IFDIR) { + new_mode = new_mode | S_ISGID; + } + if (chmod(entry->fts_path, new_mode) != 0) { + fprintf(ERRORFILE, "Failure to exec app initialization process - %s\n", + strerror(errno)); + return -1; + } + return 0; +} + +int setup_stormdist_dir(const char* local_dir) { + //This is the same as + //> chmod g+rwX -R $local_dir + //> chown -no-dereference -R $user:$supervisor-group $local_dir + + int exit_code = 0; + uid_t euser = geteuid(); + + if (local_dir == NULL) { + fprintf(ERRORFILE, "Path is null\n"); + exit_code = UNABLE_TO_BUILD_PATH; // may be malloc failed + } else { + char *(paths[]) = {strndup(local_dir,PATH_MAX), 0}; + if (paths[0] == NULL) { + fprintf(ERRORFILE, "Malloc failed in setup_stormdist_dir\n"); + return -1; + } + // check to make sure the directory exists + if (access(local_dir, F_OK) != 0) { + if (errno == ENOENT) { + fprintf(ERRORFILE, "Path does not exist %s\n", local_dir); + free(paths[0]); + paths[0] = NULL; + return UNABLE_TO_BUILD_PATH; + } + } + FTS* tree = fts_open(paths, FTS_PHYSICAL | FTS_XDEV, NULL); + FTSENT* entry = NULL; + int ret = 0; + + if (tree == NULL) { + fprintf(ERRORFILE, + "Cannot open file traversal structure for the path %s:%s.\n", + local_dir, strerror(errno)); + free(paths[0]); + paths[0] = NULL; + return -1; + } + + if (seteuid(0) != 0) { + fprintf(ERRORFILE, "Could not become root\n"); + return -1; + } + + while (((entry = fts_read(tree)) != NULL) && exit_code == 0) { + switch (entry->fts_info) { + + case FTS_DP: // A directory being visited in post-order + case FTS_DOT: // A dot directory + //NOOP + fprintf(LOGFILE, "NOOP: %s\n", entry->fts_path); break; + case FTS_D: // A directory in pre-order + case FTS_F: // A regular file + case FTS_SL: // A symbolic link + case FTS_SLNONE: // A broken symbolic link + //TODO it would be good to validate that the file is owned by the correct user first. + fprintf(LOGFILE, "visiting: %s\n", entry->fts_path); + if (setup_stormdist(entry, euser) != 0) { + exit_code = -1; + } + break; + case FTS_DEFAULT: // Unknown type of file + case FTS_DNR: // Unreadable directory + case FTS_NS: // A file with no stat(2) information + case FTS_DC: // A directory that causes a cycle + case FTS_NSOK: // No stat information requested + case FTS_ERR: // Error return + default: + fprintf(LOGFILE, "Unexpected...\n"); break; + exit_code = -1; + break; + } + } + ret = fts_close(tree); + free(paths[0]); + paths[0] = NULL; + } + return exit_code; +} + + +int signal_container_as_user(const char *user, int pid, int sig) { + if(pid <= 0) { + return INVALID_CONTAINER_PID; + } + + if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) { + return SETUID_OPER_FAILED; + } + + //Don't continue if the process-group is not alive anymore. + int has_group = 1; + if (kill(-pid,0) < 0) { + if (kill(pid, 0) < 0) { + if (errno == ESRCH) { + return INVALID_CONTAINER_PID; + } + fprintf(LOGFILE, "Error signalling container %d with %d - %s\n", + pid, sig, strerror(errno)); + return -1; + } else { + has_group = 0; + } + } + + if (kill((has_group ? -1 : 1) * pid, sig) < 0) { + if(errno != ESRCH) { + fprintf(LOGFILE, + "Error signalling process group %d with signal %d - %s\n", + -pid, sig, strerror(errno)); + fprintf(stderr, + "Error signalling process group %d with signal %d - %s\n", + -pid, sig, strerror(errno)); + fflush(LOGFILE); + return UNABLE_TO_SIGNAL_CONTAINER; + } else { + return INVALID_CONTAINER_PID; + } + } + fprintf(LOGFILE, "Killing process %s%d with %d\n", + (has_group ? "group " :""), pid, sig); + return 0; +} + +/** + * Delete a final directory as the node manager user. + */ +static int rmdir_as_nm(const char* path) { + int user_uid = geteuid(); + int user_gid = getegid(); + int ret = change_effective_user(launcher_uid, launcher_gid); + if (ret == 0) { + if (rmdir(path) != 0) { + fprintf(LOGFILE, "rmdir of %s failed - %s\n", path, strerror(errno)); + ret = -1; + } + } + // always change back + if (change_effective_user(user_uid, user_gid) != 0) { + ret = -1; + } + return ret; +} + +/** + * Recursively delete the given path. + * full_path : the path to delete + * needs_tt_user: the top level directory must be deleted by the tt user. + */ +static int delete_path(const char *full_path, + int needs_tt_user) { + int exit_code = 0; + + if (full_path == NULL) { + fprintf(LOGFILE, "Path is null\n"); + exit_code = UNABLE_TO_BUILD_PATH; // may be malloc failed + } else { + char *(paths[]) = {strndup(full_path,PATH_MAX), 0}; + if (paths[0] == NULL) { + fprintf(LOGFILE, "Malloc failed in delete_path\n"); + return -1; + } + // check to make sure the directory exists + if (access(full_path, F_OK) != 0) { + if (errno == ENOENT) { + free(paths[0]); + paths[0] = NULL; + return 0; + } + } + FTS* tree = fts_open(paths, FTS_PHYSICAL | FTS_XDEV, NULL); + FTSENT* entry = NULL; + int ret = 0; + + if (tree == NULL) { + fprintf(LOGFILE, + "Cannot open file traversal structure for the path %s:%s.\n", + full_path, strerror(errno)); + free(paths[0]); + paths[0] = NULL; + return -1; + } + while (((entry = fts_read(tree)) != NULL) && exit_code == 0) { + switch (entry->fts_info) { + + case FTS_DP: // A directory being visited in post-order + if (!needs_tt_user || + strcmp(entry->fts_path, full_path) != 0) { + if (rmdir(entry->fts_accpath) != 0) { + fprintf(LOGFILE, "Couldn't delete directory %s - %s\n", + entry->fts_path, strerror(errno)); + exit_code = -1; + } + } + break; + + case FTS_F: // A regular file + case FTS_SL: // A symbolic link + case FTS_SLNONE: // A broken symbolic link + case FTS_DEFAULT: // Unknown type of file + if (unlink(entry->fts_accpath) != 0) { + fprintf(LOGFILE, "Couldn't delete file %s - %s\n", entry->fts_path, + strerror(errno)); + exit_code = -1; + } + break; + + case FTS_DNR: // Unreadable directory + fprintf(LOGFILE, "Unreadable directory %s. Skipping..\n", + entry->fts_path); + break; + + case FTS_D: // A directory in pre-order + // if the directory isn't readable, chmod it + if ((entry->fts_statp->st_mode & 0200) == 0) { + fprintf(LOGFILE, "Unreadable directory %s, chmoding.\n", + entry->fts_path); + if (chmod(entry->fts_accpath, 0700) != 0) { + fprintf(LOGFILE, "Error chmoding %s - %s, continuing\n", + entry->fts_path, strerror(errno)); + } + } + break; + + case FTS_NS: // A file with no stat(2) information + // usually a root directory that doesn't exist + fprintf(LOGFILE, "Directory not found %s\n", entry->fts_path); + break; + + case FTS_DC: // A directory that causes a cycle + case FTS_DOT: // A dot directory + case FTS_NSOK: // No stat information requested + break; + + case FTS_ERR: // Error return + fprintf(LOGFILE, "Error traversing directory %s - %s\n", + entry->fts_path, strerror(entry->fts_errno)); + exit_code = -1; + break; + break; + default: + exit_code = -1; + break; + } + } + ret = fts_close(tree); + if (exit_code == 0 && ret != 0) { + fprintf(LOGFILE, "Error in fts_close while deleting %s\n", full_path); + exit_code = -1; + } + if (needs_tt_user) { + // If the delete failed, try a final rmdir as root on the top level. + // That handles the case where the top level directory is in a directory + // that is owned by the node manager. + exit_code = rmdir_as_nm(full_path); + } + free(paths[0]); + paths[0] = NULL; + } + return exit_code; +} + +int exec_as_user(const char * working_dir, const char * script_file) { + char *script_file_dest = NULL; + script_file_dest = get_container_launcher_file(working_dir); + if (script_file_dest == NULL) { + return OUT_OF_MEMORY; + } + + // open launch script + int script_file_source = open_file_as_nm(script_file); + if (script_file_source == -1) { + return -1; + } + + setsid(); + + // give up root privs + if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) { + return SETUID_OPER_FAILED; + } + + if (copy_file(script_file_source, script_file, script_file_dest, S_IRWXU) != 0) { + return -1; + } + + fcloseall(); + umask(0027); + if (chdir(working_dir) != 0) { + fprintf(LOGFILE, "Can't change directory to %s -%s\n", working_dir, + strerror(errno)); + return -1; + } + + if (execlp(script_file_dest, script_file_dest, NULL) != 0) { + fprintf(LOGFILE, "Couldn't execute the container launch file %s - %s", + script_file_dest, strerror(errno)); + return UNABLE_TO_EXECUTE_CONTAINER_SCRIPT; + } + + //Unreachable + return -1; +} + +/** + * Delete the given directory as the user from each of the directories + * user: the user doing the delete + * subdir: the subdir to delete (if baseDirs is empty, this is treated as + an absolute path) + * baseDirs: (optional) the baseDirs where the subdir is located + */ +int delete_as_user(const char *user, + const char *subdir, + char* const* baseDirs) { + int ret = 0; + + char** ptr; + + // TODO: No switching user? !!!! + if (baseDirs == NULL || *baseDirs == NULL) { + return delete_path(subdir, 1); + } + // do the delete + for(ptr = (char**)baseDirs; *ptr != NULL; ++ptr) { + char* full_path = concatenate("%s/%s", "user subdir", 2, + *ptr, subdir); + if (full_path == NULL) { + return -1; + } + int this_ret = delete_path(full_path, strlen(subdir) == 0); + free(full_path); + full_path = NULL; + // delete as much as we can, but remember the error + if (this_ret != 0) { + ret = this_ret; + } + } + return ret; +} + + diff --git a/storm-core/src/native/worker-launcher/impl/worker-launcher.h b/storm-core/src/native/worker-launcher/impl/worker-launcher.h new file mode 100644 index 00000000000..59ab998b791 --- /dev/null +++ b/storm-core/src/native/worker-launcher/impl/worker-launcher.h @@ -0,0 +1,129 @@ +/** + * 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. + */ +#include +#include +#include + +enum errorcodes { + INVALID_ARGUMENT_NUMBER = 1, + INVALID_USER_NAME, //2 + INVALID_COMMAND_PROVIDED, //3 + // SUPER_USER_NOT_ALLOWED_TO_RUN_TASKS (NOT USED) 4 + INVALID_NM_ROOT_DIRS = 5, + SETUID_OPER_FAILED, //6 + UNABLE_TO_EXECUTE_CONTAINER_SCRIPT, //7 + UNABLE_TO_SIGNAL_CONTAINER, //8 + INVALID_CONTAINER_PID, //9 + // ERROR_RESOLVING_FILE_PATH (NOT_USED) 10 + // RELATIVE_PATH_COMPONENTS_IN_FILE_PATH (NOT USED) 11 + // UNABLE_TO_STAT_FILE (NOT USED) 12 + // FILE_NOT_OWNED_BY_ROOT (NOT USED) 13 + // PREPARE_CONTAINER_DIRECTORIES_FAILED (NOT USED) 14 + // INITIALIZE_CONTAINER_FAILED (NOT USED) 15 + // PREPARE_CONTAINER_LOGS_FAILED (NOT USED) 16 + // INVALID_LOG_DIR (NOT USED) 17 + OUT_OF_MEMORY = 18, + // INITIALIZE_DISTCACHEFILE_FAILED (NOT USED) 19 + INITIALIZE_USER_FAILED = 20, + UNABLE_TO_BUILD_PATH, //21 + INVALID_CONTAINER_EXEC_PERMISSIONS, //22 + // PREPARE_JOB_LOGS_FAILED (NOT USED) 23 + INVALID_CONFIG_FILE = 24, + SETSID_OPER_FAILED = 25, + WRITE_PIDFILE_FAILED = 26 +}; + +#define LAUNCHER_GROUP_KEY "storm.worker-launcher.group" + +#define USER_DIR_PATTERN "%s/usercache/%s" +#define NM_APP_DIR_PATTERN USER_DIR_PATTERN "/appcache/%s" +#define CONTAINER_DIR_PATTERN NM_APP_DIR_PATTERN "/%s" +#define CONTAINER_SCRIPT "launch_container.sh" +#define CREDENTIALS_FILENAME "container_tokens" +#define MIN_USERID_KEY "min.user.id" +#define BANNED_USERS_KEY "banned.users" +#define TMP_DIR "tmp" + +extern struct passwd *user_detail; + +// the log file for messages +extern FILE *LOGFILE; +// the log file for error messages +extern FILE *ERRORFILE; + +int setup_stormdist_dir(const char* local_dir); + +int exec_as_user(const char * working_dir, const char * args); + +// delete a directory (or file) recursively as the user. The directory +// could optionally be relative to the baseDir set of directories (if the same +// directory appears on multiple disk volumes, the disk volumes should be passed +// as the baseDirs). If baseDirs is not specified, then dir_to_be_deleted is +// assumed as the absolute path +int delete_as_user(const char *user, + const char *dir_to_be_deleted, + char* const* baseDirs); + +// get the executable's filename +char* get_executable(); + +/** + * Check the permissions on the worker-launcher to make sure that security is + * permissible. For this, we need worker-launcher binary to + * * be user-owned by root + * * be group-owned by a configured special group. + * * others do not have any permissions + * * be setuid/setgid + * @param executable_file the file to check + * @return -1 on error 0 on success. + */ +int check_executor_permissions(char *executable_file); + +/** + * Function used to signal a container launched by the user. + * The function sends appropriate signal to the process group + * specified by the pid. + * @param user the user to send the signal as. + * @param pid the process id to send the signal to. + * @param sig the signal to send. + * @return an errorcode enum value on error, or 0 on success. + */ +int signal_container_as_user(const char *user, int pid, int sig); + +// set the uid and gid of the launcher. This is used when doing some +// priviledged operations for setting the effective uid and gid. +void set_launcher_uid(uid_t user, gid_t group); + +/** + * Is the user a real user account? + * Checks: + * 1. Not root + * 2. UID is above the minimum configured. + * 3. Not in banned user list + * Returns NULL on failure + */ +struct passwd* check_user(const char *user); + +// set the user +int set_user(const char *user); + +// methods to get the directories + +char *get_container_launcher_file(const char* work_dir); + +int change_user(uid_t user, gid_t group); diff --git a/storm-core/src/native/worker-launcher/test/test-worker-launcher.c b/storm-core/src/native/worker-launcher/test/test-worker-launcher.c new file mode 100644 index 00000000000..412e9228105 --- /dev/null +++ b/storm-core/src/native/worker-launcher/test/test-worker-launcher.c @@ -0,0 +1,340 @@ +/** + * 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. + */ +#include "configuration.h" +#include "worker-launcher.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#define TEST_ROOT "/tmp/test-worker-launcher" +#define DONT_TOUCH_FILE "dont-touch-me" +#define NM_LOCAL_DIRS TEST_ROOT "/local-1," TEST_ROOT "/local-2," \ + TEST_ROOT "/local-3," TEST_ROOT "/local-4," TEST_ROOT "/local-5" +#define NM_LOG_DIRS TEST_ROOT "/logdir_1," TEST_ROOT "/logdir_2," \ + TEST_ROOT "/logdir_3," TEST_ROOT "/logdir_4" +#define ARRAY_SIZE 1000 + +static char* username = NULL; +static char* local_dirs = NULL; +static char* log_dirs = NULL; + +/** + * Run the command using the effective user id. + * It can't use system, since bash seems to copy the real user id into the + * effective id. + */ +void run(const char *cmd) { + fflush(stdout); + fflush(stderr); + pid_t child = fork(); + if (child == -1) { + printf("FAIL: failed to fork - %s\n", strerror(errno)); + } else if (child == 0) { + char *cmd_copy = strdup(cmd); + char *ptr; + int words = 1; + for(ptr = strchr(cmd_copy, ' '); ptr; ptr = strchr(ptr+1, ' ')) { + words += 1; + } + char **argv = malloc(sizeof(char *) * (words + 1)); + ptr = strtok(cmd_copy, " "); + int i = 0; + argv[i++] = ptr; + while (ptr != NULL) { + ptr = strtok(NULL, " "); + argv[i++] = ptr; + } + if (execvp(argv[0], argv) != 0) { + printf("FAIL: exec failed in child %s - %s\n", cmd, strerror(errno)); + exit(42); + } + } else { + int status = 0; + if (waitpid(child, &status, 0) <= 0) { + printf("FAIL: failed waiting for child process %s pid %d - %s\n", + cmd, child, strerror(errno)); + exit(1); + } + if (!WIFEXITED(status)) { + printf("FAIL: process %s pid %d did not exit\n", cmd, child); + exit(1); + } + if (WEXITSTATUS(status) != 0) { + printf("FAIL: process %s pid %d exited with error status %d\n", cmd, + child, WEXITSTATUS(status)); + exit(1); + } + } +} + +int write_config_file(char *file_name) { + FILE *file; + file = fopen(file_name, "w"); + if (file == NULL) { + printf("Failed to open %s.\n", file_name); + return EXIT_FAILURE; + } + fprintf(file, "banned.users=bannedUser\n"); + fprintf(file, "min.user.id=1000\n"); + fclose(file); + return 0; +} + +void create_nm_roots(char ** nm_roots) { + char** nm_root; + for(nm_root=nm_roots; *nm_root != NULL; ++nm_root) { + if (mkdir(*nm_root, 0755) != 0) { + printf("FAIL: Can't create directory %s - %s\n", *nm_root, + strerror(errno)); + exit(1); + } + char buffer[100000]; + sprintf(buffer, "%s/usercache", *nm_root); + if (mkdir(buffer, 0755) != 0) { + printf("FAIL: Can't create directory %s - %s\n", buffer, + strerror(errno)); + exit(1); + } + } +} + +void test_get_container_launcher_file() { + char *expected_file = ("/tmp/launch_container.sh"); + char *app_dir = "/tmp"; + char *container_file = get_container_launcher_file(app_dir); + if (strcmp(container_file, expected_file) != 0) { + printf("failure to match expected container file %s vs %s\n", container_file, + expected_file); + exit(1); + } + free(container_file); +} + +void test_check_user() { + printf("\nTesting test_check_user\n"); + struct passwd *user = check_user(username); + if (user == NULL) { + printf("FAIL: failed check for user %s\n", username); + exit(1); + } + free(user); + if (check_user("lp") != NULL) { + printf("FAIL: failed check for system user lp\n"); + exit(1); + } + if (check_user("root") != NULL) { + printf("FAIL: failed check for system user root\n"); + exit(1); + } +} + +void test_check_configuration_permissions() { + printf("\nTesting check_configuration_permissions\n"); + if (check_configuration_permissions("/etc/passwd") != 0) { + printf("FAIL: failed permission check on /etc/passwd\n"); + exit(1); + } + if (check_configuration_permissions(TEST_ROOT) == 0) { + printf("FAIL: failed permission check on %s\n", TEST_ROOT); + exit(1); + } +} + +void run_test_in_child(const char* test_name, void (*func)()) { + printf("\nRunning test %s in child process\n", test_name); + fflush(stdout); + fflush(stderr); + pid_t child = fork(); + if (child == -1) { + printf("FAIL: fork failed\n"); + exit(1); + } else if (child == 0) { + func(); + exit(0); + } else { + int status = 0; + if (waitpid(child, &status, 0) == -1) { + printf("FAIL: waitpid %d failed - %s\n", child, strerror(errno)); + exit(1); + } + if (!WIFEXITED(status)) { + printf("FAIL: child %d didn't exit - %d\n", child, status); + exit(1); + } + if (WEXITSTATUS(status) != 0) { + printf("FAIL: child %d exited with bad status %d\n", + child, WEXITSTATUS(status)); + exit(1); + } + } +} + +void test_signal_container() { + printf("\nTesting signal_container\n"); + fflush(stdout); + fflush(stderr); + pid_t child = fork(); + if (child == -1) { + printf("FAIL: fork failed\n"); + exit(1); + } else if (child == 0) { + if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) { + exit(1); + } + sleep(3600); + exit(0); + } else { + printf("Child container launched as %d\n", child); + if (signal_container_as_user(username, child, SIGQUIT) != 0) { + exit(1); + } + int status = 0; + if (waitpid(child, &status, 0) == -1) { + printf("FAIL: waitpid failed - %s\n", strerror(errno)); + exit(1); + } + if (!WIFSIGNALED(status)) { + printf("FAIL: child wasn't signalled - %d\n", status); + exit(1); + } + if (WTERMSIG(status) != SIGQUIT) { + printf("FAIL: child was killed with %d instead of %d\n", + WTERMSIG(status), SIGQUIT); + exit(1); + } + } +} + +void test_signal_container_group() { + printf("\nTesting group signal_container\n"); + fflush(stdout); + fflush(stderr); + pid_t child = fork(); + if (child == -1) { + printf("FAIL: fork failed\n"); + exit(1); + } else if (child == 0) { + setpgrp(); + if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) { + exit(1); + } + sleep(3600); + exit(0); + } + printf("Child container launched as %d\n", child); + if (signal_container_as_user(username, child, SIGKILL) != 0) { + exit(1); + } + int status = 0; + if (waitpid(child, &status, 0) == -1) { + printf("FAIL: waitpid failed - %s\n", strerror(errno)); + exit(1); + } + if (!WIFSIGNALED(status)) { + printf("FAIL: child wasn't signalled - %d\n", status); + exit(1); + } + if (WTERMSIG(status) != SIGKILL) { + printf("FAIL: child was killed with %d instead of %d\n", + WTERMSIG(status), SIGKILL); + exit(1); + } +} + +/** + * Ensure that the given path and all of the parent directories are created + * with the desired permissions. + */ +int mkdirs(const char* path, mode_t perm) { + char *cmd = malloc(10 + strlen(path)); + int ret = 0; + sprintf(cmd, "mkdir -p %s", path); + ret = system(cmd); + free(cmd); + return ret; +} + +int main(int argc, char **argv) { + LOGFILE = stdout; + ERRORFILE = stderr; + int my_username = 0; + + // clean up any junk from previous run + system("chmod -R u=rwx " TEST_ROOT "; rm -fr " TEST_ROOT); + + if (mkdirs(TEST_ROOT "/logs/userlogs", 0755) != 0) { + exit(1); + } + + if (write_config_file(TEST_ROOT "/test.cfg") != 0) { + exit(1); + } + read_config(TEST_ROOT "/test.cfg"); + + local_dirs = (char *) malloc (sizeof(char) * ARRAY_SIZE); + strcpy(local_dirs, NM_LOCAL_DIRS); + log_dirs = (char *) malloc (sizeof(char) * ARRAY_SIZE); + strcpy(log_dirs, NM_LOG_DIRS); + + create_nm_roots(extract_values(local_dirs)); + + if (getuid() == 0 && argc == 2) { + username = argv[1]; + } else { + username = strdup(getpwuid(getuid())->pw_name); + my_username = 1; + } + set_launcher_uid(geteuid(), getegid()); + + if (set_user(username)) { + exit(1); + } + + printf("\nStarting tests\n"); + + printf("\nTesting get_container_launcher_file()\n"); + test_get_container_launcher_file(); + + printf("\nTesting check_configuration_permissions()\n"); + test_check_configuration_permissions(); + + printf("\nTesting check_user()\n"); + test_check_user(); + + // the tests that change user need to be run in a subshell, so that + // when they change user they don't give up our privs + run_test_in_child("test_signal_container", test_signal_container); + run_test_in_child("test_signal_container_group", test_signal_container_group); + + seteuid(0); + run("rm -fr " TEST_ROOT); + printf("\nFinished tests\n"); + + if (my_username) { + free(username); + } + free_configurations(); + return 0; +} diff --git a/storm-core/src/py/storm/DistributedRPC.py b/storm-core/src/py/storm/DistributedRPC.py index a7e6ef9d217..851ad658672 100644 --- a/storm-core/src/py/storm/DistributedRPC.py +++ b/storm-core/src/py/storm/DistributedRPC.py @@ -64,6 +64,8 @@ def recv_execute(self, ): return result.success if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "execute failed: unknown result"); @@ -97,6 +99,8 @@ def process_execute(self, seqid, iprot, oprot): result.success = self._handler.execute(args.functionName, args.funcArgs) except DRPCExecutionException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("execute", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -185,19 +189,22 @@ class execute_result: Attributes: - success - e + - aze """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 (1, TType.STRUCT, 'e', (DRPCExecutionException, DRPCExecutionException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.success) + hash(self.e) + return 0 + hash(self.success) + hash(self.e) + hash(self.aze) - def __init__(self, success=None, e=None,): + def __init__(self, success=None, e=None, aze=None,): self.success = success self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -219,6 +226,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -237,6 +250,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() diff --git a/storm-core/src/py/storm/DistributedRPCInvocations.py b/storm-core/src/py/storm/DistributedRPCInvocations.py index 4f951a9432f..6de22457679 100644 --- a/storm-core/src/py/storm/DistributedRPCInvocations.py +++ b/storm-core/src/py/storm/DistributedRPCInvocations.py @@ -74,6 +74,8 @@ def recv_result(self, ): result = result_result() result.read(self._iprot) self._iprot.readMessageEnd() + if result.aze is not None: + raise result.aze return def fetchRequest(self, functionName): @@ -104,6 +106,8 @@ def recv_fetchRequest(self, ): self._iprot.readMessageEnd() if result.success is not None: return result.success + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result"); def failRequest(self, id): @@ -132,6 +136,8 @@ def recv_failRequest(self, ): result = failRequest_result() result.read(self._iprot) self._iprot.readMessageEnd() + if result.aze is not None: + raise result.aze return @@ -163,7 +169,10 @@ def process_result(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = result_result() - self._handler.result(args.id, args.result) + try: + self._handler.result(args.id, args.result) + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("result", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -174,7 +183,10 @@ def process_fetchRequest(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = fetchRequest_result() - result.success = self._handler.fetchRequest(args.functionName) + try: + result.success = self._handler.fetchRequest(args.functionName) + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("fetchRequest", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -185,7 +197,10 @@ def process_failRequest(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = failRequest_result() - self._handler.failRequest(args.id) + try: + self._handler.failRequest(args.id) + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("failRequest", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -270,12 +285,21 @@ def __ne__(self, other): return not (self == other) class result_result: + """ + Attributes: + - aze + """ thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + return 0 + hash(self.aze) + + def __init__(self, aze=None,): + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -286,6 +310,12 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break + if fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -296,6 +326,10 @@ def write(self, oprot): oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('result_result') + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -381,17 +415,20 @@ class fetchRequest_result: """ Attributes: - success + - aze """ thrift_spec = ( (0, TType.STRUCT, 'success', (DRPCRequest, DRPCRequest.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + hash(self.success) + return 0 + hash(self.success) + hash(self.aze) - def __init__(self, success=None,): + def __init__(self, success=None, aze=None,): self.success = success + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -408,6 +445,12 @@ def read(self, iprot): self.success.read(iprot) else: iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -422,6 +465,10 @@ def write(self, oprot): oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -504,12 +551,21 @@ def __ne__(self, other): return not (self == other) class failRequest_result: + """ + Attributes: + - aze + """ thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + return 0 + hash(self.aze) + + def __init__(self, aze=None,): + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -520,6 +576,12 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break + if fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -530,6 +592,10 @@ def write(self, oprot): oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('failRequest_result') + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() diff --git a/storm-core/src/py/storm/Nimbus-remote b/storm-core/src/py/storm/Nimbus-remote index 4b2ff041a3a..c184fab4b5d 100755 --- a/storm-core/src/py/storm/Nimbus-remote +++ b/storm-core/src/py/storm/Nimbus-remote @@ -28,6 +28,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help': print ' void activate(string name)' print ' void deactivate(string name)' print ' void rebalance(string name, RebalanceOptions options)' + print ' void uploadNewCredentials(string name, Credentials creds)' print ' string beginFileUpload()' print ' void uploadChunk(string location, string chunk)' print ' void finishFileUpload(string location)' @@ -131,6 +132,12 @@ elif cmd == 'rebalance': sys.exit(1) pp.pprint(client.rebalance(args[0],eval(args[1]),)) +elif cmd == 'uploadNewCredentials': + if len(args) != 2: + print 'uploadNewCredentials requires 2 args' + sys.exit(1) + pp.pprint(client.uploadNewCredentials(args[0],eval(args[1]),)) + elif cmd == 'beginFileUpload': if len(args) != 0: print 'beginFileUpload requires 0 args' diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py index cd535be73a7..e1ee5103dad 100644 --- a/storm-core/src/py/storm/Nimbus.py +++ b/storm-core/src/py/storm/Nimbus.py @@ -74,6 +74,14 @@ def rebalance(self, name, options): """ pass + def uploadNewCredentials(self, name, creds): + """ + Parameters: + - name + - creds + """ + pass + def beginFileUpload(self, ): pass @@ -184,6 +192,8 @@ def recv_submitTopology(self, ): raise result.e if result.ite is not None: raise result.ite + if result.aze is not None: + raise result.aze return def submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): @@ -224,6 +234,8 @@ def recv_submitTopologyWithOpts(self, ): raise result.e if result.ite is not None: raise result.ite + if result.aze is not None: + raise result.aze return def killTopology(self, name): @@ -254,6 +266,8 @@ def recv_killTopology(self, ): self._iprot.readMessageEnd() if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze return def killTopologyWithOpts(self, name, options): @@ -286,6 +300,8 @@ def recv_killTopologyWithOpts(self, ): self._iprot.readMessageEnd() if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze return def activate(self, name): @@ -316,6 +332,8 @@ def recv_activate(self, ): self._iprot.readMessageEnd() if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze return def deactivate(self, name): @@ -346,6 +364,8 @@ def recv_deactivate(self, ): self._iprot.readMessageEnd() if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze return def rebalance(self, name, options): @@ -380,6 +400,44 @@ def recv_rebalance(self, ): raise result.e if result.ite is not None: raise result.ite + if result.aze is not None: + raise result.aze + return + + def uploadNewCredentials(self, name, creds): + """ + Parameters: + - name + - creds + """ + self.send_uploadNewCredentials(name, creds) + self.recv_uploadNewCredentials() + + def send_uploadNewCredentials(self, name, creds): + self._oprot.writeMessageBegin('uploadNewCredentials', TMessageType.CALL, self._seqid) + args = uploadNewCredentials_args() + args.name = name + args.creds = creds + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_uploadNewCredentials(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = uploadNewCredentials_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.e is not None: + raise result.e + if result.ite is not None: + raise result.ite + if result.aze is not None: + raise result.aze return def beginFileUpload(self, ): @@ -405,6 +463,8 @@ def recv_beginFileUpload(self, ): self._iprot.readMessageEnd() if result.success is not None: return result.success + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result"); def uploadChunk(self, location, chunk): @@ -435,6 +495,8 @@ def recv_uploadChunk(self, ): result = uploadChunk_result() result.read(self._iprot) self._iprot.readMessageEnd() + if result.aze is not None: + raise result.aze return def finishFileUpload(self, location): @@ -463,6 +525,8 @@ def recv_finishFileUpload(self, ): result = finishFileUpload_result() result.read(self._iprot) self._iprot.readMessageEnd() + if result.aze is not None: + raise result.aze return def beginFileDownload(self, file): @@ -493,6 +557,8 @@ def recv_beginFileDownload(self, ): self._iprot.readMessageEnd() if result.success is not None: return result.success + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result"); def downloadChunk(self, id): @@ -523,6 +589,8 @@ def recv_downloadChunk(self, ): self._iprot.readMessageEnd() if result.success is not None: return result.success + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); def getNimbusConf(self, ): @@ -548,6 +616,8 @@ def recv_getNimbusConf(self, ): self._iprot.readMessageEnd() if result.success is not None: return result.success + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result"); def getClusterInfo(self, ): @@ -573,6 +643,8 @@ def recv_getClusterInfo(self, ): self._iprot.readMessageEnd() if result.success is not None: return result.success + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result"); def getTopologyInfo(self, id): @@ -605,6 +677,8 @@ def recv_getTopologyInfo(self, ): return result.success if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result"); def getTopologyConf(self, id): @@ -637,6 +711,8 @@ def recv_getTopologyConf(self, ): return result.success if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result"); def getTopology(self, id): @@ -669,6 +745,8 @@ def recv_getTopology(self, ): return result.success if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result"); def getUserTopology(self, id): @@ -701,6 +779,8 @@ def recv_getUserTopology(self, ): return result.success if result.e is not None: raise result.e + if result.aze is not None: + raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result"); @@ -715,6 +795,7 @@ def __init__(self, handler): self._processMap["activate"] = Processor.process_activate self._processMap["deactivate"] = Processor.process_deactivate self._processMap["rebalance"] = Processor.process_rebalance + self._processMap["uploadNewCredentials"] = Processor.process_uploadNewCredentials self._processMap["beginFileUpload"] = Processor.process_beginFileUpload self._processMap["uploadChunk"] = Processor.process_uploadChunk self._processMap["finishFileUpload"] = Processor.process_finishFileUpload @@ -753,6 +834,8 @@ def process_submitTopology(self, seqid, iprot, oprot): result.e = e except InvalidTopologyException, ite: result.ite = ite + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("submitTopology", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -769,6 +852,8 @@ def process_submitTopologyWithOpts(self, seqid, iprot, oprot): result.e = e except InvalidTopologyException, ite: result.ite = ite + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("submitTopologyWithOpts", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -783,6 +868,8 @@ def process_killTopology(self, seqid, iprot, oprot): self._handler.killTopology(args.name) except NotAliveException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("killTopology", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -797,6 +884,8 @@ def process_killTopologyWithOpts(self, seqid, iprot, oprot): self._handler.killTopologyWithOpts(args.name, args.options) except NotAliveException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("killTopologyWithOpts", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -811,6 +900,8 @@ def process_activate(self, seqid, iprot, oprot): self._handler.activate(args.name) except NotAliveException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("activate", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -825,6 +916,8 @@ def process_deactivate(self, seqid, iprot, oprot): self._handler.deactivate(args.name) except NotAliveException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("deactivate", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -841,17 +934,40 @@ def process_rebalance(self, seqid, iprot, oprot): result.e = e except InvalidTopologyException, ite: result.ite = ite + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("rebalance", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() + def process_uploadNewCredentials(self, seqid, iprot, oprot): + args = uploadNewCredentials_args() + args.read(iprot) + iprot.readMessageEnd() + result = uploadNewCredentials_result() + try: + self._handler.uploadNewCredentials(args.name, args.creds) + except NotAliveException, e: + result.e = e + except InvalidTopologyException, ite: + result.ite = ite + except AuthorizationException, aze: + result.aze = aze + oprot.writeMessageBegin("uploadNewCredentials", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + def process_beginFileUpload(self, seqid, iprot, oprot): args = beginFileUpload_args() args.read(iprot) iprot.readMessageEnd() result = beginFileUpload_result() - result.success = self._handler.beginFileUpload() + try: + result.success = self._handler.beginFileUpload() + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("beginFileUpload", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -862,7 +978,10 @@ def process_uploadChunk(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = uploadChunk_result() - self._handler.uploadChunk(args.location, args.chunk) + try: + self._handler.uploadChunk(args.location, args.chunk) + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("uploadChunk", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -873,7 +992,10 @@ def process_finishFileUpload(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = finishFileUpload_result() - self._handler.finishFileUpload(args.location) + try: + self._handler.finishFileUpload(args.location) + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("finishFileUpload", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -884,7 +1006,10 @@ def process_beginFileDownload(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = beginFileDownload_result() - result.success = self._handler.beginFileDownload(args.file) + try: + result.success = self._handler.beginFileDownload(args.file) + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("beginFileDownload", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -895,7 +1020,10 @@ def process_downloadChunk(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = downloadChunk_result() - result.success = self._handler.downloadChunk(args.id) + try: + result.success = self._handler.downloadChunk(args.id) + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("downloadChunk", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -906,7 +1034,10 @@ def process_getNimbusConf(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = getNimbusConf_result() - result.success = self._handler.getNimbusConf() + try: + result.success = self._handler.getNimbusConf() + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("getNimbusConf", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -917,7 +1048,10 @@ def process_getClusterInfo(self, seqid, iprot, oprot): args.read(iprot) iprot.readMessageEnd() result = getClusterInfo_result() - result.success = self._handler.getClusterInfo() + try: + result.success = self._handler.getClusterInfo() + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("getClusterInfo", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -932,6 +1066,8 @@ def process_getTopologyInfo(self, seqid, iprot, oprot): result.success = self._handler.getTopologyInfo(args.id) except NotAliveException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("getTopologyInfo", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -946,6 +1082,8 @@ def process_getTopologyConf(self, seqid, iprot, oprot): result.success = self._handler.getTopologyConf(args.id) except NotAliveException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("getTopologyConf", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -960,6 +1098,8 @@ def process_getTopology(self, seqid, iprot, oprot): result.success = self._handler.getTopology(args.id) except NotAliveException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("getTopology", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -974,6 +1114,8 @@ def process_getUserTopology(self, seqid, iprot, oprot): result.success = self._handler.getUserTopology(args.id) except NotAliveException, e: result.e = e + except AuthorizationException, aze: + result.aze = aze oprot.writeMessageBegin("getUserTopology", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() @@ -1087,20 +1229,23 @@ class submitTopology_result: Attributes: - e - ite + - aze """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 3 ) def __hash__(self): - return 0 + hash(self.e) + hash(self.ite) + return 0 + hash(self.e) + hash(self.ite) + hash(self.aze) - def __init__(self, e=None, ite=None,): + def __init__(self, e=None, ite=None, aze=None,): self.e = e self.ite = ite + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -1123,6 +1268,12 @@ def read(self, iprot): self.ite.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -1141,6 +1292,10 @@ def write(self, oprot): oprot.writeFieldBegin('ite', TType.STRUCT, 2) self.ite.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 3) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -1277,20 +1432,23 @@ class submitTopologyWithOpts_result: Attributes: - e - ite + - aze """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 3 ) def __hash__(self): - return 0 + hash(self.e) + hash(self.ite) + return 0 + hash(self.e) + hash(self.ite) + hash(self.aze) - def __init__(self, e=None, ite=None,): + def __init__(self, e=None, ite=None, aze=None,): self.e = e self.ite = ite + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -1313,6 +1471,12 @@ def read(self, iprot): self.ite.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -1331,6 +1495,10 @@ def write(self, oprot): oprot.writeFieldBegin('ite', TType.STRUCT, 2) self.ite.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 3) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -1416,18 +1584,21 @@ class killTopology_result: """ Attributes: - e + - aze """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.e) + return 0 + hash(self.e) + hash(self.aze) - def __init__(self, e=None,): + def __init__(self, e=None, aze=None,): self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -1444,6 +1615,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -1458,6 +1635,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -1556,18 +1737,21 @@ class killTopologyWithOpts_result: """ Attributes: - e + - aze """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.e) + return 0 + hash(self.e) + hash(self.aze) - def __init__(self, e=None,): + def __init__(self, e=None, aze=None,): self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -1584,6 +1768,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -1598,6 +1788,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -1683,18 +1877,21 @@ class activate_result: """ Attributes: - e + - aze """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.e) + return 0 + hash(self.e) + hash(self.aze) - def __init__(self, e=None,): + def __init__(self, e=None, aze=None,): self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -1711,6 +1908,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -1725,6 +1928,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -1810,18 +2017,21 @@ class deactivate_result: """ Attributes: - e + - aze """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.e) + return 0 + hash(self.e) + hash(self.aze) - def __init__(self, e=None,): + def __init__(self, e=None, aze=None,): self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -1838,6 +2048,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -1852,6 +2068,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -1951,20 +2171,23 @@ class rebalance_result: Attributes: - e - ite + - aze """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 3 ) def __hash__(self): - return 0 + hash(self.e) + hash(self.ite) + return 0 + hash(self.e) + hash(self.ite) + hash(self.aze) - def __init__(self, e=None, ite=None,): + def __init__(self, e=None, ite=None, aze=None,): self.e = e self.ite = ite + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -1987,6 +2210,12 @@ def read(self, iprot): self.ite.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2005,6 +2234,176 @@ def write(self, oprot): oprot.writeFieldBegin('ite', TType.STRUCT, 2) self.ite.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 3) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class uploadNewCredentials_args: + """ + Attributes: + - name + - creds + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRUCT, 'creds', (Credentials, Credentials.thrift_spec), None, ), # 2 + ) + + def __hash__(self): + return 0 + hash(self.name) + hash(self.creds) + + def __init__(self, name=None, creds=None,): + self.name = name + self.creds = creds + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.creds = Credentials() + self.creds.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('uploadNewCredentials_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.creds is not None: + oprot.writeFieldBegin('creds', TType.STRUCT, 2) + self.creds.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class uploadNewCredentials_result: + """ + Attributes: + - e + - ite + - aze + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 3 + ) + + def __hash__(self): + return 0 + hash(self.e) + hash(self.ite) + hash(self.aze) + + def __init__(self, e=None, ite=None, aze=None,): + self.e = e + self.ite = ite + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.ite = InvalidTopologyException() + self.ite.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('uploadNewCredentials_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.ite is not None: + oprot.writeFieldBegin('ite', TType.STRUCT, 2) + self.ite.write(oprot) + oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 3) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2072,17 +2471,20 @@ class beginFileUpload_result: """ Attributes: - success + - aze """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + hash(self.success) + return 0 + hash(self.success) + hash(self.aze) - def __init__(self, success=None,): + def __init__(self, success=None, aze=None,): self.success = success + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2098,6 +2500,12 @@ def read(self, iprot): self.success = iprot.readString().decode('utf-8') else: iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2112,6 +2520,10 @@ def write(self, oprot): oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success.encode('utf-8')) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2206,12 +2618,21 @@ def __ne__(self, other): return not (self == other) class uploadChunk_result: + """ + Attributes: + - aze + """ thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + return 0 + hash(self.aze) + + def __init__(self, aze=None,): + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2222,6 +2643,12 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break + if fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2232,6 +2659,10 @@ def write(self, oprot): oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('uploadChunk_result') + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2314,12 +2745,21 @@ def __ne__(self, other): return not (self == other) class finishFileUpload_result: + """ + Attributes: + - aze + """ thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + return 0 + hash(self.aze) + + def __init__(self, aze=None,): + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2330,6 +2770,12 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break + if fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2340,6 +2786,10 @@ def write(self, oprot): oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return oprot.writeStructBegin('finishFileUpload_result') + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2425,17 +2875,20 @@ class beginFileDownload_result: """ Attributes: - success + - aze """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + hash(self.success) + return 0 + hash(self.success) + hash(self.aze) - def __init__(self, success=None,): + def __init__(self, success=None, aze=None,): self.success = success + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2451,6 +2904,12 @@ def read(self, iprot): self.success = iprot.readString().decode('utf-8') else: iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2465,6 +2924,10 @@ def write(self, oprot): oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success.encode('utf-8')) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2550,17 +3013,20 @@ class downloadChunk_result: """ Attributes: - success + - aze """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + hash(self.success) + return 0 + hash(self.success) + hash(self.aze) - def __init__(self, success=None,): + def __init__(self, success=None, aze=None,): self.success = success + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2576,6 +3042,12 @@ def read(self, iprot): self.success = iprot.readString(); else: iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2590,6 +3062,10 @@ def write(self, oprot): oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2657,17 +3133,20 @@ class getNimbusConf_result: """ Attributes: - success + - aze """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + hash(self.success) + return 0 + hash(self.success) + hash(self.aze) - def __init__(self, success=None,): + def __init__(self, success=None, aze=None,): self.success = success + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2683,6 +3162,12 @@ def read(self, iprot): self.success = iprot.readString().decode('utf-8') else: iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2697,6 +3182,10 @@ def write(self, oprot): oprot.writeFieldBegin('success', TType.STRING, 0) oprot.writeString(self.success.encode('utf-8')) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2764,17 +3253,20 @@ class getClusterInfo_result: """ Attributes: - success + - aze """ thrift_spec = ( (0, TType.STRUCT, 'success', (ClusterSummary, ClusterSummary.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1 ) def __hash__(self): - return 0 + hash(self.success) + return 0 + hash(self.success) + hash(self.aze) - def __init__(self, success=None,): + def __init__(self, success=None, aze=None,): self.success = success + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2791,6 +3283,12 @@ def read(self, iprot): self.success.read(iprot) else: iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2805,6 +3303,10 @@ def write(self, oprot): oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 1) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2891,19 +3393,22 @@ class getTopologyInfo_result: Attributes: - success - e + - aze """ thrift_spec = ( (0, TType.STRUCT, 'success', (TopologyInfo, TopologyInfo.thrift_spec), None, ), # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.success) + hash(self.e) + return 0 + hash(self.success) + hash(self.e) + hash(self.aze) - def __init__(self, success=None, e=None,): + def __init__(self, success=None, e=None, aze=None,): self.success = success self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2926,6 +3431,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2944,6 +3455,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -3030,19 +3545,22 @@ class getTopologyConf_result: Attributes: - success - e + - aze """ thrift_spec = ( (0, TType.STRING, 'success', None, None, ), # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.success) + hash(self.e) + return 0 + hash(self.success) + hash(self.e) + hash(self.aze) - def __init__(self, success=None, e=None,): + def __init__(self, success=None, e=None, aze=None,): self.success = success self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -3064,6 +3582,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -3082,6 +3606,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -3168,19 +3696,22 @@ class getTopology_result: Attributes: - success - e + - aze """ thrift_spec = ( (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.success) + hash(self.e) + return 0 + hash(self.success) + hash(self.e) + hash(self.aze) - def __init__(self, success=None, e=None,): + def __init__(self, success=None, e=None, aze=None,): self.success = success self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -3203,6 +3734,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -3221,6 +3758,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -3307,19 +3848,22 @@ class getUserTopology_result: Attributes: - success - e + - aze """ thrift_spec = ( (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0 (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.success) + hash(self.e) + return 0 + hash(self.success) + hash(self.e) + hash(self.aze) - def __init__(self, success=None, e=None,): + def __init__(self, success=None, e=None, aze=None,): self.success = success self.e = e + self.aze = aze def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -3342,6 +3886,12 @@ def read(self, iprot): self.e.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -3360,6 +3910,10 @@ def write(self, oprot): oprot.writeFieldBegin('e', TType.STRUCT, 1) self.e.write(oprot) oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py index 2c0a50bd8cf..8fa34b0e0ee 100644 --- a/storm-core/src/py/storm/ttypes.py +++ b/storm-core/src/py/storm/ttypes.py @@ -1407,6 +1407,74 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) +class AuthorizationException(Exception): + """ + Attributes: + - msg + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'msg', None, None, ), # 1 + ) + + def __hash__(self): + return 0 + hash(self.msg) + + def __init__(self, msg=None,): + self.msg = msg + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('AuthorizationException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + class InvalidTopologyException(Exception): """ Attributes: @@ -1485,6 +1553,8 @@ class TopologySummary: - num_workers - uptime_secs - status + - sched_status + - owner """ thrift_spec = ( @@ -1496,12 +1566,519 @@ class TopologySummary: (5, TType.I32, 'num_workers', None, None, ), # 5 (6, TType.I32, 'uptime_secs', None, None, ), # 6 (7, TType.STRING, 'status', None, None, ), # 7 + None, # 8 + None, # 9 + None, # 10 + None, # 11 + None, # 12 + None, # 13 + None, # 14 + None, # 15 + None, # 16 + None, # 17 + None, # 18 + None, # 19 + None, # 20 + None, # 21 + None, # 22 + None, # 23 + None, # 24 + None, # 25 + None, # 26 + None, # 27 + None, # 28 + None, # 29 + None, # 30 + None, # 31 + None, # 32 + None, # 33 + None, # 34 + None, # 35 + None, # 36 + None, # 37 + None, # 38 + None, # 39 + None, # 40 + None, # 41 + None, # 42 + None, # 43 + None, # 44 + None, # 45 + None, # 46 + None, # 47 + None, # 48 + None, # 49 + None, # 50 + None, # 51 + None, # 52 + None, # 53 + None, # 54 + None, # 55 + None, # 56 + None, # 57 + None, # 58 + None, # 59 + None, # 60 + None, # 61 + None, # 62 + None, # 63 + None, # 64 + None, # 65 + None, # 66 + None, # 67 + None, # 68 + None, # 69 + None, # 70 + None, # 71 + None, # 72 + None, # 73 + None, # 74 + None, # 75 + None, # 76 + None, # 77 + None, # 78 + None, # 79 + None, # 80 + None, # 81 + None, # 82 + None, # 83 + None, # 84 + None, # 85 + None, # 86 + None, # 87 + None, # 88 + None, # 89 + None, # 90 + None, # 91 + None, # 92 + None, # 93 + None, # 94 + None, # 95 + None, # 96 + None, # 97 + None, # 98 + None, # 99 + None, # 100 + None, # 101 + None, # 102 + None, # 103 + None, # 104 + None, # 105 + None, # 106 + None, # 107 + None, # 108 + None, # 109 + None, # 110 + None, # 111 + None, # 112 + None, # 113 + None, # 114 + None, # 115 + None, # 116 + None, # 117 + None, # 118 + None, # 119 + None, # 120 + None, # 121 + None, # 122 + None, # 123 + None, # 124 + None, # 125 + None, # 126 + None, # 127 + None, # 128 + None, # 129 + None, # 130 + None, # 131 + None, # 132 + None, # 133 + None, # 134 + None, # 135 + None, # 136 + None, # 137 + None, # 138 + None, # 139 + None, # 140 + None, # 141 + None, # 142 + None, # 143 + None, # 144 + None, # 145 + None, # 146 + None, # 147 + None, # 148 + None, # 149 + None, # 150 + None, # 151 + None, # 152 + None, # 153 + None, # 154 + None, # 155 + None, # 156 + None, # 157 + None, # 158 + None, # 159 + None, # 160 + None, # 161 + None, # 162 + None, # 163 + None, # 164 + None, # 165 + None, # 166 + None, # 167 + None, # 168 + None, # 169 + None, # 170 + None, # 171 + None, # 172 + None, # 173 + None, # 174 + None, # 175 + None, # 176 + None, # 177 + None, # 178 + None, # 179 + None, # 180 + None, # 181 + None, # 182 + None, # 183 + None, # 184 + None, # 185 + None, # 186 + None, # 187 + None, # 188 + None, # 189 + None, # 190 + None, # 191 + None, # 192 + None, # 193 + None, # 194 + None, # 195 + None, # 196 + None, # 197 + None, # 198 + None, # 199 + None, # 200 + None, # 201 + None, # 202 + None, # 203 + None, # 204 + None, # 205 + None, # 206 + None, # 207 + None, # 208 + None, # 209 + None, # 210 + None, # 211 + None, # 212 + None, # 213 + None, # 214 + None, # 215 + None, # 216 + None, # 217 + None, # 218 + None, # 219 + None, # 220 + None, # 221 + None, # 222 + None, # 223 + None, # 224 + None, # 225 + None, # 226 + None, # 227 + None, # 228 + None, # 229 + None, # 230 + None, # 231 + None, # 232 + None, # 233 + None, # 234 + None, # 235 + None, # 236 + None, # 237 + None, # 238 + None, # 239 + None, # 240 + None, # 241 + None, # 242 + None, # 243 + None, # 244 + None, # 245 + None, # 246 + None, # 247 + None, # 248 + None, # 249 + None, # 250 + None, # 251 + None, # 252 + None, # 253 + None, # 254 + None, # 255 + None, # 256 + None, # 257 + None, # 258 + None, # 259 + None, # 260 + None, # 261 + None, # 262 + None, # 263 + None, # 264 + None, # 265 + None, # 266 + None, # 267 + None, # 268 + None, # 269 + None, # 270 + None, # 271 + None, # 272 + None, # 273 + None, # 274 + None, # 275 + None, # 276 + None, # 277 + None, # 278 + None, # 279 + None, # 280 + None, # 281 + None, # 282 + None, # 283 + None, # 284 + None, # 285 + None, # 286 + None, # 287 + None, # 288 + None, # 289 + None, # 290 + None, # 291 + None, # 292 + None, # 293 + None, # 294 + None, # 295 + None, # 296 + None, # 297 + None, # 298 + None, # 299 + None, # 300 + None, # 301 + None, # 302 + None, # 303 + None, # 304 + None, # 305 + None, # 306 + None, # 307 + None, # 308 + None, # 309 + None, # 310 + None, # 311 + None, # 312 + None, # 313 + None, # 314 + None, # 315 + None, # 316 + None, # 317 + None, # 318 + None, # 319 + None, # 320 + None, # 321 + None, # 322 + None, # 323 + None, # 324 + None, # 325 + None, # 326 + None, # 327 + None, # 328 + None, # 329 + None, # 330 + None, # 331 + None, # 332 + None, # 333 + None, # 334 + None, # 335 + None, # 336 + None, # 337 + None, # 338 + None, # 339 + None, # 340 + None, # 341 + None, # 342 + None, # 343 + None, # 344 + None, # 345 + None, # 346 + None, # 347 + None, # 348 + None, # 349 + None, # 350 + None, # 351 + None, # 352 + None, # 353 + None, # 354 + None, # 355 + None, # 356 + None, # 357 + None, # 358 + None, # 359 + None, # 360 + None, # 361 + None, # 362 + None, # 363 + None, # 364 + None, # 365 + None, # 366 + None, # 367 + None, # 368 + None, # 369 + None, # 370 + None, # 371 + None, # 372 + None, # 373 + None, # 374 + None, # 375 + None, # 376 + None, # 377 + None, # 378 + None, # 379 + None, # 380 + None, # 381 + None, # 382 + None, # 383 + None, # 384 + None, # 385 + None, # 386 + None, # 387 + None, # 388 + None, # 389 + None, # 390 + None, # 391 + None, # 392 + None, # 393 + None, # 394 + None, # 395 + None, # 396 + None, # 397 + None, # 398 + None, # 399 + None, # 400 + None, # 401 + None, # 402 + None, # 403 + None, # 404 + None, # 405 + None, # 406 + None, # 407 + None, # 408 + None, # 409 + None, # 410 + None, # 411 + None, # 412 + None, # 413 + None, # 414 + None, # 415 + None, # 416 + None, # 417 + None, # 418 + None, # 419 + None, # 420 + None, # 421 + None, # 422 + None, # 423 + None, # 424 + None, # 425 + None, # 426 + None, # 427 + None, # 428 + None, # 429 + None, # 430 + None, # 431 + None, # 432 + None, # 433 + None, # 434 + None, # 435 + None, # 436 + None, # 437 + None, # 438 + None, # 439 + None, # 440 + None, # 441 + None, # 442 + None, # 443 + None, # 444 + None, # 445 + None, # 446 + None, # 447 + None, # 448 + None, # 449 + None, # 450 + None, # 451 + None, # 452 + None, # 453 + None, # 454 + None, # 455 + None, # 456 + None, # 457 + None, # 458 + None, # 459 + None, # 460 + None, # 461 + None, # 462 + None, # 463 + None, # 464 + None, # 465 + None, # 466 + None, # 467 + None, # 468 + None, # 469 + None, # 470 + None, # 471 + None, # 472 + None, # 473 + None, # 474 + None, # 475 + None, # 476 + None, # 477 + None, # 478 + None, # 479 + None, # 480 + None, # 481 + None, # 482 + None, # 483 + None, # 484 + None, # 485 + None, # 486 + None, # 487 + None, # 488 + None, # 489 + None, # 490 + None, # 491 + None, # 492 + None, # 493 + None, # 494 + None, # 495 + None, # 496 + None, # 497 + None, # 498 + None, # 499 + None, # 500 + None, # 501 + None, # 502 + None, # 503 + None, # 504 + None, # 505 + None, # 506 + None, # 507 + None, # 508 + None, # 509 + None, # 510 + None, # 511 + None, # 512 + (513, TType.STRING, 'sched_status', None, None, ), # 513 + (514, TType.STRING, 'owner', None, None, ), # 514 ) def __hash__(self): - return 0 + hash(self.id) + hash(self.name) + hash(self.num_tasks) + hash(self.num_executors) + hash(self.num_workers) + hash(self.uptime_secs) + hash(self.status) + return 0 + hash(self.id) + hash(self.name) + hash(self.num_tasks) + hash(self.num_executors) + hash(self.num_workers) + hash(self.uptime_secs) + hash(self.status) + hash(self.sched_status) + hash(self.owner) - def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None,): + def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None,): self.id = id self.name = name self.num_tasks = num_tasks @@ -1509,6 +2086,8 @@ def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_w self.num_workers = num_workers self.uptime_secs = uptime_secs self.status = status + self.sched_status = sched_status + self.owner = owner def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -1554,6 +2133,16 @@ def read(self, iprot): self.status = iprot.readString().decode('utf-8') else: iprot.skip(ftype) + elif fid == 513: + if ftype == TType.STRING: + self.sched_status = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 514: + if ftype == TType.STRING: + self.owner = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -1592,6 +2181,14 @@ def write(self, oprot): oprot.writeFieldBegin('status', TType.STRING, 7) oprot.writeString(self.status.encode('utf-8')) oprot.writeFieldEnd() + if self.sched_status is not None: + oprot.writeFieldBegin('sched_status', TType.STRING, 513) + oprot.writeString(self.sched_status.encode('utf-8')) + oprot.writeFieldEnd() + if self.owner is not None: + oprot.writeFieldBegin('owner', TType.STRING, 514) + oprot.writeString(self.owner.encode('utf-8')) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2749,6 +3346,8 @@ class TopologyInfo: - executors - status - errors + - sched_status + - owner """ thrift_spec = ( @@ -2759,18 +3358,528 @@ class TopologyInfo: (4, TType.LIST, 'executors', (TType.STRUCT,(ExecutorSummary, ExecutorSummary.thrift_spec)), None, ), # 4 (5, TType.STRING, 'status', None, None, ), # 5 (6, TType.MAP, 'errors', (TType.STRING,None,TType.LIST,(TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec))), None, ), # 6 + None, # 7 + None, # 8 + None, # 9 + None, # 10 + None, # 11 + None, # 12 + None, # 13 + None, # 14 + None, # 15 + None, # 16 + None, # 17 + None, # 18 + None, # 19 + None, # 20 + None, # 21 + None, # 22 + None, # 23 + None, # 24 + None, # 25 + None, # 26 + None, # 27 + None, # 28 + None, # 29 + None, # 30 + None, # 31 + None, # 32 + None, # 33 + None, # 34 + None, # 35 + None, # 36 + None, # 37 + None, # 38 + None, # 39 + None, # 40 + None, # 41 + None, # 42 + None, # 43 + None, # 44 + None, # 45 + None, # 46 + None, # 47 + None, # 48 + None, # 49 + None, # 50 + None, # 51 + None, # 52 + None, # 53 + None, # 54 + None, # 55 + None, # 56 + None, # 57 + None, # 58 + None, # 59 + None, # 60 + None, # 61 + None, # 62 + None, # 63 + None, # 64 + None, # 65 + None, # 66 + None, # 67 + None, # 68 + None, # 69 + None, # 70 + None, # 71 + None, # 72 + None, # 73 + None, # 74 + None, # 75 + None, # 76 + None, # 77 + None, # 78 + None, # 79 + None, # 80 + None, # 81 + None, # 82 + None, # 83 + None, # 84 + None, # 85 + None, # 86 + None, # 87 + None, # 88 + None, # 89 + None, # 90 + None, # 91 + None, # 92 + None, # 93 + None, # 94 + None, # 95 + None, # 96 + None, # 97 + None, # 98 + None, # 99 + None, # 100 + None, # 101 + None, # 102 + None, # 103 + None, # 104 + None, # 105 + None, # 106 + None, # 107 + None, # 108 + None, # 109 + None, # 110 + None, # 111 + None, # 112 + None, # 113 + None, # 114 + None, # 115 + None, # 116 + None, # 117 + None, # 118 + None, # 119 + None, # 120 + None, # 121 + None, # 122 + None, # 123 + None, # 124 + None, # 125 + None, # 126 + None, # 127 + None, # 128 + None, # 129 + None, # 130 + None, # 131 + None, # 132 + None, # 133 + None, # 134 + None, # 135 + None, # 136 + None, # 137 + None, # 138 + None, # 139 + None, # 140 + None, # 141 + None, # 142 + None, # 143 + None, # 144 + None, # 145 + None, # 146 + None, # 147 + None, # 148 + None, # 149 + None, # 150 + None, # 151 + None, # 152 + None, # 153 + None, # 154 + None, # 155 + None, # 156 + None, # 157 + None, # 158 + None, # 159 + None, # 160 + None, # 161 + None, # 162 + None, # 163 + None, # 164 + None, # 165 + None, # 166 + None, # 167 + None, # 168 + None, # 169 + None, # 170 + None, # 171 + None, # 172 + None, # 173 + None, # 174 + None, # 175 + None, # 176 + None, # 177 + None, # 178 + None, # 179 + None, # 180 + None, # 181 + None, # 182 + None, # 183 + None, # 184 + None, # 185 + None, # 186 + None, # 187 + None, # 188 + None, # 189 + None, # 190 + None, # 191 + None, # 192 + None, # 193 + None, # 194 + None, # 195 + None, # 196 + None, # 197 + None, # 198 + None, # 199 + None, # 200 + None, # 201 + None, # 202 + None, # 203 + None, # 204 + None, # 205 + None, # 206 + None, # 207 + None, # 208 + None, # 209 + None, # 210 + None, # 211 + None, # 212 + None, # 213 + None, # 214 + None, # 215 + None, # 216 + None, # 217 + None, # 218 + None, # 219 + None, # 220 + None, # 221 + None, # 222 + None, # 223 + None, # 224 + None, # 225 + None, # 226 + None, # 227 + None, # 228 + None, # 229 + None, # 230 + None, # 231 + None, # 232 + None, # 233 + None, # 234 + None, # 235 + None, # 236 + None, # 237 + None, # 238 + None, # 239 + None, # 240 + None, # 241 + None, # 242 + None, # 243 + None, # 244 + None, # 245 + None, # 246 + None, # 247 + None, # 248 + None, # 249 + None, # 250 + None, # 251 + None, # 252 + None, # 253 + None, # 254 + None, # 255 + None, # 256 + None, # 257 + None, # 258 + None, # 259 + None, # 260 + None, # 261 + None, # 262 + None, # 263 + None, # 264 + None, # 265 + None, # 266 + None, # 267 + None, # 268 + None, # 269 + None, # 270 + None, # 271 + None, # 272 + None, # 273 + None, # 274 + None, # 275 + None, # 276 + None, # 277 + None, # 278 + None, # 279 + None, # 280 + None, # 281 + None, # 282 + None, # 283 + None, # 284 + None, # 285 + None, # 286 + None, # 287 + None, # 288 + None, # 289 + None, # 290 + None, # 291 + None, # 292 + None, # 293 + None, # 294 + None, # 295 + None, # 296 + None, # 297 + None, # 298 + None, # 299 + None, # 300 + None, # 301 + None, # 302 + None, # 303 + None, # 304 + None, # 305 + None, # 306 + None, # 307 + None, # 308 + None, # 309 + None, # 310 + None, # 311 + None, # 312 + None, # 313 + None, # 314 + None, # 315 + None, # 316 + None, # 317 + None, # 318 + None, # 319 + None, # 320 + None, # 321 + None, # 322 + None, # 323 + None, # 324 + None, # 325 + None, # 326 + None, # 327 + None, # 328 + None, # 329 + None, # 330 + None, # 331 + None, # 332 + None, # 333 + None, # 334 + None, # 335 + None, # 336 + None, # 337 + None, # 338 + None, # 339 + None, # 340 + None, # 341 + None, # 342 + None, # 343 + None, # 344 + None, # 345 + None, # 346 + None, # 347 + None, # 348 + None, # 349 + None, # 350 + None, # 351 + None, # 352 + None, # 353 + None, # 354 + None, # 355 + None, # 356 + None, # 357 + None, # 358 + None, # 359 + None, # 360 + None, # 361 + None, # 362 + None, # 363 + None, # 364 + None, # 365 + None, # 366 + None, # 367 + None, # 368 + None, # 369 + None, # 370 + None, # 371 + None, # 372 + None, # 373 + None, # 374 + None, # 375 + None, # 376 + None, # 377 + None, # 378 + None, # 379 + None, # 380 + None, # 381 + None, # 382 + None, # 383 + None, # 384 + None, # 385 + None, # 386 + None, # 387 + None, # 388 + None, # 389 + None, # 390 + None, # 391 + None, # 392 + None, # 393 + None, # 394 + None, # 395 + None, # 396 + None, # 397 + None, # 398 + None, # 399 + None, # 400 + None, # 401 + None, # 402 + None, # 403 + None, # 404 + None, # 405 + None, # 406 + None, # 407 + None, # 408 + None, # 409 + None, # 410 + None, # 411 + None, # 412 + None, # 413 + None, # 414 + None, # 415 + None, # 416 + None, # 417 + None, # 418 + None, # 419 + None, # 420 + None, # 421 + None, # 422 + None, # 423 + None, # 424 + None, # 425 + None, # 426 + None, # 427 + None, # 428 + None, # 429 + None, # 430 + None, # 431 + None, # 432 + None, # 433 + None, # 434 + None, # 435 + None, # 436 + None, # 437 + None, # 438 + None, # 439 + None, # 440 + None, # 441 + None, # 442 + None, # 443 + None, # 444 + None, # 445 + None, # 446 + None, # 447 + None, # 448 + None, # 449 + None, # 450 + None, # 451 + None, # 452 + None, # 453 + None, # 454 + None, # 455 + None, # 456 + None, # 457 + None, # 458 + None, # 459 + None, # 460 + None, # 461 + None, # 462 + None, # 463 + None, # 464 + None, # 465 + None, # 466 + None, # 467 + None, # 468 + None, # 469 + None, # 470 + None, # 471 + None, # 472 + None, # 473 + None, # 474 + None, # 475 + None, # 476 + None, # 477 + None, # 478 + None, # 479 + None, # 480 + None, # 481 + None, # 482 + None, # 483 + None, # 484 + None, # 485 + None, # 486 + None, # 487 + None, # 488 + None, # 489 + None, # 490 + None, # 491 + None, # 492 + None, # 493 + None, # 494 + None, # 495 + None, # 496 + None, # 497 + None, # 498 + None, # 499 + None, # 500 + None, # 501 + None, # 502 + None, # 503 + None, # 504 + None, # 505 + None, # 506 + None, # 507 + None, # 508 + None, # 509 + None, # 510 + None, # 511 + None, # 512 + (513, TType.STRING, 'sched_status', None, None, ), # 513 + (514, TType.STRING, 'owner', None, None, ), # 514 ) def __hash__(self): - return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.executors) + hash(self.status) + hash(self.errors) + return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.executors) + hash(self.status) + hash(self.errors) + hash(self.sched_status) + hash(self.owner) - def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None,): + def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, sched_status=None, owner=None,): self.id = id self.name = name self.uptime_secs = uptime_secs self.executors = executors self.status = status self.errors = errors + self.sched_status = sched_status + self.owner = owner def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2829,6 +3938,16 @@ def read(self, iprot): iprot.readMapEnd() else: iprot.skip(ftype) + elif fid == 513: + if ftype == TType.STRING: + self.sched_status = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 514: + if ftype == TType.STRING: + self.owner = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2873,6 +3992,14 @@ def write(self, oprot): oprot.writeListEnd() oprot.writeMapEnd() oprot.writeFieldEnd() + if self.sched_status is not None: + oprot.writeFieldBegin('sched_status', TType.STRING, 513) + oprot.writeString(self.sched_status.encode('utf-8')) + oprot.writeFieldEnd() + if self.owner is not None: + oprot.writeFieldBegin('owner', TType.STRING, 514) + oprot.writeString(self.owner.encode('utf-8')) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -3063,22 +4190,100 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) +class Credentials: + """ + Attributes: + - creds + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'creds', (TType.STRING,None,TType.STRING,None), None, ), # 1 + ) + + def __hash__(self): + return 0 + hash(self.creds) + + def __init__(self, creds=None,): + self.creds = creds + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.creds = {} + (_ktype293, _vtype294, _size292 ) = iprot.readMapBegin() + for _i296 in xrange(_size292): + _key297 = iprot.readString().decode('utf-8') + _val298 = iprot.readString().decode('utf-8') + self.creds[_key297] = _val298 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('Credentials') + if self.creds is not None: + oprot.writeFieldBegin('creds', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds)) + for kiter299,viter300 in self.creds.items(): + oprot.writeString(kiter299.encode('utf-8')) + oprot.writeString(viter300.encode('utf-8')) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.creds is None: + raise TProtocol.TProtocolException(message='Required field creds is unset!') + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + class SubmitOptions: """ Attributes: - initial_status + - creds """ thrift_spec = ( None, # 0 (1, TType.I32, 'initial_status', None, None, ), # 1 + (2, TType.STRUCT, 'creds', (Credentials, Credentials.thrift_spec), None, ), # 2 ) def __hash__(self): - return 0 + hash(self.initial_status) + return 0 + hash(self.initial_status) + hash(self.creds) - def __init__(self, initial_status=None,): + def __init__(self, initial_status=None, creds=None,): self.initial_status = initial_status + self.creds = creds def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -3094,6 +4299,12 @@ def read(self, iprot): self.initial_status = iprot.readI32(); else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.creds = Credentials() + self.creds.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -3108,6 +4319,10 @@ def write(self, oprot): oprot.writeFieldBegin('initial_status', TType.I32, 1) oprot.writeI32(self.initial_status) oprot.writeFieldEnd() + if self.creds is not None: + oprot.writeFieldBegin('creds', TType.STRUCT, 2) + self.creds.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift index 3c54d305de3..679d29e9c57 100644 --- a/storm-core/src/storm.thrift +++ b/storm-core/src/storm.thrift @@ -127,6 +127,10 @@ exception NotAliveException { 1: required string msg; } +exception AuthorizationException { + 1: required string msg; +} + exception InvalidTopologyException { 1: required string msg; } @@ -139,6 +143,8 @@ struct TopologySummary { 5: required i32 num_workers; 6: required i32 uptime_secs; 7: required string status; +513: optional string sched_status; +514: optional string owner; } struct SupervisorSummary { @@ -208,6 +214,8 @@ struct TopologyInfo { 4: required list executors; 5: required string status; 6: required map> errors; +513: optional string sched_status; +514: optional string owner; } struct KillOptions { @@ -220,42 +228,48 @@ struct RebalanceOptions { 3: optional map num_executors; } +struct Credentials { + 1: required map creds; +} + enum TopologyInitialStatus { ACTIVE = 1, INACTIVE = 2 } struct SubmitOptions { 1: required TopologyInitialStatus initial_status; + 2: optional Credentials creds; } service Nimbus { - void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite); - void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite); - void killTopology(1: string name) throws (1: NotAliveException e); - void killTopologyWithOpts(1: string name, 2: KillOptions options) throws (1: NotAliveException e); - void activate(1: string name) throws (1: NotAliveException e); - void deactivate(1: string name) throws (1: NotAliveException e); - void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite); + void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze); + void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze); + void killTopology(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze); + void killTopologyWithOpts(1: string name, 2: KillOptions options) throws (1: NotAliveException e, 2: AuthorizationException aze); + void activate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze); + void deactivate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze); + void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze); + void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze); // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs - string beginFileUpload(); - void uploadChunk(1: string location, 2: binary chunk); - void finishFileUpload(1: string location); + string beginFileUpload() throws (1: AuthorizationException aze); + void uploadChunk(1: string location, 2: binary chunk) throws (1: AuthorizationException aze); + void finishFileUpload(1: string location) throws (1: AuthorizationException aze); - string beginFileDownload(1: string file); + string beginFileDownload(1: string file) throws (1: AuthorizationException aze); //can stop downloading chunks when receive 0-length byte array back - binary downloadChunk(1: string id); + binary downloadChunk(1: string id) throws (1: AuthorizationException aze); // returns json - string getNimbusConf(); + string getNimbusConf() throws (1: AuthorizationException aze); // stats functions - ClusterSummary getClusterInfo(); - TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e); + ClusterSummary getClusterInfo() throws (1: AuthorizationException aze); + TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze); //returns json - string getTopologyConf(1: string id) throws (1: NotAliveException e); - StormTopology getTopology(1: string id) throws (1: NotAliveException e); - StormTopology getUserTopology(1: string id) throws (1: NotAliveException e); + string getTopologyConf(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze); + StormTopology getTopology(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze); + StormTopology getUserTopology(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze); } struct DRPCRequest { @@ -268,11 +282,11 @@ exception DRPCExecutionException { } service DistributedRPC { - string execute(1: string functionName, 2: string funcArgs) throws (1: DRPCExecutionException e); + string execute(1: string functionName, 2: string funcArgs) throws (1: DRPCExecutionException e, 2: AuthorizationException aze); } service DistributedRPCInvocations { - void result(1: string id, 2: string result); - DRPCRequest fetchRequest(1: string functionName); - void failRequest(1: string id); + void result(1: string id, 2: string result) throws (1: AuthorizationException aze); + DRPCRequest fetchRequest(1: string functionName) throws (1: AuthorizationException aze); + void failRequest(1: string id) throws (1: AuthorizationException aze); } diff --git a/storm-core/src/ui/public/css/style.css b/storm-core/src/ui/public/css/style.css index 6c7c119f51c..29a45ebd73d 100644 --- a/storm-core/src/ui/public/css/style.css +++ b/storm-core/src/ui/public/css/style.css @@ -19,8 +19,14 @@ display: none; } +.ui-user { + float: right; + padding: 0.5em; +} + body { color: #808080; + padding: 0.2em; } table { diff --git a/storm-core/test/clj/backtype/storm/cluster_test.clj b/storm-core/test/clj/backtype/storm/cluster_test.clj index 987429b7ef3..ba94ab161eb 100644 --- a/storm-core/test/clj/backtype/storm/cluster_test.clj +++ b/storm-core/test/clj/backtype/storm/cluster_test.clj @@ -16,6 +16,14 @@ (ns backtype.storm.cluster-test (:import [java.util Arrays]) (:import [backtype.storm.daemon.common Assignment StormBase SupervisorInfo]) + (:import [org.apache.zookeeper ZooDefs ZooDefs$Ids]) + (:import [org.mockito Mockito]) + (:import [org.mockito.exceptions.base MockitoAssertionError]) + (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder]) + (:import [backtype.storm.utils Utils TestUtils ZookeeperAuthInfo]) + (:require [backtype.storm [zookeeper :as zk]]) + (:require [conjure.core]) + (:use [conjure core]) (:use [clojure test]) (:use [backtype.storm cluster config util testing])) @@ -25,7 +33,8 @@ STORM-ZOOKEEPER-SERVERS ["localhost"]})) (defn mk-state - ([zk-port] (mk-distributed-cluster-state (mk-config zk-port))) + ([zk-port] (let [conf (mk-config zk-port)] + (mk-distributed-cluster-state conf :auth-conf conf))) ([zk-port cb] (let [ret (mk-state zk-port)] (.register ret cb) @@ -36,16 +45,16 @@ (deftest test-basics (with-inprocess-zookeeper zk-port (let [state (mk-state zk-port)] - (.set-data state "/root" (barr 1 2 3)) + (.set-data state "/root" (barr 1 2 3) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (Arrays/equals (barr 1 2 3) (.get-data state "/root" false))) (is (= nil (.get-data state "/a" false))) - (.set-data state "/root/a" (barr 1 2)) - (.set-data state "/root" (barr 1)) + (.set-data state "/root/a" (barr 1 2) ZooDefs$Ids/OPEN_ACL_UNSAFE) + (.set-data state "/root" (barr 1) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (Arrays/equals (barr 1) (.get-data state "/root" false))) (is (Arrays/equals (barr 1 2) (.get-data state "/root/a" false))) - (.set-data state "/a/b/c/d" (barr 99)) + (.set-data state "/a/b/c/d" (barr 99) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (Arrays/equals (barr 99) (.get-data state "/a/b/c/d" false))) - (.mkdirs state "/lalala") + (.mkdirs state "/lalala" ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (= [] (.get-children state "/lalala" false))) (is (= #{"root" "a" "lalala"} (set (.get-children state "/" false)))) (.delete-node state "/a") @@ -58,7 +67,7 @@ (with-inprocess-zookeeper zk-port (let [state1 (mk-state zk-port) state2 (mk-state zk-port)] - (.set-data state1 "/root" (barr 1)) + (.set-data state1 "/root" (barr 1) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (Arrays/equals (barr 1) (.get-data state1 "/root" false))) (is (Arrays/equals (barr 1) (.get-data state2 "/root" false))) (.delete-node state2 "/root") @@ -73,7 +82,7 @@ (let [state1 (mk-state zk-port) state2 (mk-state zk-port) state3 (mk-state zk-port)] - (.set-ephemeral-node state1 "/a" (barr 1)) + (.set-ephemeral-node state1 "/a" (barr 1) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (Arrays/equals (barr 1) (.get-data state1 "/a" false))) (is (Arrays/equals (barr 1) (.get-data state2 "/a" false))) (.close state3) @@ -111,37 +120,37 @@ state1 (mk-state zk-port state1-cb) [state2-last-cb state2-cb] (mk-callback-tester) state2 (mk-state zk-port state2-cb)] - (.set-data state1 "/root" (barr 1)) + (.set-data state1 "/root" (barr 1) ZooDefs$Ids/OPEN_ACL_UNSAFE) (.get-data state2 "/root" true) (is (= nil @state1-last-cb)) (is (= nil @state2-last-cb)) - (.set-data state2 "/root" (barr 2)) + (.set-data state2 "/root" (barr 2) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (= {:type :node-data-changed :path "/root"} (read-and-reset! state2-last-cb))) (is (= nil @state1-last-cb)) - (.set-data state2 "/root" (barr 3)) + (.set-data state2 "/root" (barr 3) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (= nil @state2-last-cb)) (.get-data state2 "/root" true) (.get-data state2 "/root" false) (.delete-node state1 "/root") (is (= {:type :node-deleted :path "/root"} (read-and-reset! state2-last-cb))) (.get-data state2 "/root" true) - (.set-ephemeral-node state1 "/root" (barr 1 2 3 4)) + (.set-ephemeral-node state1 "/root" (barr 1 2 3 4) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (= {:type :node-created :path "/root"} (read-and-reset! state2-last-cb))) (.get-children state1 "/" true) - (.set-data state2 "/a" (barr 9)) + (.set-data state2 "/a" (barr 9) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (= nil @state2-last-cb)) (is (= {:type :node-children-changed :path "/"} (read-and-reset! state1-last-cb))) (.get-data state2 "/root" true) - (.set-ephemeral-node state1 "/root" (barr 1 2)) + (.set-ephemeral-node state1 "/root" (barr 1 2) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (= {:type :node-data-changed :path "/root"} (read-and-reset! state2-last-cb))) - (.mkdirs state1 "/ccc") + (.mkdirs state1 "/ccc" ZooDefs$Ids/OPEN_ACL_UNSAFE) (.get-children state1 "/ccc" true) (.get-data state2 "/ccc/b" true) - (.set-data state2 "/ccc/b" (barr 8)) + (.set-data state2 "/ccc/b" (barr 8) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (= {:type :node-created :path "/ccc/b"} (read-and-reset! state2-last-cb))) (is (= {:type :node-children-changed :path "/ccc"} (read-and-reset! state1-last-cb))) @@ -150,7 +159,7 @@ (.close state1) (is (= {:type :node-deleted :path "/root"} (read-and-reset! state2-last-cb))) - (.set-data state2 "/root2" (barr 9)) + (.set-data state2 "/root2" (barr 9) ZooDefs$Ids/OPEN_ACL_UNSAFE) (is (= {:type :node-created :path "/root2"} (read-and-reset! state2-last-cb))) (.close state2) ))) @@ -161,8 +170,8 @@ (let [state (mk-storm-state zk-port) assignment1 (Assignment. "/aaa" {} {1 [2 2002 1]} {}) assignment2 (Assignment. "/aaa" {} {1 [2 2002]} {}) - base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {}) - base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {})] + base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "") + base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "")] (is (= [] (.assignments state nil))) (.set-assignment! state "storm1" assignment1) (is (= assignment1 (.assignment-info state "storm1" nil))) @@ -186,6 +195,11 @@ (is (= base2 (.storm-base state "storm2" nil))) (is (= #{"storm2"} (set (.active-storms state)))) + (is (nil? (.credentials state "storm1" nil))) + (.set-credentials! state "storm1" {"a" "a"} {}) + (is (= {"a" "a"} (.credentials state "storm1" nil))) + (.set-credentials! state "storm1" {"b" "b"} {}) + (is (= {"b" "b"} (.credentials state "storm1" nil))) ;; TODO add tests for task info and task heartbeat setting and getting (.disconnect state) @@ -240,9 +254,44 @@ (.disconnect state1) ))) + + +(deftest test-cluster-authentication + (with-inprocess-zookeeper zk-port + (let [builder (Mockito/mock CuratorFrameworkFactory$Builder) + conf (merge + (mk-config zk-port) + {STORM-ZOOKEEPER-CONNECTION-TIMEOUT 10 + STORM-ZOOKEEPER-SESSION-TIMEOUT 10 + STORM-ZOOKEEPER-RETRY-INTERVAL 5 + STORM-ZOOKEEPER-RETRY-TIMES 2 + STORM-ZOOKEEPER-RETRY-INTERVAL-CEILING 15 + STORM-ZOOKEEPER-AUTH-SCHEME "digest" + STORM-ZOOKEEPER-AUTH-PAYLOAD "storm:thisisapoorpassword"})] + (. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder)) + (. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder)) + (. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder)) + (TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf)) + (is (nil? + (try + (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD)))) + (catch MockitoAssertionError e + e))))))) + (deftest test-storm-state-callbacks ;; TODO finish ) - - +(deftest test-cluster-state-default-acls + (testing "The default ACLs are empty." + (stubbing [zk/mkdirs nil + zk/mk-client (reify CuratorFramework (^void close [this] nil))] + (mk-distributed-cluster-state {}) + (verify-call-times-for zk/mkdirs 1) + (verify-first-call-args-for-indices zk/mkdirs [2] nil)) + (stubbing [mk-distributed-cluster-state nil + register nil + mkdirs nil] + (mk-storm-cluster-state {}) + (verify-call-times-for mk-distributed-cluster-state 1) + (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil)))) diff --git a/storm-core/test/clj/backtype/storm/config_test.clj b/storm-core/test/clj/backtype/storm/config_test.clj index 01f788ba738..2b7f96e0bcb 100644 --- a/storm-core/test/clj/backtype/storm/config_test.clj +++ b/storm-core/test/clj/backtype/storm/config_test.clj @@ -84,6 +84,17 @@ (is (thrown-cause? java.lang.IllegalArgumentException (.validateField validator "test" 42))))) +(deftest test-positive-integer-validator + (let [validator ConfigValidation/PositiveIntegerValidator] + (doseq [x [42.42 -32 0 -0 "Forty-two"]] + (is (thrown-cause? java.lang.IllegalArgumentException + (.validateField validator "test" x)))) + + (doseq [x [42 4294967296 1 nil]] + (is (nil? (try + (.validateField validator "test" x) + (catch Exception e e))))))) + (deftest test-worker-childopts-is-string-or-string-list (let [pass-cases [nil "some string" ["some" "string" "list"]]] (testing "worker.childopts validates" diff --git a/storm-core/test/clj/backtype/storm/logviewer_test.clj b/storm-core/test/clj/backtype/storm/logviewer_test.clj new file mode 100644 index 00000000000..37e63b915cd --- /dev/null +++ b/storm-core/test/clj/backtype/storm/logviewer_test.clj @@ -0,0 +1,187 @@ +;; 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.logviewer-test + (:use [backtype.storm config util]) + (:require [backtype.storm.daemon [logviewer :as logviewer] + [supervisor :as supervisor]]) + (:require [conjure.core]) + (:use [clojure test]) + (:use [conjure core]) + (:import [org.mockito Mockito])) + +(defmulti mk-mock-File #(:type %)) + +(defmethod mk-mock-File :file [{file-name :name mtime :mtime + :or {file-name "afile" mtime 1}}] + (let [mockFile (Mockito/mock java.io.File)] + (. (Mockito/when (.getName mockFile)) thenReturn file-name) + (. (Mockito/when (.lastModified mockFile)) thenReturn mtime) + (. (Mockito/when (.isFile mockFile)) thenReturn true) + (. (Mockito/when (.getCanonicalPath mockFile)) + thenReturn (str "/mock/canonical/path/to/" file-name)) + mockFile)) + +(defmethod mk-mock-File :directory [{dir-name :name mtime :mtime + :or {dir-name "adir" mtime 1}}] + (let [mockDir (Mockito/mock java.io.File)] + (. (Mockito/when (.getName mockDir)) thenReturn dir-name) + (. (Mockito/when (.lastModified mockDir)) thenReturn mtime) + (. (Mockito/when (.isFile mockDir)) thenReturn false) + mockDir)) + +(deftest test-mk-FileFilter-for-log-cleanup + (testing "log file filter selects the correct log files for purge" + (let [now-millis (current-time-millis) + conf {LOGVIEWER-CLEANUP-AGE-MINS 60 + LOGVIEWER-CLEANUP-INTERVAL-SECS 300} + cutoff-millis (logviewer/cleanup-cutoff-age-millis conf now-millis) + old-mtime-millis (- cutoff-millis 500) + new-mtime-millis (+ cutoff-millis 500) + matching-files (map #(mk-mock-File %) + [{:name "oldlog-1-2-worker-3.log" + :type :file + :mtime old-mtime-millis} + {:name "oldlog-1-2-worker-3.log.8" + :type :file + :mtime old-mtime-millis} + {:name "foobar*_topo-1-24242-worker-2834238.log" + :type :file + :mtime old-mtime-millis}]) + excluded-files (map #(mk-mock-File %) + [{:name "oldlog-1-2-worker-.log" + :type :file + :mtime old-mtime-millis} + {:name "olddir-1-2-worker.log" + :type :directory + :mtime old-mtime-millis} + {:name "newlog-1-2-worker.log" + :type :file + :mtime new-mtime-millis} + {:name "some-old-file.txt" + :type :file + :mtime old-mtime-millis} + {:name "metadata" + :type :directory + :mtime old-mtime-millis} + {:name "newdir-1-2-worker.log" + :type :directory + :mtime new-mtime-millis} + {:name "newdir" + :type :directory + :mtime new-mtime-millis} + ]) + file-filter (logviewer/mk-FileFilter-for-log-cleanup conf now-millis)] + (is (every? #(.accept file-filter %) matching-files)) + (is (not-any? #(.accept file-filter %) excluded-files)) + ))) + +(deftest test-get-log-root->files-map + (testing "returns map of root name to list of files" + (let [files (vec (map #(java.io.File. %) ["log-1-2-worker-3.log" + "log-1-2-worker-3.log.1" + "log-2-4-worker-6.log.1"])) + expected {"log-1-2-worker-3" #{(files 0) (files 1)} + "log-2-4-worker-6" #{(files 2)}}] + (is (= expected (logviewer/get-log-root->files-map files)))))) + +(deftest test-identify-worker-log-files + (testing "Does not include metadata file when there are any log files that + should not be cleaned up" + (let [cutoff-millis 2000 + old-logFile (mk-mock-File {:name "mock-1-1-worker-1.log.1" + :type :file + :mtime (- cutoff-millis 1000)}) + mock-metaFile (mk-mock-File {:name "mock-1-1-worker-1.yaml" + :type :file + :mtime 1}) + new-logFile (mk-mock-File {:name "mock-1-1-worker-1.log" + :type :file + :mtime (+ cutoff-millis 1000)}) + exp-id "id12345" + exp-user "alice" + expected {exp-id {:owner exp-user + :files #{old-logFile}}}] + (stubbing [supervisor/read-worker-heartbeats nil + logviewer/get-metadata-file-for-log-root-name mock-metaFile + read-dir-contents [(.getName old-logFile) (.getName new-logFile)] + logviewer/get-worker-id-from-metadata-file exp-id + logviewer/get-topo-owner-from-metadata-file exp-user] + (is (= expected (logviewer/identify-worker-log-files [old-logFile] "/tmp/"))))))) + +(deftest test-get-dead-worker-files-and-owners + (testing "removes any files of workers that are still alive" + (let [conf {SUPERVISOR-WORKER-TIMEOUT-SECS 5} + id->hb {"42" {:time-secs 1}} + now-secs 2 + log-files #{:expected-file :unexpected-file} + exp-owner "alice"] + (stubbing [logviewer/identify-worker-log-files {"42" {:owner exp-owner + :files #{:unexpected-file}} + "007" {:owner exp-owner + :files #{:expected-file}}} + logviewer/get-topo-owner-from-metadata-file "alice" + supervisor/read-worker-heartbeats id->hb] + (is (= [{:owner exp-owner :files #{:expected-file}}] + (logviewer/get-dead-worker-files-and-owners conf now-secs log-files "/tmp/"))))))) + +(deftest test-cleanup-fn + (testing "cleanup function removes file as user when one is specified" + (let [exp-user "mock-user" + mockfile1 (mk-mock-File {:name "file1" :type :file}) + mockfile2 (mk-mock-File {:name "file2" :type :file}) + mockfile3 (mk-mock-File {:name "file3" :type :file}) + mockyaml (mk-mock-File {:name "foo.yaml" :type :file}) + exp-cmd (str "rmr /mock/canonical/path/to/" (.getName mockfile3))] + (stubbing [logviewer/select-files-for-cleanup + [(mk-mock-File {:name "throwaway" :type :file})] + logviewer/get-dead-worker-files-and-owners + [{:owner nil :files #{mockfile1}} + {:files #{mockfile2}} + {:owner exp-user :files #{mockfile3 mockyaml}}] + supervisor/worker-launcher nil + rmr nil] + (logviewer/cleanup-fn! "/tmp/") + (verify-call-times-for supervisor/worker-launcher 1) + (verify-first-call-args-for-indices supervisor/worker-launcher + [1 2] exp-user exp-cmd) + (verify-call-times-for rmr 3) + (verify-nth-call-args-for 1 rmr (.getCanonicalPath mockfile1)) + (verify-nth-call-args-for 2 rmr (.getCanonicalPath mockfile2)) + (verify-nth-call-args-for 3 rmr (.getCanonicalPath mockyaml)))))) + +(deftest test-authorized-log-user + (testing "allow cluster admin" + (let [conf {NIMBUS-ADMINS ["alice"]}] + (stubbing [logviewer/get-log-user-whitelist []] + (is (logviewer/authorized-log-user? "alice" "non-blank-fname" conf))))) + + (testing "ignore any cluster-set topology.users" + (let [conf {TOPOLOGY-USERS ["alice"]}] + (stubbing [logviewer/get-log-user-whitelist []] + (is (not (logviewer/authorized-log-user? "alice" "non-blank-fname" conf)))))) + + (testing "allow cluster logs user" + (let [conf {LOGS-USERS ["alice"]}] + (stubbing [logviewer/get-log-user-whitelist []] + (is (logviewer/authorized-log-user? "alice" "non-blank-fname" conf))))) + + (testing "allow whitelisted topology user" + (stubbing [logviewer/get-log-user-whitelist ["alice"]] + (is (logviewer/authorized-log-user? "alice" "non-blank-fname" {})))) + + (testing "disallow user not in nimbus admin, topo user, logs user, or whitelist" + (stubbing [logviewer/get-log-user-whitelist []] + (is (not (logviewer/authorized-log-user? "alice" "non-blank-fname" {})))))) diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj index 31e69e8eff2..8534c827b99 100644 --- a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj +++ b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj @@ -30,8 +30,7 @@ STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1 - STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1 - }] + STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1}] (let [topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)} {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.) diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj index 1e6554a8bff..c902119fa57 100644 --- a/storm-core/test/clj/backtype/storm/nimbus_test.clj +++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj @@ -15,12 +15,15 @@ ;; limitations under the License. (ns backtype.storm.nimbus-test (:use [clojure test]) + (:require [backtype.storm [util :as util]]) (:require [backtype.storm.daemon [nimbus :as nimbus]]) - (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter]) (:import [backtype.storm.scheduler INimbus]) + (:import [backtype.storm.generated Credentials]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) + (:require [conjure.core]) + (:use [conjure core]) ) (bootstrap) @@ -159,6 +162,8 @@ (is (not-nil? ((:executor->start-time-secs assignment) e)))) )) + + (deftest test-bogusId (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}] (let [state (:storm-cluster-state cluster) @@ -167,6 +172,7 @@ (is (thrown? NotAliveException (.getTopology nimbus "bogus-id"))) (is (thrown? NotAliveException (.getUserTopology nimbus "bogus-id"))) (is (thrown? NotAliveException (.getTopologyInfo nimbus "bogus-id"))) + (is (thrown? NotAliveException (.uploadNewCredentials nimbus "bogus-id" (Credentials.)))) ))) (deftest test-assignment @@ -724,10 +730,77 @@ ))) + +(defn check-for-collisions [state] + (log-message "Checking for collision") + (let [assignments (.assignments state nil)] + (log-message "Assignemts: " assignments) + (let [id->node->ports (into {} (for [id assignments + :let [executor->node+port (:executor->node+port (.assignment-info state id nil)) + node+ports (set (.values executor->node+port)) + node->ports (apply merge-with (fn [a b] (distinct (concat a b))) (for [[node port] node+ports] {node [port]}))]] + {id node->ports})) + _ (log-message "id->node->ports: " id->node->ports) + all-nodes (apply merge-with (fn [a b] + (let [ret (concat a b)] + (log-message "Can we combine " (pr-str a) " and " (pr-str b) " without collisions? " (apply distinct? ret) " => " (pr-str ret)) + (is (apply distinct? ret)) + (distinct ret))) + (.values id->node->ports))] +))) + +(deftest test-rebalance-constrained-cluster + (with-simulated-time-local-cluster [cluster :supervisors 1 :ports-per-supervisor 4 + :daemon-conf {SUPERVISOR-ENABLE false + NIMBUS-MONITOR-FREQ-SECS 10 + TOPOLOGY-MESSAGE-TIMEOUT-SECS 30 + TOPOLOGY-ACKER-EXECUTORS 0}] + (letlocals + (bind topology (thrift/mk-topology + {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)} + {})) + (bind topology2 (thrift/mk-topology + {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)} + {})) + (bind topology3 (thrift/mk-topology + {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)} + {})) + (bind state (:storm-cluster-state cluster)) + (submit-local-topology (:nimbus cluster) + "test" + {TOPOLOGY-WORKERS 3 + TOPOLOGY-MESSAGE-TIMEOUT-SECS 90} topology) + (submit-local-topology (:nimbus cluster) + "test2" + {TOPOLOGY-WORKERS 3 + TOPOLOGY-MESSAGE-TIMEOUT-SECS 90} topology2) + (submit-local-topology (:nimbus cluster) + "test3" + {TOPOLOGY-WORKERS 3 + TOPOLOGY-MESSAGE-TIMEOUT-SECS 90} topology3) + + (advance-cluster-time cluster 31) + + (check-for-collisions state) + (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.) + (.set_num_workers 4) + (.set_wait_secs 0) + )) + + (advance-cluster-time cluster 11) + (check-for-collisions state) + + (advance-cluster-time cluster 30) + (check-for-collisions state) + ))) + + (deftest test-submit-invalid (with-simulated-time-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false - TOPOLOGY-ACKER-EXECUTORS 0}] + TOPOLOGY-ACKER-EXECUTORS 0 + NIMBUS-EXECUTORS-PER-TOPOLOGY 8 + NIMBUS-SLOTS-PER-TOPOLOGY 8}] (letlocals (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 0 :conf {TOPOLOGY-TASKS 1})} @@ -747,7 +820,31 @@ "test/aaa" {} topology))) - ))) + (bind topology (thrift/mk-topology + {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) + :parallelism-hint 16 + :conf {TOPOLOGY-TASKS 16})} + {})) + (bind state (:storm-cluster-state cluster)) + (is (thrown? InvalidTopologyException + (submit-local-topology (:nimbus cluster) + "test" + {TOPOLOGY-WORKERS 3} + topology))) + (bind topology (thrift/mk-topology + {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) + :parallelism-hint 5 + :conf {TOPOLOGY-TASKS 5})} + {})) + (is (thrown? InvalidTopologyException + (submit-local-topology (:nimbus cluster) + "test" + {TOPOLOGY-WORKERS 16} + topology))) + (is (nil? (submit-local-topology (:nimbus cluster) + "test" + {TOPOLOGY-WORKERS 8} + topology)))))) (deftest test-cleans-corrupt (with-inprocess-zookeeper zk-port @@ -775,13 +872,13 @@ (.disconnect cluster-state) )))) -(deftest test-no-overlapping-slots - ;; test that same node+port never appears across 2 assignments - ) +;(deftest test-no-overlapping-slots +; ;; test that same node+port never appears across 2 assignments +; ) -(deftest test-stateless - ;; test that nimbus can die and restart without any problems - ) +;(deftest test-stateless +; ;; test that nimbus can die and restart without any problems +; ) (deftest test-clean-inbox "Tests that the inbox correctly cleans jar files." @@ -816,11 +913,228 @@ (assert-files-in-dir []) )))) +(deftest test-nimbus-iface-submitTopologyWithOpts-checks-authorization + (with-local-cluster [cluster + :daemon-conf {NIMBUS-AUTHORIZER + "backtype.storm.security.auth.authorizer.DenyAuthorizer"}] + (let [ + nimbus (:nimbus cluster) + topology (thrift/mk-topology {} {}) + ] + (is (thrown? AuthorizationException + (submit-local-topology-with-opts nimbus "mystorm" {} topology + (SubmitOptions. TopologyInitialStatus/INACTIVE)) + )) + ) + ) +) + +(deftest test-nimbus-iface-methods-check-authorization + (with-local-cluster [cluster + :daemon-conf {NIMBUS-AUTHORIZER + "backtype.storm.security.auth.authorizer.DenyAuthorizer"}] + (let [ + nimbus (:nimbus cluster) + topology (thrift/mk-topology {} {}) + ] + ; Fake good authorization as part of setup. + (mocking [nimbus/check-authorization!] + (submit-local-topology-with-opts nimbus "test" {} topology + (SubmitOptions. TopologyInitialStatus/INACTIVE)) + ) + (stubbing [nimbus/storm-active? true] + (is (thrown? AuthorizationException + (.rebalance nimbus "test" (RebalanceOptions.)) + )) + ) + (is (thrown? AuthorizationException + (.activate nimbus "test") + )) + (is (thrown? AuthorizationException + (.deactivate nimbus "test") + )) + ) + ) +) + +(deftest test-nimbus-check-authorization-params + (with-local-cluster [cluster + :daemon-conf {NIMBUS-AUTHORIZER "backtype.storm.security.auth.authorizer.NoopAuthorizer"}] + (let [nimbus (:nimbus cluster) + topology-name "test-nimbus-check-autho-params" + topology (thrift/mk-topology {} {})] + + (submit-local-topology-with-opts nimbus topology-name {} topology + (SubmitOptions. TopologyInitialStatus/INACTIVE)) + + (let [expected-name topology-name + expected-conf {TOPOLOGY-NAME expected-name + :foo :bar}] + + (testing "getTopologyConf calls check-authorization! with the correct parameters." + (let [expected-operation "getTopologyConf"] + (stubbing [nimbus/check-authorization! nil + nimbus/try-read-storm-conf expected-conf + util/to-json nil] + (try + (.getTopologyConf nimbus "fake-id") + (catch NotAliveException e) + (finally + (verify-first-call-args-for-indices + nimbus/check-authorization! + [1 2 3] expected-name expected-conf expected-operation) + (verify-first-call-args-for util/to-json expected-conf)))))) + + (testing "getTopology calls check-authorization! with the correct parameters." + (let [expected-operation "getTopology"] + (stubbing [nimbus/check-authorization! nil + nimbus/try-read-storm-conf expected-conf + nimbus/try-read-storm-topology nil + system-topology! nil] + (try + (.getTopology nimbus "fake-id") + (catch NotAliveException e) + (finally + (verify-first-call-args-for-indices + nimbus/check-authorization! + [1 2 3] expected-name expected-conf expected-operation) + (verify-first-call-args-for-indices + system-topology! [0] expected-conf)))))) + + (testing "getUserTopology calls check-authorization with the correct parameters." + (let [expected-operation "getUserTopology"] + (stubbing [nimbus/check-authorization! nil + nimbus/try-read-storm-conf expected-conf + nimbus/try-read-storm-topology nil] + (try + (.getUserTopology nimbus "fake-id") + (catch NotAliveException e) + (finally + (verify-first-call-args-for-indices + nimbus/check-authorization! + [1 2 3] expected-name expected-conf expected-operation) + (verify-first-call-args-for-indices + nimbus/try-read-storm-topology [0] expected-conf)))))))))) + +(deftest test-nimbus-iface-getTopology-methods-throw-correctly + (with-local-cluster [cluster] + (let [ + nimbus (:nimbus cluster) + id "bogus ID" + ] + (is (thrown? NotAliveException (.getTopology nimbus id))) + (try + (.getTopology nimbus id) + (catch NotAliveException e + (is (= id (.get_msg e))) + ) + ) + + (is (thrown? NotAliveException (.getTopologyConf nimbus id))) + (try (.getTopologyConf nimbus id) + (catch NotAliveException e + (is (= id (.get_msg e))) + ) + ) + + (is (thrown? NotAliveException (.getTopologyInfo nimbus id))) + (try (.getTopologyInfo nimbus id) + (catch NotAliveException e + (is (= id (.get_msg e))) + ) + ) + + (is (thrown? NotAliveException (.getUserTopology nimbus id))) + (try (.getUserTopology nimbus id) + (catch NotAliveException e + (is (= id (.get_msg e))) + ) + ) + ) + ) +) + +(deftest test-nimbus-iface-getClusterInfo-filters-topos-without-bases + (with-local-cluster [cluster] + (let [ + nimbus (:nimbus cluster) + bogus-secs 42 + bogus-type "bogusType" + bogus-bases { + "1" nil + "2" {:launch-time-secs bogus-secs + :storm-name "id2-name" + :status {:type bogus-type}} + "3" nil + "4" {:launch-time-secs bogus-secs + :storm-name "id4-name" + :status {:type bogus-type}} + } + ] + (stubbing [topology-bases bogus-bases] + (let [topos (.get_topologies (.getClusterInfo nimbus))] + ; The number of topologies in the summary is correct. + (is (= (count + (filter (fn [b] (second b)) bogus-bases)) (count topos))) + ; Each topology present has a valid name. + (is (empty? + (filter (fn [t] (or (nil? t) (nil? (.get_name t)))) topos))) + ; The topologies are those with valid bases. + (is (empty? + (filter (fn [t] + (or + (nil? t) + (not (number? (read-string (.get_id t)))) + (odd? (read-string (.get_id t))) + )) topos))) + ) + ) + ) + ) +) + +(deftest test-defserverfn-numbus-iface-instance + (test-nimbus-iface-submitTopologyWithOpts-checks-authorization) + (test-nimbus-iface-methods-check-authorization) + (test-nimbus-iface-getTopology-methods-throw-correctly) + (test-nimbus-iface-getClusterInfo-filters-topos-without-bases) +) + +(deftest test-nimbus-data-acls + (testing "nimbus-data uses correct ACLs" + (let [scheme "digest" + digest "storm:thisisapoorpassword" + auth-conf {STORM-ZOOKEEPER-AUTH-SCHEME scheme + STORM-ZOOKEEPER-AUTH-PAYLOAD digest} + expected-acls nimbus/NIMBUS-ZK-ACLS + fake-inimbus (reify INimbus (getForcedScheduler [this] nil))] + (stubbing [mk-authorization-handler nil + cluster/mk-storm-cluster-state nil + nimbus/file-cache-map nil + uptime-computer nil + new-instance nil + mk-timer nil + nimbus/mk-scheduler nil] + (nimbus/nimbus-data auth-conf fake-inimbus) + (verify-call-times-for cluster/mk-storm-cluster-state 1) + (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2] + expected-acls))))) + +(deftest test-file-bogus-download + (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}] + (let [nimbus (:nimbus cluster)] + (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus nil))) + (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus ""))) + (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus "/bogus-path/foo"))) + ))) + (deftest test-validate-topo-config-on-submit (with-local-cluster [cluster] (let [nimbus (:nimbus cluster) topology (thrift/mk-topology {} {}) - bad-config {"topology.workers" "3"}] - (is (thrown-cause? InvalidTopologyException - (submit-local-topology-with-opts nimbus "test" bad-config topology - (SubmitOptions.))))))) + bad-config {"topology.isolate.machines" "2"}] + ; Fake good authorization as part of setup. + (mocking [nimbus/check-authorization!] + (is (thrown-cause? InvalidTopologyException + (submit-local-topology-with-opts nimbus "test" bad-config topology + (SubmitOptions.)))))))) diff --git a/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj new file mode 100644 index 00000000000..4e792402437 --- /dev/null +++ b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj @@ -0,0 +1,737 @@ +;; 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.scheduler.multitenant-scheduler-test + (:use [clojure test]) + (:use [backtype.storm bootstrap config testing]) + (:require [backtype.storm.daemon [nimbus :as nimbus]]) + (:import [backtype.storm.generated StormTopology]) + (:import [backtype.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails + SchedulerAssignmentImpl Topologies TopologyDetails]) + (:import [backtype.storm.scheduler.multitenant Node NodePool FreePool DefaultPool + IsolatedPool MultitenantScheduler])) + +(bootstrap) + +(defn gen-supervisors [count] + (into {} (for [id (range count) + :let [supervisor (SupervisorDetails. (str "super" id) (str "host" id) (list ) (map int (list 1 2 3 4)))]] + {(.getId supervisor) supervisor}))) + +(defn to-top-map [topologies] + (into {} (for [top topologies] {(.getId top) top}))) + +(defn ed [id] (ExecutorDetails. (int id) (int id))) + +(defn mk-ed-map [arg] + (into {} + (for [[name start end] arg] + (into {} + (for [at (range start end)] + {(ed at) name}))))) + +(deftest test-node + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster)] + (is (= 5 (.size node-map))) + (let [node (.get node-map "super0")] + (is (= "super0" (.getId node))) + (is (= true (.isAlive node))) + (is (= 0 (.size (.getRunningTopologies node)))) + (is (= true (.isTotallyFree node))) + (is (= 4 (.totalSlotsFree node))) + (is (= 0 (.totalSlotsUsed node))) + (is (= 4 (.totalSlots node))) + (.assign node "topology1" (list (ExecutorDetails. 1 1)) cluster) + (is (= 1 (.size (.getRunningTopologies node)))) + (is (= false (.isTotallyFree node))) + (is (= 3 (.totalSlotsFree node))) + (is (= 1 (.totalSlotsUsed node))) + (is (= 4 (.totalSlots node))) + (.assign node "topology1" (list (ExecutorDetails. 2 2)) cluster) + (is (= 1 (.size (.getRunningTopologies node)))) + (is (= false (.isTotallyFree node))) + (is (= 2 (.totalSlotsFree node))) + (is (= 2 (.totalSlotsUsed node))) + (is (= 4 (.totalSlots node))) + (.assign node "topology2" (list (ExecutorDetails. 1 1)) cluster) + (is (= 2 (.size (.getRunningTopologies node)))) + (is (= false (.isTotallyFree node))) + (is (= 1 (.totalSlotsFree node))) + (is (= 3 (.totalSlotsUsed node))) + (is (= 4 (.totalSlots node))) + (.assign node "topology2" (list (ExecutorDetails. 2 2)) cluster) + (is (= 2 (.size (.getRunningTopologies node)))) + (is (= false (.isTotallyFree node))) + (is (= 0 (.totalSlotsFree node))) + (is (= 4 (.totalSlotsUsed node))) + (is (= 4 (.totalSlots node))) + (.freeAllSlots node cluster) + (is (= 0 (.size (.getRunningTopologies node)))) + (is (= true (.isTotallyFree node))) + (is (= 4 (.totalSlotsFree node))) + (is (= 0 (.totalSlotsUsed node))) + (is (= 4 (.totalSlots node))) + ))) + +(deftest test-free-pool + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. )] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list (ExecutorDetails. 1 1)) cluster) + (.init free-pool cluster node-map) + (is (= 4 (.nodesAvailable free-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (let [ns-count-1 (.getNodeAndSlotCountIfSlotsWereTaken free-pool 1) + ns-count-3 (.getNodeAndSlotCountIfSlotsWereTaken free-pool 3) + ns-count-4 (.getNodeAndSlotCountIfSlotsWereTaken free-pool 4) + ns-count-5 (.getNodeAndSlotCountIfSlotsWereTaken free-pool 5)] + (is (= 1 (._nodes ns-count-1))) + (is (= 4 (._slots ns-count-1))) + (is (= 1 (._nodes ns-count-3))) + (is (= 4 (._slots ns-count-3))) + (is (= 1 (._nodes ns-count-4))) + (is (= 4 (._slots ns-count-4))) + (is (= 2 (._nodes ns-count-5))) + (is (= 8 (._slots ns-count-5))) + ) + (let [nodes (.takeNodesBySlots free-pool 5)] + (is (= 2 (.size nodes))) + (is (= 8 (Node/countFreeSlotsAlive nodes))) + (is (= 8 (Node/countTotalSlotsAlive nodes))) + (is (= 2 (.nodesAvailable free-pool))) + (is (= (* 2 4) (.slotsAvailable free-pool))) + ) + (let [nodes (.takeNodes free-pool 3)] ;;Only 2 should be left + (is (= 2 (.size nodes))) + (is (= 8 (Node/countFreeSlotsAlive nodes))) + (is (= 8 (Node/countTotalSlotsAlive nodes))) + (is (= 0 (.nodesAvailable free-pool))) + (is (= 0 (.slotsAvailable free-pool))) + ))) + +(deftest test-default-pool-simple + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + default-pool (DefaultPool. ) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1"} + (StormTopology.) + 2 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt2"})] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list executor1) cluster) + (.init free-pool cluster node-map) + (.init default-pool cluster node-map) + (is (= true (.canAdd default-pool topology1))) + (.addTopology default-pool topology1) + ;;Only 1 node is in the default-pool because only one nodes was scheduled already + (is (= 4 (.slotsAvailable default-pool))) + (is (= 1 (.nodesAvailable default-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (.scheduleAsNeeded default-pool (into-array NodePool [free-pool])) + (is (= 4 (.slotsAvailable default-pool))) + (is (= 1 (.nodesAvailable default-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 2 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1"))) +)) + +(deftest test-default-pool-big-request + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + default-pool (DefaultPool. ) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1"} + (StormTopology.) + 5 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt2"})] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list executor1) cluster) + (.init free-pool cluster node-map) + (.init default-pool cluster node-map) + (is (= true (.canAdd default-pool topology1))) + (.addTopology default-pool topology1) + ;;Only 1 node is in the default-pool because only one nodes was scheduled already + (is (= 4 (.slotsAvailable default-pool))) + (is (= 1 (.nodesAvailable default-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (.scheduleAsNeeded default-pool (into-array NodePool [free-pool])) + (is (= 4 (.slotsAvailable default-pool))) + (is (= 1 (.nodesAvailable default-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 3 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (is (= "Fully Scheduled (requested 5 slots, but could only use 3)" (.get (.getStatusMap cluster) "topology1"))) +)) + +(deftest test-default-pool-big-request-2 + (let [supers (gen-supervisors 1) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + default-pool (DefaultPool. ) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + executor4 (ed 4) + executor5 (ed 5) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1"} + (StormTopology.) + 5 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt1" + executor4 "bolt1" + executor5 "bolt2"})] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list executor1) cluster) + (.init free-pool cluster node-map) + (.init default-pool cluster node-map) + (is (= true (.canAdd default-pool topology1))) + (.addTopology default-pool topology1) + ;;Only 1 node is in the default-pool because only one nodes was scheduled already + (is (= 4 (.slotsAvailable default-pool))) + (is (= 1 (.nodesAvailable default-pool))) + (is (= 0 (.slotsAvailable free-pool))) + (is (= 0 (.nodesAvailable free-pool))) + (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (.scheduleAsNeeded default-pool (into-array NodePool [free-pool])) + (is (= 4 (.slotsAvailable default-pool))) + (is (= 1 (.nodesAvailable default-pool))) + (is (= 0 (.slotsAvailable free-pool))) + (is (= 0 (.nodesAvailable free-pool))) + (is (= 4 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (is (= "Running with fewer slots than requested (4/5)" (.get (.getStatusMap cluster) "topology1"))) +)) + +(deftest test-default-pool-full + (let [supers (gen-supervisors 2) ;;make 2 supervisors but only schedule with one of them + single-super {(ffirst supers) (second (first supers))} + single-cluster (Cluster. (nimbus/standalone-nimbus) single-super {}) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + executor4 (ed 4) + executor5 (ed 5) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1"} + (StormTopology.) + 5 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt2" + executor4 "bolt3" + executor5 "bolt4"})] + (let [node-map (Node/getAllNodesFrom single-cluster) + free-pool (FreePool. ) + default-pool (DefaultPool. )] + (.init free-pool single-cluster node-map) + (.init default-pool single-cluster node-map) + (.addTopology default-pool topology1) + (.scheduleAsNeeded default-pool (into-array NodePool [free-pool])) + ;; The cluster should be full and have 4 slots used, but the topology would like 1 more + (is (= 4 (.size (.getUsedSlots single-cluster)))) + (is (= "Running with fewer slots than requested (4/5)" (.get (.getStatusMap single-cluster) "topology1"))) + ) + + (let [cluster (Cluster. (nimbus/standalone-nimbus) supers (.getAssignments single-cluster)) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + default-pool (DefaultPool. )] + (.init free-pool cluster node-map) + (.init default-pool cluster node-map) + (.addTopology default-pool topology1) + (.scheduleAsNeeded default-pool (into-array NodePool [free-pool])) + ;; The cluster should now have 5 slots used + (is (= 5 (.size (.getUsedSlots cluster)))) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1"))) + ) +)) + + +(deftest test-default-pool-complex + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + default-pool (DefaultPool. ) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + executor11 (ed 11) + executor12 (ed 12) + executor13 (ed 13) + executor14 (ed 14) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1"} + (StormTopology.) + 2 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt2"}) + topology2 (TopologyDetails. "topology2" + {TOPOLOGY-NAME "topology-name-2"} + (StormTopology.) + 4 + {executor11 "spout11" + executor12 "bolt12" + executor13 "bolt13" + executor14 "bolt14"})] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list executor1) cluster) + (.init free-pool cluster node-map) + (.init default-pool cluster node-map) + (is (= true (.canAdd default-pool topology1))) + (.addTopology default-pool topology1) + (is (= true (.canAdd default-pool topology2))) + (.addTopology default-pool topology2) + ;;Only 1 node is in the default-pool because only one nodes was scheduled already + (is (= 4 (.slotsAvailable default-pool))) + (is (= 1 (.nodesAvailable default-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (is (= nil (.getAssignmentById cluster "topology2"))) + (.scheduleAsNeeded default-pool (into-array NodePool [free-pool])) + ;;We steal a node from the free pool to handle the extra + (is (= 8 (.slotsAvailable default-pool))) + (is (= 2 (.nodesAvailable default-pool))) + (is (= (* 3 4) (.slotsAvailable free-pool))) + (is (= 3 (.nodesAvailable free-pool))) + (is (= 2 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (is (= 4 (.size (.getSlots (.getAssignmentById cluster "topology2"))))) + (let [ns-count-1 (.getNodeAndSlotCountIfSlotsWereTaken default-pool 1) + ns-count-3 (.getNodeAndSlotCountIfSlotsWereTaken default-pool 3) + ns-count-4 (.getNodeAndSlotCountIfSlotsWereTaken default-pool 4) + ns-count-5 (.getNodeAndSlotCountIfSlotsWereTaken default-pool 5)] + (is (= 1 (._nodes ns-count-1))) + (is (= 4 (._slots ns-count-1))) + (is (= 1 (._nodes ns-count-3))) + (is (= 4 (._slots ns-count-3))) + (is (= 1 (._nodes ns-count-4))) + (is (= 4 (._slots ns-count-4))) + (is (= 2 (._nodes ns-count-5))) + (is (= 8 (._slots ns-count-5))) + ) + (let [nodes (.takeNodesBySlots default-pool 3)] + (is (= 1 (.size nodes))) + (is (= 4 (Node/countFreeSlotsAlive nodes))) + (is (= 4 (Node/countTotalSlotsAlive nodes))) + (is (= 1 (.nodesAvailable default-pool))) + (is (= (* 1 4) (.slotsAvailable default-pool))) + ) + (let [nodes (.takeNodes default-pool 3)] ;;Only 1 should be left + (is (= 1 (.size nodes))) + (is (= 4 (Node/countFreeSlotsAlive nodes))) + (is (= 4 (Node/countTotalSlotsAlive nodes))) + (is (= 0 (.nodesAvailable default-pool))) + (is (= 0 (.slotsAvailable default-pool))) + ) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1"))) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2"))) +)) + +(deftest test-isolated-pool-simple + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + isolated-pool (IsolatedPool. 5) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + executor4 (ed 4) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1" + TOPOLOGY-ISOLATED-MACHINES 4} + (StormTopology.) + 4 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt2" + executor4 "bolt4"})] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list executor1) cluster) + (.init free-pool cluster node-map) + (.init isolated-pool cluster node-map) + (is (= true (.canAdd isolated-pool topology1))) + (.addTopology isolated-pool topology1) + ;;Isolated topologies cannot have their resources stolen + (is (= 0 (.slotsAvailable isolated-pool))) + (is (= 0 (.nodesAvailable isolated-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (.scheduleAsNeeded isolated-pool (into-array NodePool [free-pool])) + (is (= 0 (.slotsAvailable isolated-pool))) + (is (= 0 (.nodesAvailable isolated-pool))) + (is (= (* 1 4) (.slotsAvailable free-pool))) + (is (= 1 (.nodesAvailable free-pool))) + (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))] + ;; 4 slots on 4 machines + (is (= 4 (.size assigned-slots))) + (is (= 4 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + ) + (is (= "Scheduled Isolated on 4 Nodes" (.get (.getStatusMap cluster) "topology1"))) +)) + +(deftest test-isolated-pool-big-ask + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + isolated-pool (IsolatedPool. 5) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + executor4 (ed 4) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1" + TOPOLOGY-ISOLATED-MACHINES 4} + (StormTopology.) + 10 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt2" + executor4 "bolt4"})] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list executor1) cluster) + (.init free-pool cluster node-map) + (.init isolated-pool cluster node-map) + (is (= true (.canAdd isolated-pool topology1))) + (.addTopology isolated-pool topology1) + ;;Isolated topologies cannot have their resources stolen + (is (= 0 (.slotsAvailable isolated-pool))) + (is (= 0 (.nodesAvailable isolated-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (.scheduleAsNeeded isolated-pool (into-array NodePool [free-pool])) + (is (= 0 (.slotsAvailable isolated-pool))) + (is (= 0 (.nodesAvailable isolated-pool))) + (is (= (* 1 4) (.slotsAvailable free-pool))) + (is (= 1 (.nodesAvailable free-pool))) + (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))] + ;; 4 slots on 4 machines + (is (= 4 (.size assigned-slots))) + (is (= 4 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + ) + (is (= "Scheduled Isolated on 4 Nodes" (.get (.getStatusMap cluster) "topology1"))) +)) + +(deftest test-isolated-pool-complex + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + isolated-pool (IsolatedPool. 5) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + executor4 (ed 4) + executor11 (ed 11) + executor12 (ed 12) + executor13 (ed 13) + executor14 (ed 14) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1"} + (StormTopology.) + 4 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt2" + executor4 "bolt4"}) + topology2 (TopologyDetails. "topology2" + {TOPOLOGY-NAME "topology-name-2" + TOPOLOGY-ISOLATED-MACHINES 2} + (StormTopology.) + 4 + {executor11 "spout11" + executor12 "bolt12" + executor13 "bolt13" + executor14 "bolt14"})] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list executor1) cluster) + (.init free-pool cluster node-map) + (.init isolated-pool cluster node-map) + (is (= true (.canAdd isolated-pool topology1))) + (.addTopology isolated-pool topology1) + (is (= true (.canAdd isolated-pool topology2))) + (.addTopology isolated-pool topology2) + ;; nodes can be stolen from non-isolted tops in the pool + (is (= 4 (.slotsAvailable isolated-pool))) + (is (= 1 (.nodesAvailable isolated-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (is (= nil (.getAssignmentById cluster "topology2"))) + (.scheduleAsNeeded isolated-pool (into-array NodePool [free-pool])) + ;;We steal 2 nodes from the free pool to handle the extra (but still only 1 node for the non-isolated top + (is (= 4 (.slotsAvailable isolated-pool))) + (is (= 1 (.nodesAvailable isolated-pool))) + (is (= (* 2 4) (.slotsAvailable free-pool))) + (is (= 2 (.nodesAvailable free-pool))) + (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))] + ;; 4 slots on 1 machine + (is (= 4 (.size assigned-slots))) + (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + ) + (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology2"))] + ;; 4 slots on 2 machines + (is (= 4 (.size assigned-slots))) + (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + ) + + (let [ns-count-1 (.getNodeAndSlotCountIfSlotsWereTaken isolated-pool 1) + ns-count-3 (.getNodeAndSlotCountIfSlotsWereTaken isolated-pool 3) + ns-count-4 (.getNodeAndSlotCountIfSlotsWereTaken isolated-pool 4) + ns-count-5 (.getNodeAndSlotCountIfSlotsWereTaken isolated-pool 5)] + (is (= 1 (._nodes ns-count-1))) + (is (= 4 (._slots ns-count-1))) + (is (= 1 (._nodes ns-count-3))) + (is (= 4 (._slots ns-count-3))) + (is (= 1 (._nodes ns-count-4))) + (is (= 4 (._slots ns-count-4))) + (is (= 1 (._nodes ns-count-5))) ;;Only 1 node can be stolen right now + (is (= 4 (._slots ns-count-5))) + ) + (let [nodes (.takeNodesBySlots isolated-pool 3)] + (is (= 1 (.size nodes))) + (is (= 4 (Node/countFreeSlotsAlive nodes))) + (is (= 4 (Node/countTotalSlotsAlive nodes))) + (is (= 0 (.nodesAvailable isolated-pool))) + (is (= (* 0 4) (.slotsAvailable isolated-pool))) + ) + (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))] + ;; 4 slots on 1 machine + (is (= 0 (.size assigned-slots))) + (is (= 0 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + ) + (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology2"))] + ;; 4 slots on 2 machines + (is (= 4 (.size assigned-slots))) + (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + ) + (let [nodes (.takeNodes isolated-pool 3)] ;;Cannot steal from the isolated scheduler + (is (= 0 (.size nodes))) + (is (= 0 (Node/countFreeSlotsAlive nodes))) + (is (= 0 (Node/countTotalSlotsAlive nodes))) + (is (= 0 (.nodesAvailable isolated-pool))) + (is (= 0 (.slotsAvailable isolated-pool))) + ) + (is (= "Scheduled Isolated on 1 Nodes" (.get (.getStatusMap cluster) "topology1"))) + (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2"))) +)) + +(deftest test-isolated-pool-complex-2 + (let [supers (gen-supervisors 5) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + free-pool (FreePool. ) + ;;like before but now we can only hold 2 nodes max. Don't go over + isolated-pool (IsolatedPool. 2) + executor1 (ed 1) + executor2 (ed 2) + executor3 (ed 3) + executor4 (ed 4) + executor11 (ed 11) + executor12 (ed 12) + executor13 (ed 13) + executor14 (ed 14) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1"} + (StormTopology.) + 4 + {executor1 "spout1" + executor2 "bolt1" + executor3 "bolt2" + executor4 "bolt4"}) + topology2 (TopologyDetails. "topology2" + {TOPOLOGY-NAME "topology-name-2" + TOPOLOGY-ISOLATED-MACHINES 2} + (StormTopology.) + 4 + {executor11 "spout11" + executor12 "bolt12" + executor13 "bolt13" + executor14 "bolt14"})] + ;; assign one node so it is not in the pool + (.assign (.get node-map "super0") "topology1" (list executor1) cluster) + (.init free-pool cluster node-map) + (.init isolated-pool cluster node-map) + (is (= true (.canAdd isolated-pool topology1))) + (.addTopology isolated-pool topology1) + (is (= true (.canAdd isolated-pool topology2))) + (.addTopology isolated-pool topology2) + ;; nodes can be stolen from non-isolted tops in the pool + (is (= 4 (.slotsAvailable isolated-pool))) + (is (= 1 (.nodesAvailable isolated-pool))) + (is (= (* 4 4) (.slotsAvailable free-pool))) + (is (= 4 (.nodesAvailable free-pool))) + (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1"))))) + (is (= nil (.getAssignmentById cluster "topology2"))) + (.scheduleAsNeeded isolated-pool (into-array NodePool [free-pool])) + ;;We steal 1 node from the free pool and 1 from ourself to handle the extra + (is (= 0 (.slotsAvailable isolated-pool))) + (is (= 0 (.nodesAvailable isolated-pool))) + (is (= (* 3 4) (.slotsAvailable free-pool))) + (is (= 3 (.nodesAvailable free-pool))) + (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))] + ;; 0 slots on 0 machine + (is (= 0 (.size assigned-slots))) + (is (= 0 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + ) + (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology2"))] + ;; 4 slots on 2 machines + (is (= 4 (.size assigned-slots))) + (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + ) + (is (= "Max Nodes(2) for this user would be exceeded. 1 more nodes needed to run topology." (.get (.getStatusMap cluster) "topology1"))) + (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2"))) +)) + +(deftest test-multitenant-scheduler + (let [supers (gen-supervisors 10) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1" + TOPOLOGY-SUBMITTER-USER "userC"} + (StormTopology.) + 4 + (mk-ed-map [["spout1" 0 5] + ["bolt1" 5 10] + ["bolt2" 10 15] + ["bolt3" 15 20]])) + topology2 (TopologyDetails. "topology2" + {TOPOLOGY-NAME "topology-name-2" + TOPOLOGY-ISOLATED-MACHINES 2 + TOPOLOGY-SUBMITTER-USER "userA"} + (StormTopology.) + 4 + (mk-ed-map [["spout11" 0 5] + ["bolt12" 5 6] + ["bolt13" 6 7] + ["bolt14" 7 10]])) + topology3 (TopologyDetails. "topology3" + {TOPOLOGY-NAME "topology-name-3" + TOPOLOGY-ISOLATED-MACHINES 5 + TOPOLOGY-SUBMITTER-USER "userB"} + (StormTopology.) + 10 + (mk-ed-map [["spout21" 0 10] + ["bolt22" 10 20] + ["bolt23" 20 30] + ["bolt24" 30 40]])) + cluster (Cluster. (nimbus/standalone-nimbus) supers {}) + node-map (Node/getAllNodesFrom cluster) + topologies (Topologies. (to-top-map [topology1 topology2 topology3])) + conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}} + scheduler (MultitenantScheduler.)] + (.assign (.get node-map "super0") "topology1" (list (ed 1)) cluster) + (.assign (.get node-map "super1") "topology2" (list (ed 5)) cluster) + (.prepare scheduler conf) + (.schedule scheduler topologies cluster) + (let [assignment (.getAssignmentById cluster "topology1") + assigned-slots (.getSlots assignment) + executors (.getExecutors assignment)] + ;; 4 slots on 1 machine, all executors assigned + (is (= 4 (.size assigned-slots))) + (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + (is (= 20 (.size executors))) + ) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1"))) + (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2"))) + (is (= "Scheduled Isolated on 5 Nodes" (.get (.getStatusMap cluster) "topology3"))) +)) + + +(deftest test-multitenant-scheduler-bad-starting-state + (let [supers (gen-supervisors 10) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1" + TOPOLOGY-SUBMITTER-USER "userC"} + (StormTopology.) + 4 + (mk-ed-map [["spout1" 0 5] + ["bolt1" 5 10] + ["bolt2" 10 15] + ["bolt3" 15 20]])) + topology2 (TopologyDetails. "topology2" + {TOPOLOGY-NAME "topology-name-2" + TOPOLOGY-ISOLATED-MACHINES 2 + TOPOLOGY-SUBMITTER-USER "userA"} + (StormTopology.) + 4 + (mk-ed-map [["spout11" 0 5] + ["bolt12" 5 6] + ["bolt13" 6 7] + ["bolt14" 7 10]])) + topology3 (TopologyDetails. "topology3" + {TOPOLOGY-NAME "topology-name-3" + TOPOLOGY-ISOLATED-MACHINES 5 + TOPOLOGY-SUBMITTER-USER "userB"} + (StormTopology.) + 10 + (mk-ed-map [["spout21" 0 10] + ["bolt22" 10 20] + ["bolt23" 20 30] + ["bolt24" 30 40]])) + existing-assignments { + "topology2" (SchedulerAssignmentImpl. "topology2" {(ExecutorDetails. 0 5) (WorkerSlot. "super1" 1)}) + "topology3" (SchedulerAssignmentImpl. "topology3" {(ExecutorDetails. 0 10) (WorkerSlot. "super1" 1)}) + } + cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments) + topologies (Topologies. (to-top-map [topology1 topology2 topology3])) + conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}} + scheduler (MultitenantScheduler.)] + (.prepare scheduler conf) + (.schedule scheduler topologies cluster) + (let [assignment (.getAssignmentById cluster "topology1") + assigned-slots (.getSlots assignment) + executors (.getExecutors assignment)] + ;; 4 slots on 1 machine, all executors assigned + (is (= 4 (.size assigned-slots))) + (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + (is (= 20 (.size executors))) + ) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1"))) + (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2"))) + (is (= "Scheduled Isolated on 5 Nodes" (.get (.getStatusMap cluster) "topology3"))) +)) + + diff --git a/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj b/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj index ed21904201d..2d96b18ff00 100644 --- a/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj +++ b/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj @@ -14,13 +14,16 @@ ;; See the License for the specific language governing permissions and ;; limitations under the License. (ns backtype.storm.security.auth.AuthUtils-test - (:import [backtype.storm.security.auth AuthUtils]) + (:import [backtype.storm.security.auth AuthUtils IAutoCredentials]) (:import [java.io IOException]) (:import [javax.security.auth.login AppConfigurationEntry Configuration]) (:import [org.mockito Mockito]) (:use [clojure test]) + (:use [backtype.storm bootstrap]) ) +(bootstrap) + (deftest test-throws-on-missing-section (is (thrown? IOException (AuthUtils/get (Mockito/mock Configuration) "bogus-section" ""))) @@ -61,5 +64,16 @@ (is (not (nil? (AuthUtils/get conf section k)))) (is (= (AuthUtils/get conf section k) expected)) ) + )) + +(deftest test-empty-auto-creds + (let [result (AuthUtils/GetAutoCredentials {})] + (is (.isEmpty result)) + ) +) + +(deftest test-empty-creds-renewers + (let [result (AuthUtils/GetCredentialRenewers {})] + (is (.isEmpty result)) ) ) diff --git a/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj b/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj new file mode 100644 index 00000000000..ab54d822fe7 --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj @@ -0,0 +1,40 @@ +(ns backtype.storm.security.auth.DefaultHttpCredentialsPlugin-test + (:use [clojure test]) + (:import [javax.security.auth Subject]) + (:import [javax.servlet.http HttpServletRequest]) + (:import [org.mockito Mockito]) + (:import [backtype.storm.security.auth DefaultHttpCredentialsPlugin + ReqContext SingleUserPrincipal]) + ) + +(deftest test-getUserName + (let [handler (doto (DefaultHttpCredentialsPlugin.) (.prepare {}))] + (testing "returns null when request is null" + (is (nil? (.getUserName handler nil)))) + + (testing "returns null when user principal is null" + (let [req (Mockito/mock HttpServletRequest)] + (is (nil? (.getUserName handler req))))) + + (testing "returns null when user is blank" + (let [princ (SingleUserPrincipal. "") + req (Mockito/mock HttpServletRequest)] + (. (Mockito/when (. req getUserPrincipal)) + thenReturn princ) + (is (nil? (.getUserName handler req))))) + + (testing "returns correct user from requests principal" + (let [exp-name "Alice" + princ (SingleUserPrincipal. exp-name) + req (Mockito/mock HttpServletRequest)] + (. (Mockito/when (. req getUserPrincipal)) + thenReturn princ) + (is (.equals exp-name (.getUserName handler req))))))) + +(deftest test-populate-req-context-noop-on-null-user + (let [req (Mockito/mock HttpServletRequest) + handler (doto (DefaultHttpCredentialsPlugin.) (.prepare {})) + expected-subj (Subject.) + context (ReqContext. expected-subj)] + (is (.equals expected-subj + (-> handler (.populateContext context req) (.subject)))))) diff --git a/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj b/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj index 7dcd86d2b19..2eee963f284 100644 --- a/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj +++ b/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj @@ -14,28 +14,32 @@ ;; See the License for the specific language governing permissions and ;; limitations under the License. (ns backtype.storm.security.auth.ThriftClient-test - (:use [backtype.storm config]) + (:use [backtype.storm config util]) (:use [clojure test]) - (:import [backtype.storm.security.auth ThriftClient]) + (:import [backtype.storm.security.auth ThriftClient ThriftConnectionType]) (:import [org.apache.thrift.transport TTransportException]) ) (deftest test-ctor-throws-if-port-invalid - (let [conf (read-default-config) + (let [conf (merge + (read-default-config) + {STORM-NIMBUS-RETRY-TIMES 0}) timeout (Integer. 30)] - (is (thrown? java.lang.IllegalArgumentException - (ThriftClient. conf "bogushost" -1 timeout))) - (is (thrown? java.lang.IllegalArgumentException - (ThriftClient. conf "bogushost" 0 timeout))) + (is (thrown-cause? java.lang.IllegalArgumentException + (ThriftClient. conf ThriftConnectionType/DRPC "bogushost" (int -1) timeout))) + (is (thrown-cause? java.lang.IllegalArgumentException + (ThriftClient. conf ThriftConnectionType/DRPC "bogushost" (int 0) timeout))) ) ) (deftest test-ctor-throws-if-host-not-set - (let [conf (read-default-config) + (let [conf (merge + (read-default-config) + {STORM-NIMBUS-RETRY-TIMES 0}) timeout (Integer. 60)] - (is (thrown? TTransportException - (ThriftClient. conf "" 4242 timeout))) - (is (thrown? IllegalArgumentException - (ThriftClient. conf nil 4242 timeout))) + (is (thrown-cause? TTransportException + (ThriftClient. conf ThriftConnectionType/DRPC "" (int 4242) timeout))) + (is (thrown-cause? IllegalArgumentException + (ThriftClient. conf ThriftConnectionType/DRPC nil (int 4242) timeout))) ) ) diff --git a/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj b/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj index 6213d4fffd7..c8ed70e55bf 100644 --- a/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj +++ b/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj @@ -16,14 +16,16 @@ (ns backtype.storm.security.auth.ThriftServer-test (:use [backtype.storm config]) (:use [clojure test]) - (:import [backtype.storm.security.auth ThriftServer]) + (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType]) (:import [org.apache.thrift.transport TTransportException]) ) (deftest test-stop-checks-for-null - (let [server (ThriftServer. (read-default-config) nil 12345)] + (let [server (ThriftServer. (read-default-config) nil + ThriftConnectionType/DRPC)] (.stop server))) (deftest test-isServing-checks-for-null - (let [server (ThriftServer. (read-default-config) nil 12345)] + (let [server (ThriftServer. (read-default-config) nil + ThriftConnectionType/DRPC)] (is (not (.isServing server))))) diff --git a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj index c7d7411cf05..6fdd485c039 100644 --- a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj +++ b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj @@ -19,36 +19,40 @@ (:import [org.apache.thrift TException]) (:import [org.apache.thrift.transport TTransportException]) (:import [java.nio ByteBuffer]) + (:import [java.security Principal AccessController]) + (:import [javax.security.auth Subject]) + (:import [java.net InetAddress]) (:import [backtype.storm Config]) + (:import [backtype.storm.generated AuthorizationException]) (:import [backtype.storm.utils NimbusClient]) + (:import [backtype.storm.security.auth.authorizer SimpleWhitelistAuthorizer SimpleACLAuthorizer]) (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient - ReqContext]) + ReqContext SimpleTransportPlugin KerberosPrincipalToLocal ThriftConnectionType]) (:use [backtype.storm bootstrap util]) (:use [backtype.storm.daemon common]) (:use [backtype.storm bootstrap testing]) - (:import [backtype.storm.generated Nimbus Nimbus$Client]) - ) + (:import [backtype.storm.generated Nimbus Nimbus$Client])) (bootstrap) -(def nimbus-timeout (Integer. 120)) +(defn mk-principal [name] + (reify Principal + (equals [this other] + (= name (.getName other))) + (getName [this] name) + (toString [this] name) + (hashCode [this] (.hashCode name)))) + +(defn mk-subject [name] + (Subject. true #{(mk-principal name)} #{} #{})) -(defn mk-authorization-handler [storm-conf] - (let [klassname (storm-conf NIMBUS-AUTHORIZER) - aznClass (if klassname (Class/forName klassname)) - aznHandler (if aznClass (.newInstance aznClass))] - (if aznHandler (.prepare aznHandler storm-conf)) - (log-debug "authorization class name:" klassname - " class:" aznClass - " handler:" aznHandler) - aznHandler - )) +(def nimbus-timeout (Integer. 120)) (defn nimbus-data [storm-conf inimbus] (let [forced-scheduler (.getForcedScheduler inimbus)] {:conf storm-conf :inimbus inimbus - :authorization-handler (mk-authorization-handler storm-conf) + :authorization-handler (mk-authorization-handler (storm-conf NIMBUS-AUTHORIZER) storm-conf) :submitted-count (atom 0) :storm-cluster-state nil :submit-lock (Object.) @@ -61,71 +65,75 @@ :scheduler nil })) -(defn check-authorization! [nimbus storm-name storm-conf operation] - (let [aclHandler (:authorization-handler nimbus)] - (log-debug "check-authorization with handler: " aclHandler) - (if aclHandler - (if-not (.permit aclHandler - (ReqContext/context) - operation - (if storm-conf storm-conf {TOPOLOGY-NAME storm-name})) - (throw (RuntimeException. (str operation " on topology " storm-name " is not authorized"))) - )))) - -(defn dummy-service-handler [conf inimbus] - (let [nimbus (nimbus-data conf inimbus)] - (reify Nimbus$Iface - (^void submitTopologyWithOpts [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology - ^SubmitOptions submitOptions] - (check-authorization! nimbus storm-name nil "submitTopology")) - - (^void killTopology [this ^String storm-name] - (check-authorization! nimbus storm-name nil "killTopology")) - - (^void killTopologyWithOpts [this ^String storm-name ^KillOptions options] - (check-authorization! nimbus storm-name nil "killTopology")) - - (^void rebalance [this ^String storm-name ^RebalanceOptions options] - (check-authorization! nimbus storm-name nil "rebalance")) - - (activate [this storm-name] - (check-authorization! nimbus storm-name nil "activate")) - - (deactivate [this storm-name] - (check-authorization! nimbus storm-name nil "deactivate")) - - (beginFileUpload [this]) - - (^void uploadChunk [this ^String location ^ByteBuffer chunk]) - - (^void finishFileUpload [this ^String location]) - - (^String beginFileDownload [this ^String file]) - - (^ByteBuffer downloadChunk [this ^String id]) - - (^String getNimbusConf [this]) +(defn dummy-service-handler + ([conf inimbus auth-context] + (let [nimbus-d (nimbus-data conf inimbus) + topo-conf (atom nil)] + (reify Nimbus$Iface + (^void submitTopologyWithOpts [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology + ^SubmitOptions submitOptions] + (if (not (nil? serializedConf)) (swap! topo-conf (fn [prev new] new) (from-json serializedConf))) + (nimbus/check-authorization! nimbus-d storm-name @topo-conf "submitTopology" auth-context)) + + (^void killTopology [this ^String storm-name] + (nimbus/check-authorization! nimbus-d storm-name @topo-conf "killTopology" auth-context)) + + (^void killTopologyWithOpts [this ^String storm-name ^KillOptions options] + (nimbus/check-authorization! nimbus-d storm-name @topo-conf "killTopology" auth-context)) + + (^void rebalance [this ^String storm-name ^RebalanceOptions options] + (nimbus/check-authorization! nimbus-d storm-name @topo-conf "rebalance" auth-context)) + + (activate [this storm-name] + (nimbus/check-authorization! nimbus-d storm-name @topo-conf "activate" auth-context)) + + (deactivate [this storm-name] + (nimbus/check-authorization! nimbus-d storm-name @topo-conf "deactivate" auth-context)) - (^String getTopologyConf [this ^String id]) + (uploadNewCredentials [this storm-name creds] + (nimbus/check-authorization! nimbus-d storm-name @topo-conf "uploadNewCredentials" auth-context)) + + (beginFileUpload [this]) + + (^void uploadChunk [this ^String location ^ByteBuffer chunk]) + + (^void finishFileUpload [this ^String location]) + + (^String beginFileDownload [this ^String file] + (nimbus/check-authorization! nimbus-d nil nil "fileDownload" auth-context) + "Done!") + + (^ByteBuffer downloadChunk [this ^String id]) + + (^String getNimbusConf [this]) + + (^String getTopologyConf [this ^String id]) + + (^StormTopology getTopology [this ^String id]) + + (^StormTopology getUserTopology [this ^String id]) + + (^ClusterSummary getClusterInfo [this]) + + (^TopologyInfo getTopologyInfo [this ^String storm-id])))) + ([conf inimbus] + (dummy-service-handler conf inimbus nil))) + - (^StormTopology getTopology [this ^String id]) - - (^StormTopology getUserTopology [this ^String id]) - - (^ClusterSummary getClusterInfo [this]) - - (^TopologyInfo getTopologyInfo [this ^String storm-id])))) - -(defn launch-server [server-port login-cfg aznClass transportPluginClass] +(defn launch-server [server-port login-cfg aznClass transportPluginClass serverConf] (let [conf1 (merge (read-storm-config) - {NIMBUS-AUTHORIZER aznClass - NIMBUS-HOST "localhost" - NIMBUS-THRIFT-PORT server-port - STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass}) - conf (if login-cfg (merge conf1 {"java.security.auth.login.config" login-cfg}) conf1) + {NIMBUS-AUTHORIZER aznClass + NIMBUS-HOST "localhost" + NIMBUS-THRIFT-PORT server-port + STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass}) + conf2 (if login-cfg (merge conf1 {"java.security.auth.login.config" login-cfg}) conf1) + conf (if serverConf (merge conf2 serverConf) conf2) nimbus (nimbus/standalone-nimbus) service-handler (dummy-service-handler conf nimbus) - server (ThriftServer. conf (Nimbus$Processor. service-handler) (int (conf NIMBUS-THRIFT-PORT)))] + server (ThriftServer. + conf + (Nimbus$Processor. service-handler) + ThriftConnectionType/NIMBUS)] (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop server)))) (.start (Thread. #(.serve server))) (wait-for-condition #(.isServing server)) @@ -133,32 +141,172 @@ (defmacro with-server [args & body] `(let [server# (launch-server ~@args)] - ~@body - (.stop server#) - )) + ~@body + (.stop server#) + )) -(deftest Simple-authentication-test +(deftest kerb-to-local-test + (let [kptol (KerberosPrincipalToLocal. )] + (.prepare kptol {}) + (is (= "me" (.toLocal kptol (mk-principal "me@realm")))) + (is (= "simple" (.toLocal kptol (mk-principal "simple")))) + (is (= "someone" (.toLocal kptol (mk-principal "someone/host@realm")))))) + +(deftest Simple-authentication-test (let [a-port (available-port)] - (with-server [a-port nil nil "backtype.storm.security.auth.SimpleTransportPlugin"] + (with-server [a-port nil nil "backtype.storm.security.auth.SimpleTransportPlugin" nil] (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"}) client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) nimbus_client (.getClient client)] (.activate nimbus_client "security_auth_test_topology") (.close client)) - + (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" - "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"})] + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf" + STORM-NIMBUS-RETRY-TIMES 0})] (testing "(Negative authentication) Server: Simple vs. Client: Digest" (is (thrown-cause? java.net.SocketTimeoutException (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))))) +(deftest negative-whitelist-authorization-test + (let [a-port (available-port)] + (with-server [a-port nil + "backtype.storm.security.auth.authorizer.SimpleWhitelistAuthorizer" + "backtype.storm.testing.SingleUserSimpleTransport" nil] + (let [storm-conf (merge (read-storm-config) + {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.testing.SingleUserSimpleTransport"}) + client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) + nimbus_client (.getClient client)] + (testing "(Negative authorization) Authorization plugin should reject client request" + (is (thrown-cause? AuthorizationException + (.activate nimbus_client "security_auth_test_topology")))) + (.close client))))) + +(deftest positive-whitelist-authorization-test + (let [a-port (available-port)] + (with-server [a-port nil + "backtype.storm.security.auth.authorizer.SimpleWhitelistAuthorizer" + "backtype.storm.testing.SingleUserSimpleTransport" {SimpleWhitelistAuthorizer/WHITELIST_USERS_CONF ["user"]}] + (let [storm-conf (merge (read-storm-config) + {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.testing.SingleUserSimpleTransport"}) + client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) + nimbus_client (.getClient client)] + (testing "(Positive authorization) Authorization plugin should accept client request" + (.activate nimbus_client "security_auth_test_topology")) + (.close client))))) + +(deftest simple-acl-user-auth-test + (let [cluster-conf (merge (read-storm-config) + {NIMBUS-ADMINS ["admin"] + NIMBUS-SUPERVISOR-USERS ["supervisor"]}) + authorizer (SimpleACLAuthorizer. ) + admin-user (mk-subject "admin") + supervisor-user (mk-subject "supervisor") + user-a (mk-subject "user-a") + user-b (mk-subject "user-b")] + (.prepare authorizer cluster-conf) + (is (= true (.permit authorizer (ReqContext. user-a) "submitTopology" {}))) + (is (= true (.permit authorizer (ReqContext. user-b) "submitTopology" {}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "submitTopology" {}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "submitTopology" {}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "fileUpload" nil))) + (is (= true (.permit authorizer (ReqContext. user-b) "fileUpload" nil))) + (is (= true (.permit authorizer (ReqContext. admin-user) "fileUpload" nil))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "fileUpload" nil))) + + (is (= true (.permit authorizer (ReqContext. user-a) "getNimbusConf" nil))) + (is (= true (.permit authorizer (ReqContext. user-b) "getNimbusConf" nil))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getNimbusConf" nil))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "getNimbusConf" nil))) + + (is (= true (.permit authorizer (ReqContext. user-a) "getClusterInfo" nil))) + (is (= true (.permit authorizer (ReqContext. user-b) "getClusterInfo" nil))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getClusterInfo" nil))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "getClusterInfo" nil))) + + (is (= false (.permit authorizer (ReqContext. user-a) "fileDownload" nil))) + (is (= false (.permit authorizer (ReqContext. user-b) "fileDownload" nil))) + (is (= true (.permit authorizer (ReqContext. admin-user) "fileDownload" nil))) + (is (= true (.permit authorizer (ReqContext. supervisor-user) "fileDownload" nil))) + + (is (= true (.permit authorizer (ReqContext. user-a) "killTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "killTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "killTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "killTopolgy" {TOPOLOGY-USERS ["user-a"]}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "rebalance" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "rebalance" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "rebalance" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "rebalance" {TOPOLOGY-USERS ["user-a"]}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "activate" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "activate" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "activate" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "activate" {TOPOLOGY-USERS ["user-a"]}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "deactivate" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "deactivate" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "deactivate" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "deactivate" {TOPOLOGY-USERS ["user-a"]}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "getTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "getTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "getTopology" {TOPOLOGY-USERS ["user-a"]}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "getUserTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "getUserTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getUserTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "getUserTopology" {TOPOLOGY-USERS ["user-a"]}))) + + (is (= true (.permit authorizer (ReqContext. user-a) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. user-b) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]}))) + (is (= false (.permit authorizer (ReqContext. supervisor-user) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]}))) +)) + +(deftest simple-acl-same-user-auth-test + (let [cluster-conf (merge (read-storm-config) + {NIMBUS-ADMINS ["admin"] + NIMBUS-SUPERVISOR-USERS ["admin"]}) + authorizer (SimpleACLAuthorizer. ) + admin-user (mk-subject "admin")] + (.prepare authorizer cluster-conf) + (is (= true (.permit authorizer (ReqContext. admin-user) "submitTopology" {}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "fileUpload" nil))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getNimbusConf" nil))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getClusterInfo" nil))) + (is (= true (.permit authorizer (ReqContext. admin-user) "fileDownload" nil))) + (is (= true (.permit authorizer (ReqContext. admin-user) "killTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "rebalance" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "activate" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "deactivate" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getUserTopology" {TOPOLOGY-USERS ["user-a"]}))) + (is (= true (.permit authorizer (ReqContext. admin-user) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]}))) +)) + + (deftest positive-authorization-test (let [a-port (available-port)] - (with-server [a-port nil + (with-server [a-port nil "backtype.storm.security.auth.authorizer.NoopAuthorizer" - "backtype.storm.security.auth.SimpleTransportPlugin"] + "backtype.storm.security.auth.SimpleTransportPlugin" nil] (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"}) client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) @@ -171,7 +319,7 @@ (let [a-port (available-port)] (with-server [a-port nil "backtype.storm.security.auth.authorizer.DenyAuthorizer" - "backtype.storm.security.auth.SimpleTransportPlugin"] + "backtype.storm.security.auth.SimpleTransportPlugin" nil] (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin" Config/NIMBUS_HOST "localhost" @@ -180,19 +328,20 @@ client (NimbusClient/getConfiguredClient storm-conf) nimbus_client (.getClient client)] (testing "(Negative authorization) Authorization plugin should reject client request" - (is (thrown? TTransportException - (.activate nimbus_client "security_auth_test_topology")))) + (is (thrown-cause? AuthorizationException + (.activate nimbus_client "security_auth_test_topology")))) (.close client))))) (deftest digest-authentication-test (let [a-port (available-port)] - (with-server [a-port + (with-server [a-port "test/clj/backtype/storm/security/auth/jaas_digest.conf" nil - "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"] + "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" nil] (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" - "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"}) + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf" + STORM-NIMBUS-RETRY-TIMES 0}) client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) nimbus_client (.getClient client)] (testing "(Positive authentication) valid digest authentication" @@ -200,7 +349,8 @@ (.close client)) (let [storm-conf (merge (read-storm-config) - {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"}) + {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin" + STORM-NIMBUS-RETRY-TIMES 0}) client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout) nimbus_client (.getClient client)] (testing "(Negative authentication) Server: Digest vs. Client: Simple" @@ -210,34 +360,37 @@ (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" - "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_bad_password.conf"})] + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_bad_password.conf" + STORM-NIMBUS-RETRY-TIMES 0})] (testing "(Negative authentication) Invalid password" - (is (thrown? TTransportException - (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) + (is (thrown-cause? TTransportException + (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" - "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_unknown_user.conf"})] + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_unknown_user.conf" + STORM-NIMBUS-RETRY-TIMES 0})] (testing "(Negative authentication) Unknown user" - (is (thrown? TTransportException - (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) - + (is (thrown-cause? TTransportException + (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) + (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" - "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/nonexistent.conf"})] + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/nonexistent.conf" + STORM-NIMBUS-RETRY-TIMES 0})] (testing "(Negative authentication) nonexistent configuration file" - (is (thrown? RuntimeException - (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) + (is (thrown-cause? RuntimeException + (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))) (let [storm-conf (merge (read-storm-config) {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" - "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_missing_client.conf"})] + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_missing_client.conf" + STORM-NIMBUS-RETRY-TIMES 0})] (testing "(Negative authentication) Missing client" (is (thrown-cause? java.io.IOException (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))))) - - + (deftest test-GetTransportPlugin-throws-RuntimeException (let [conf (merge (read-storm-config) {Config/STORM_THRIFT_TRANSPORT_PLUGIN "null.invalid"})] - (is (thrown? RuntimeException (AuthUtils/GetTransportPlugin conf nil))))) + (is (thrown-cause? RuntimeException (AuthUtils/GetTransportPlugin conf nil nil))))) diff --git a/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj b/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj new file mode 100644 index 00000000000..d592a400fca --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj @@ -0,0 +1,226 @@ +(ns backtype.storm.security.auth.authorizer.DRPCSimpleAclAuthorizer-test + (:use [clojure test]) + (:import [org.mockito Mockito]) + (:import [backtype.storm Config]) + (:import [backtype.storm.security.auth ReqContext SingleUserPrincipal]) + (:import [backtype.storm.security.auth.authorizer DRPCSimpleACLAuthorizer]) + (:use [backtype.storm config util]) + ) + +(defn- mk-mock-context [user] + (let [mock-context (Mockito/mock ReqContext)] + (. (Mockito/when (.principal mock-context)) thenReturn + (SingleUserPrincipal. user)) + mock-context)) + +(let [function "jump" + partial-function "partial" + alice-context (mk-mock-context "alice") + alice-kerb-context (mk-mock-context "alice@SOME.RELM") + bob-context (mk-mock-context "bob") + charlie-context (mk-mock-context "charlie") + acl-file "drpc-simple-acl-test-scenario.yaml" + strict-handler (doto (DRPCSimpleACLAuthorizer.) + (.prepare {DRPC-AUTHORIZER-ACL-STRICT true + DRPC-AUTHORIZER-ACL-FILENAME acl-file + STORM-PRINCIPAL-TO-LOCAL-PLUGIN "backtype.storm.security.auth.KerberosPrincipalToLocal"})) + permissive-handler (doto (DRPCSimpleACLAuthorizer.) + (.prepare {DRPC-AUTHORIZER-ACL-STRICT false + DRPC-AUTHORIZER-ACL-FILENAME acl-file + STORM-PRINCIPAL-TO-LOCAL-PLUGIN "backtype.storm.security.auth.KerberosPrincipalToLocal"}))] + + (deftest test-partial-authorization + (testing "deny execute to unauthorized user" + (is (not + (.permit strict-handler + (ReqContext/context) + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY partial-function})))) + + (testing "allow execute to authorized kerb user for correct function" + (is + (.permit + strict-handler + alice-kerb-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY partial-function}))) + + (testing "deny fetchRequest to unauthorized user for correct function" + (is (not + (.permit + strict-handler + alice-kerb-context + "fetchRequest" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY partial-function})))) + ) + + (deftest test-client-authorization-strict + (testing "deny execute to unauthorized user" + (is (not + (.permit strict-handler + (ReqContext/context) + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))) + + (testing "deny execute to valid user for incorrect function" + (is (not + (.permit + strict-handler + alice-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"})))) + + (testing "allow execute to authorized kerb user for correct function" + (is + (.permit + strict-handler + alice-kerb-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))) + + (testing "allow execute to authorized user for correct function" + (is + (.permit + strict-handler + alice-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))) + + + (deftest test-client-authorization-permissive + (testing "deny execute to unauthorized user for correct function" + (is (not + (.permit permissive-handler + (ReqContext/context) + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))) + + (testing "allow execute for user for incorrect function when permissive" + (is + (.permit permissive-handler + alice-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"}))) + + (testing "allow execute for user for incorrect function when permissive" + (is + (.permit permissive-handler + alice-kerb-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"}))) + + (testing "allow execute to authorized user for correct function" + (is + (.permit permissive-handler + bob-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))) + + (deftest test-invocation-authorization-strict + (doseq [operation ["fetchRequest" "failRequest" "result"]] + + (testing (str "deny " operation + " to unauthorized user for correct function") + (is (not + (.permit + strict-handler + alice-context + operation + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))) + + (testing (str "deny " operation + " to user for incorrect function when strict") + (is (not + (.permit + strict-handler + charlie-context + operation + {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"})))) + + (testing (str "allow " operation + " to authorized user for correct function") + (is + (.permit + strict-handler + charlie-context + operation + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))))) + + (deftest test-invocation-authorization-permissive + (doseq [operation ["fetchRequest" "failRequest" "result"]] + + (testing (str "deny " operation + " to unauthorized user for correct function") + (is (not + (.permit + permissive-handler + bob-context + operation + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))) + + (testing (str "allow " operation + " to user for incorrect function when permissive") + (is + (.permit + permissive-handler + charlie-context + operation + {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"}))) + + (testing (str operation " is allowed for authorized user") + (is + (.permit + permissive-handler + charlie-context + operation + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))))) + + (deftest test-deny-when-no-function-given + (is (not + (.permit strict-handler alice-context "execute" {}))) + + (is (not + (.permit + strict-handler + alice-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY nil}))) + + (is (not + (.permit permissive-handler bob-context "execute" {}))) + + (is (not + (.permit + permissive-handler + bob-context + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY nil})))) + + (deftest test-deny-when-invalid-user-given + (is (not + (.permit + strict-handler + (Mockito/mock ReqContext) + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))) + + (is (not + (.permit + strict-handler + nil + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))) + + (is (not + (.permit + permissive-handler + (Mockito/mock ReqContext) + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))) + + (is (not + (.permit + permissive-handler + nil + "execute" + {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))))) diff --git a/storm-core/test/clj/backtype/storm/security/auth/auto_login_module_test.clj b/storm-core/test/clj/backtype/storm/security/auth/auto_login_module_test.clj new file mode 100644 index 00000000000..2056509ef7e --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/auto_login_module_test.clj @@ -0,0 +1,91 @@ +;; 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.security.auth.auto-login-module-test + (:use [clojure test]) + (:use [backtype.storm util]) + (:import [backtype.storm.security.auth.kerberos AutoTGT + AutoTGTKrb5LoginModule AutoTGTKrb5LoginModuleTest]) + (:import [javax.security.auth Subject Subject]) + (:import [javax.security.auth.kerberos KerberosTicket]) + (:import [org.mockito Mockito]) + ) + +(deftest login-module-no-subj-no-tgt-test + (testing "Behavior is correct when there is no Subject or TGT" + (let [login-module (AutoTGTKrb5LoginModule.)] + + (is (thrown-cause? javax.security.auth.login.LoginException + (.login login-module))) + (is (not (.commit login-module))) + (is (not (.abort login-module))) + (is (.logout login-module))))) + +(deftest login-module-readonly-subj-no-tgt-test + (testing "Behavior is correct when there is a read-only Subject and no TGT" + (let [readonly-subj (Subject. true #{} #{} #{}) + login-module (AutoTGTKrb5LoginModule.)] + (.initialize login-module readonly-subj nil nil nil) + (is (not (.commit login-module))) + (is (.logout login-module))))) + +(deftest login-module-with-subj-no-tgt-test + (testing "Behavior is correct when there is a Subject and no TGT" + (let [login-module (AutoTGTKrb5LoginModule.)] + (.initialize login-module (Subject.) nil nil nil) + (is (thrown-cause? javax.security.auth.login.LoginException + (.login login-module))) + (is (not (.commit login-module))) + (is (not (.abort login-module))) + (is (.logout login-module))))) + +(deftest login-module-no-subj-with-tgt-test + (testing "Behavior is correct when there is no Subject and a TGT" + (let [login-module (AutoTGTKrb5LoginModuleTest.)] + (.setKerbTicket login-module (Mockito/mock KerberosTicket)) + (is (.login login-module)) + (is (thrown-cause? javax.security.auth.login.LoginException + (.commit login-module))) + + (.setKerbTicket login-module (Mockito/mock KerberosTicket)) + (is (.abort login-module)) + (is (.logout login-module))))) + +(deftest login-module-readonly-subj-with-tgt-test + (testing "Behavior is correct when there is a read-only Subject and a TGT" + (let [readonly-subj (Subject. true #{} #{} #{}) + login-module (AutoTGTKrb5LoginModuleTest.)] + (.initialize login-module readonly-subj nil nil nil) + (.setKerbTicket login-module (Mockito/mock KerberosTicket)) + (is (.login login-module)) + (is (thrown-cause? javax.security.auth.login.LoginException + (.commit login-module))) + + (.setKerbTicket login-module (Mockito/mock KerberosTicket)) + (is (.abort login-module)) + (is (.logout login-module))))) + +(deftest login-module-with-subj-and-tgt + (testing "Behavior is correct when there is a Subject and a TGT" + (let [login-module (AutoTGTKrb5LoginModuleTest.) + _ (set! (. login-module client) (Mockito/mock + java.security.Principal)) + ticket (Mockito/mock KerberosTicket)] + (.initialize login-module (Subject.) nil nil nil) + (.setKerbTicket login-module ticket) + (is (.login login-module)) + (is (.commit login-module)) + (is (.abort login-module)) + (is (.logout login-module))))) diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas new file mode 100644 index 00000000000..cd691ae65d1 --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas @@ -0,0 +1,5 @@ +StormClient { + org.apache.zookeeper.server.auth.DigestLoginModule required + username="alice" + password="poorpasswordforalice"; +}; diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas new file mode 100644 index 00000000000..e4ca097cd9b --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas @@ -0,0 +1,5 @@ +StormClient { + org.apache.zookeeper.server.auth.DigestLoginModule required + username="bob" + password="poorpasswordforbob"; +}; diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas new file mode 100644 index 00000000000..3473d6dbd0c --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas @@ -0,0 +1,5 @@ +StormClient { + org.apache.zookeeper.server.auth.DigestLoginModule required + username="charlie" + password="poorpasswordforcharlie"; +}; diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-server.jaas b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-server.jaas new file mode 100644 index 00000000000..3b22d21092f --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-server.jaas @@ -0,0 +1,6 @@ +StormServer { + org.apache.zookeeper.server.auth.DigestLoginModule required + user_alice="poorpasswordforalice" + user_bob="poorpasswordforbob" + user_charlie="poorpasswordforcharlie"; +}; diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc_auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/drpc_auth_test.clj new file mode 100644 index 00000000000..ff431ec0b83 --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/drpc_auth_test.clj @@ -0,0 +1,315 @@ +;; 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.security.auth.drpc-auth-test + (:use [clojure test]) + (:require [backtype.storm.daemon [drpc :as drpc]]) + (:import [backtype.storm.generated AuthorizationException + DRPCExecutionException DistributedRPC$Processor + DistributedRPCInvocations$Processor]) + (:import [backtype.storm Config]) + (:import [backtype.storm.security.auth ReqContext SingleUserPrincipal ThriftServer ThriftConnectionType]) + (:import [backtype.storm.utils DRPCClient]) + (:import [backtype.storm.drpc DRPCInvocationsClient]) + (:import [java.util.concurrent TimeUnit]) + (:import [javax.security.auth Subject]) + (:use [backtype.storm bootstrap util]) + (:use [backtype.storm.daemon common]) + (:use [backtype.storm bootstrap testing]) + ) + +(bootstrap) + +(def drpc-timeout (Integer. 30)) + +(defn launch-server [conf drpcAznClass transportPluginClass login-cfg client-port invocations-port] + (let [conf (if drpcAznClass (assoc conf DRPC-AUTHORIZER drpcAznClass) conf) + conf (if transportPluginClass (assoc conf STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass) conf) + conf (if login-cfg (assoc conf "java.security.auth.login.config" login-cfg) conf) + conf (assoc conf DRPC-PORT client-port) + conf (assoc conf DRPC-INVOCATIONS-PORT invocations-port) + service-handler (drpc/service-handler conf) + handler-server (ThriftServer. conf + (DistributedRPC$Processor. service-handler) + ThriftConnectionType/DRPC) + invoke-server (ThriftServer. conf + (DistributedRPCInvocations$Processor. service-handler) + ThriftConnectionType/DRPC_INVOCATIONS)] + (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop handler-server) (.stop invoke-server)))) + (log-message "storm conf:" conf) + (log-message "Starting DRPC invocation server ...") + (.start (Thread. #(.serve invoke-server))) + (wait-for-condition #(.isServing invoke-server)) + (log-message "Starting DRPC handler server ...") + (.start (Thread. #(.serve handler-server))) + (wait-for-condition #(.isServing handler-server)) + [handler-server invoke-server])) + +(defmacro with-server [args & body] + `(let [[handler-server# invoke-server#] (launch-server ~@args)] + ~@body + (log-message "Stopping DRPC servers ...") + (.stop handler-server#) + (.stop invoke-server#) + )) + +(deftest deny-drpc-test + (let [client-port (available-port) + invocations-port (available-port (inc client-port)) + storm-conf (read-storm-config)] + (with-server [storm-conf "backtype.storm.security.auth.authorizer.DenyAuthorizer" + nil nil client-port invocations-port] + (let [drpc (DRPCClient. storm-conf "localhost" client-port) + drpc_client (.getClient drpc) + invocations (DRPCInvocationsClient. storm-conf "localhost" invocations-port) + invocations_client (.getClient invocations)] + (is (thrown? AuthorizationException (.execute drpc_client "func-foo" "args-bar"))) + (is (thrown? AuthorizationException (.fetchRequest invocations_client nil))) + (.close drpc) + (.close invocations))))) + +(deftest deny-drpc-digest-test + (let [client-port (available-port) + invocations-port (available-port (inc client-port)) + storm-conf (read-storm-config)] + (with-server [storm-conf "backtype.storm.security.auth.authorizer.DenyAuthorizer" + "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" + "test/clj/backtype/storm/security/auth/jaas_digest.conf" + client-port invocations-port] + (let [conf (merge storm-conf {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"}) + drpc (DRPCClient. conf "localhost" client-port) + drpc_client (.getClient drpc) + invocations (DRPCInvocationsClient. conf "localhost" invocations-port) + invocations_client (.getClient invocations)] + (is (thrown? AuthorizationException (.execute drpc_client "func-foo" "args-bar"))) + (is (thrown? AuthorizationException (.fetchRequest invocations_client nil))) + (.close drpc) + (.close invocations))))) + +(defmacro with-simple-drpc-test-scenario + [[strict? alice-client bob-client charlie-client alice-invok charlie-invok] & body] + (let [client-port (available-port) + invocations-port (available-port (inc client-port)) + storm-conf (merge (read-storm-config) + {DRPC-AUTHORIZER-ACL-STRICT strict? + DRPC-AUTHORIZER-ACL-FILENAME "drpc-simple-acl-test-scenario.yaml" + STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"})] + `(with-server [~storm-conf + "backtype.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer" + "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" + "test/clj/backtype/storm/security/auth/drpc-auth-server.jaas" + ~client-port ~invocations-port] + (let [~alice-client (DRPCClient. + (merge ~storm-conf {"java.security.auth.login.config" + "test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas"}) + "localhost" + ~client-port) + ~bob-client (DRPCClient. + (merge ~storm-conf {"java.security.auth.login.config" + "test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas"}) + "localhost" + ~client-port) + ~charlie-client (DRPCClient. + (merge ~storm-conf {"java.security.auth.login.config" + "test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas"}) + "localhost" + ~client-port) + ~alice-invok (DRPCInvocationsClient. + (merge ~storm-conf {"java.security.auth.login.config" + "test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas"}) + "localhost" + ~invocations-port) + ~charlie-invok (DRPCInvocationsClient. + (merge ~storm-conf {"java.security.auth.login.config" + "test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas"}) + "localhost" + ~invocations-port)] + (try + ~@body + (finally + (.close ~alice-client) + (.close ~bob-client) + (.close ~charlie-client) + (.close ~alice-invok) + (.close ~charlie-invok))))))) + +(deftest drpc-per-function-auth-strict-test + (with-simple-drpc-test-scenario [true alice-client bob-client charlie-client alice-invok charlie-invok] + (let [drpc-timeout-seconds 10] + (testing "Permitted user can execute a function in the ACL" + (let [func "jump" + exec-ftr (future (.execute alice-client func "some args")) + id (atom "") + expected "Authorized DRPC"] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (.result charlie-invok @id expected) + (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))) + + (testing "execute fails when function is not in ACL" + (is (thrown-cause? AuthorizationException + (.execute alice-client "jog" "some args")))) + + (testing "fetchRequest fails when function is not in ACL" + (is (thrown-cause? AuthorizationException + (.fetchRequest charlie-invok "jog")))) + + (testing "authorized user can fail a request" + (let [func "jump" + exec-ftr (future (.execute alice-client func "some args")) + id (atom "")] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (.failRequest charlie-invok @id) + (is (thrown-cause? DRPCExecutionException + (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))) + + (testing "unauthorized invocation user is denied returning a result" + (let [func "jump" + exec-ftr (future (.execute bob-client func "some args")) + id (atom "") + expected "Only Authorized User can populate the result"] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (is (thrown-cause? AuthorizationException + (.result alice-invok @id "not the expected result"))) + (.result charlie-invok @id expected) + (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))) + + (testing "unauthorized invocation user is denied failing a request" + (let [func "jump" + exec-ftr (future (.execute alice-client func "some args")) + id (atom "")] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (is (thrown-cause? AuthorizationException + (.failRequest alice-invok @id))) + (.failRequest charlie-invok @id)))) + + (testing "unauthorized invocation user is denied fetching a request" + (let [func "jump" + exec-ftr (future (.execute bob-client func "some args")) + id (atom "") + expected "Only authorized users can fetchRequest"] + (Thread/sleep 1000) + (is (thrown-cause? AuthorizationException + (-> alice-invok (.fetchRequest func) .get_request_id))) + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (.result charlie-invok @id expected) + (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))))) + +(deftest drpc-per-function-auth-non-strict-test + (with-simple-drpc-test-scenario [false alice-client bob-client charlie-client alice-invok charlie-invok] + (let [drpc-timeout-seconds 10] + (testing "Permitted user can execute a function in the ACL" + (let [func "jump" + exec-ftr (future (.execute alice-client func "some args")) + id (atom "") + expected "Authorized DRPC"] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (.result charlie-invok @id expected) + (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))) + + (testing "DRPC succeeds for anyone when function is not in ACL" + (let [func "jog" + exec-ftr (future (.execute charlie-client func "some args")) + id (atom "") + expected "Permissive/No ACL Entry"] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> alice-invok (.fetchRequest func) .get_request_id))) + (.result alice-invok @id expected) + (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))) + + (testing "failure of a request is allowed when function is not in ACL" + (let [func "jog" + exec-ftr (future (.execute charlie-client func "some args")) + id (atom "")] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> alice-invok (.fetchRequest func) .get_request_id))) + (.failRequest alice-invok @id) + (is (thrown-cause? DRPCExecutionException + (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))) + + (testing "authorized user can fail a request" + (let [func "jump" + exec-ftr (future (.execute alice-client func "some args")) + id (atom "")] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (.failRequest charlie-invok @id) + (is (thrown-cause? DRPCExecutionException + (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))) + + (testing "unauthorized invocation user is denied returning a result" + (let [func "jump" + exec-ftr (future (.execute bob-client func "some args")) + id (atom "") + expected "Only Authorized User can populate the result"] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (is (thrown-cause? AuthorizationException + (.result alice-invok @id "not the expected result"))) + (.result charlie-invok @id expected) + (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))) + + (testing "unauthorized invocation user is denied failing a request" + (let [func "jump" + exec-ftr (future (.execute alice-client func "some args")) + id (atom "")] + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (is (thrown-cause? AuthorizationException + (.failRequest alice-invok @id))) + (.failRequest charlie-invok @id)))) + + (testing "unauthorized invocation user is denied fetching a request" + (let [func "jump" + exec-ftr (future (.execute bob-client func "some args")) + id (atom "") + expected "Only authorized users can fetchRequest"] + (Thread/sleep 1000) + (is (thrown-cause? AuthorizationException + (-> alice-invok (.fetchRequest func) .get_request_id))) + (with-timeout drpc-timeout-seconds TimeUnit/SECONDS + (while (empty? @id) + (reset! id + (-> charlie-invok (.fetchRequest func) .get_request_id))) + (.result charlie-invok @id expected) + (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))))) diff --git a/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj new file mode 100644 index 00000000000..bb7023944b0 --- /dev/null +++ b/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj @@ -0,0 +1,181 @@ +;; 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.security.auth.nimbus-auth-test + (:use [clojure test]) + (:require [backtype.storm [testing :as testing]]) + (:require [backtype.storm.daemon [nimbus :as nimbus]]) + (:require [backtype.storm [zookeeper :as zk]]) + (:import [java.nio ByteBuffer]) + (:import [backtype.storm Config]) + (:import [backtype.storm.utils NimbusClient]) + (:import [backtype.storm.generated NotAliveException]) + (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient + ReqContext ThriftConnectionType]) + (:use [backtype.storm bootstrap cluster util]) + (:use [backtype.storm.daemon common nimbus]) + (:use [backtype.storm bootstrap]) + (:import [backtype.storm.generated Nimbus Nimbus$Client + AuthorizationException SubmitOptions TopologyInitialStatus KillOptions]) + (:require [conjure.core]) + (:use [conjure core]) + ) + +(bootstrap) + +(def nimbus-timeout (Integer. 30)) + +(defn launch-test-cluster [nimbus-port login-cfg aznClass transportPluginClass] + (let [conf {NIMBUS-AUTHORIZER aznClass + NIMBUS-THRIFT-PORT nimbus-port + STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass } + conf (if login-cfg (merge conf {"java.security.auth.login.config" login-cfg}) conf) + cluster-map (testing/mk-local-storm-cluster :supervisors 0 + :ports-per-supervisor 0 + :daemon-conf conf) + nimbus-server (ThriftServer. (:daemon-conf cluster-map) + (Nimbus$Processor. (:nimbus cluster-map)) + ThriftConnectionType/NIMBUS)] + (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop nimbus-server)))) + (.start (Thread. #(.serve nimbus-server))) + (wait-for-condition #(.isServing nimbus-server)) + [cluster-map nimbus-server])) + +(defmacro with-test-cluster [args & body] + `(let [[cluster-map# nimbus-server#] (launch-test-cluster ~@args)] + ~@body + (log-debug "Shutdown cluster from macro") + (testing/kill-local-storm-cluster cluster-map#) + (.stop nimbus-server#))) + +(deftest Simple-authentication-test + (with-test-cluster [6627 nil nil "backtype.storm.security.auth.SimpleTransportPlugin"] + (let [storm-conf (merge (read-storm-config) + {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin" + STORM-NIMBUS-RETRY-TIMES 0}) + client (NimbusClient. storm-conf "localhost" 6627 nimbus-timeout) + nimbus_client (.getClient client)] + (testing "(Positive authorization) Simple protocol w/o authentication/authorization enforcement" + (is (thrown-cause? NotAliveException + (.activate nimbus_client "topo-name")))) + (.close client)))) + +(deftest test-noop-authorization-w-simple-transport + (with-test-cluster [6628 nil + "backtype.storm.security.auth.authorizer.NoopAuthorizer" + "backtype.storm.security.auth.SimpleTransportPlugin"] + (let [storm-conf (merge (read-storm-config) + {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin" + STORM-NIMBUS-RETRY-TIMES 0}) + client (NimbusClient. storm-conf "localhost" 6628 nimbus-timeout) + nimbus_client (.getClient client)] + (testing "(Positive authorization) Authorization plugin should accept client request" + (is (thrown-cause? NotAliveException + (.activate nimbus_client "topo-name")))) + (.close client)))) + +(deftest test-deny-authorization-w-simple-transport + (with-test-cluster [6629 nil + "backtype.storm.security.auth.authorizer.DenyAuthorizer" + "backtype.storm.security.auth.SimpleTransportPlugin"] + (let [storm-conf (merge (read-storm-config) + {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin" + Config/NIMBUS_HOST "localhost" + Config/NIMBUS_THRIFT_PORT 6629 + STORM-NIMBUS-RETRY-TIMES 0}) + client (NimbusClient/getConfiguredClient storm-conf) + nimbus_client (.getClient client) + topologyInitialStatus (TopologyInitialStatus/findByValue 2) + submitOptions (SubmitOptions. topologyInitialStatus)] + (is (thrown-cause? AuthorizationException (.submitTopology nimbus_client "topo-name" nil nil nil))) + (is (thrown-cause? AuthorizationException (.submitTopologyWithOpts nimbus_client "topo-name" nil nil nil submitOptions))) + (is (thrown-cause? AuthorizationException (.beginFileUpload nimbus_client))) + (is (thrown-cause? AuthorizationException (.uploadChunk nimbus_client nil nil))) + (is (thrown-cause? AuthorizationException (.finishFileUpload nimbus_client nil))) + (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus_client nil))) + (is (thrown-cause? AuthorizationException (.downloadChunk nimbus_client nil))) + (is (thrown-cause? AuthorizationException (.getNimbusConf nimbus_client))) + (is (thrown-cause? AuthorizationException (.getClusterInfo nimbus_client))) + (stubbing [nimbus/check-storm-active! nil + nimbus/try-read-storm-conf-from-name {}] + (is (thrown-cause? AuthorizationException (.killTopology nimbus_client "topo-name"))) + (is (thrown-cause? AuthorizationException (.killTopologyWithOpts nimbus_client "topo-name" (KillOptions.)))) + (is (thrown-cause? AuthorizationException (.activate nimbus_client "topo-name"))) + (is (thrown-cause? AuthorizationException (.deactivate nimbus_client "topo-name"))) + (is (thrown-cause? AuthorizationException (.rebalance nimbus_client "topo-name" nil))) + ) + (stubbing [nimbus/try-read-storm-conf {}] + (is (thrown-cause? AuthorizationException (.getTopologyConf nimbus_client "topo-ID"))) + (is (thrown-cause? AuthorizationException (.getTopology nimbus_client "topo-ID"))) + (is (thrown-cause? AuthorizationException (.getUserTopology nimbus_client "topo-ID"))) + (is (thrown-cause? AuthorizationException (.getTopologyInfo nimbus_client "topo-ID")))) + (.close client)))) + +(deftest test-noop-authorization-w-sasl-digest + (with-test-cluster [6630 + "test/clj/backtype/storm/security/auth/jaas_digest.conf" + "backtype.storm.security.auth.authorizer.NoopAuthorizer" + "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"] + (let [storm-conf (merge (read-storm-config) + {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf" + Config/NIMBUS_HOST "localhost" + Config/NIMBUS_THRIFT_PORT 6630 + STORM-NIMBUS-RETRY-TIMES 0}) + client (NimbusClient/getConfiguredClient storm-conf) + nimbus_client (.getClient client)] + (testing "(Positive authorization) Authorization plugin should accept client request" + (is (thrown-cause? NotAliveException + (.activate nimbus_client "topo-name")))) + (.close client)))) + +(deftest test-deny-authorization-w-sasl-digest + (with-test-cluster [6631 + "test/clj/backtype/storm/security/auth/jaas_digest.conf" + "backtype.storm.security.auth.authorizer.DenyAuthorizer" + "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"] + (let [storm-conf (merge (read-storm-config) + {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" + "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf" + Config/NIMBUS_HOST "localhost" + Config/NIMBUS_THRIFT_PORT 6631 + STORM-NIMBUS-RETRY-TIMES 0}) + client (NimbusClient/getConfiguredClient storm-conf) + nimbus_client (.getClient client) + topologyInitialStatus (TopologyInitialStatus/findByValue 2) + submitOptions (SubmitOptions. topologyInitialStatus)] + (is (thrown-cause? AuthorizationException (.submitTopology nimbus_client "topo-name" nil nil nil))) + (is (thrown-cause? AuthorizationException (.submitTopologyWithOpts nimbus_client "topo-name" nil nil nil submitOptions))) + (is (thrown-cause? AuthorizationException (.beginFileUpload nimbus_client))) + (is (thrown-cause? AuthorizationException (.uploadChunk nimbus_client nil nil))) + (is (thrown-cause? AuthorizationException (.finishFileUpload nimbus_client nil))) + (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus_client nil))) + (is (thrown-cause? AuthorizationException (.downloadChunk nimbus_client nil))) + (is (thrown-cause? AuthorizationException (.getNimbusConf nimbus_client))) + (is (thrown-cause? AuthorizationException (.getClusterInfo nimbus_client))) + (stubbing [nimbus/check-storm-active! nil + nimbus/try-read-storm-conf-from-name {}] + (is (thrown-cause? AuthorizationException (.killTopology nimbus_client "topo-name"))) + (is (thrown-cause? AuthorizationException (.killTopologyWithOpts nimbus_client "topo-name" (KillOptions.)))) + (is (thrown-cause? AuthorizationException (.activate nimbus_client "topo-name"))) + (is (thrown-cause? AuthorizationException (.deactivate nimbus_client "topo-name"))) + (is (thrown-cause? AuthorizationException (.rebalance nimbus_client "topo-name" nil)))) + (stubbing [nimbus/try-read-storm-conf {}] + (is (thrown-cause? AuthorizationException (.getTopologyConf nimbus_client "topo-ID"))) + (is (thrown-cause? AuthorizationException (.getTopology nimbus_client "topo-ID"))) + (is (thrown-cause? AuthorizationException (.getUserTopology nimbus_client "topo-ID"))) + (is (thrown-cause? AuthorizationException (.getTopologyInfo nimbus_client "topo-ID")))) + (.close client)))) + diff --git a/storm-core/test/clj/backtype/storm/submitter_test.clj b/storm-core/test/clj/backtype/storm/submitter_test.clj new file mode 100644 index 00000000000..6f8829700e6 --- /dev/null +++ b/storm-core/test/clj/backtype/storm/submitter_test.clj @@ -0,0 +1,75 @@ +;; 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.submitter-test + (:use [clojure test]) + (:use [backtype.storm config testing]) + (:import [backtype.storm StormSubmitter]) + ) + +(deftest test-md5-digest-secret-generation + (testing "No payload or scheme are generated when already present" + (let [conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD "foobar:12345" + STORM-ZOOKEEPER-AUTH-SCHEME "anything"} + result (StormSubmitter/prepareZookeeperAuthentication conf) + actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD) + actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)] + (is (nil? actual-payload)) + (is (= "digest" actual-scheme)))) + + (testing "Scheme is set to digest if not already." + (let [conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD "foobar:12345"} + result (StormSubmitter/prepareZookeeperAuthentication conf) + actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD) + actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)] + (is (nil? actual-payload)) + (is (= "digest" actual-scheme)))) + + (testing "A payload is generated when no payload is present." + (let [conf {STORM-ZOOKEEPER-AUTH-SCHEME "anything"} + result (StormSubmitter/prepareZookeeperAuthentication conf) + actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD) + actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)] + (is (not (clojure.string/blank? actual-payload))) + (is (= "digest" actual-scheme)))) + + (testing "A payload is generated when payload is not correctly formatted." + (let [bogus-payload "not-a-valid-payload" + conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD bogus-payload + STORM-ZOOKEEPER-AUTH-SCHEME "anything"} + result (StormSubmitter/prepareZookeeperAuthentication conf) + actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD) + actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)] + (is (not (StormSubmitter/validateZKDigestPayload bogus-payload))) ; Is this test correct? + (is (not (clojure.string/blank? actual-payload))) + (is (= "digest" actual-scheme)))) + + (testing "A payload is generated when payload is null." + (let [conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD nil + STORM-ZOOKEEPER-AUTH-SCHEME "anything"} + result (StormSubmitter/prepareZookeeperAuthentication conf) + actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD) + actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)] + (is (not (clojure.string/blank? actual-payload))) + (is (= "digest" actual-scheme)))) + + (testing "A payload is generated when payload is blank." + (let [conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD "" + STORM-ZOOKEEPER-AUTH-SCHEME "anything"} + result (StormSubmitter/prepareZookeeperAuthentication conf) + actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD) + actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)] + (is (not (clojure.string/blank? actual-payload))) + (is (= "digest" actual-scheme))))) diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj index 1d0804a7f01..3dcd2758b7b 100644 --- a/storm-core/test/clj/backtype/storm/supervisor_test.clj +++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj @@ -15,12 +15,18 @@ ;; limitations under the License. (ns backtype.storm.supervisor-test (:use [clojure test]) + (:require [conjure.core]) + (:use [conjure core]) + (:require [clojure.contrib [string :as contrib-str]]) (:require [clojure [string :as string]]) (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter]) + (:import [backtype.storm.scheduler ISupervisor]) + (:import [java.util UUID]) (:use [backtype.storm bootstrap config testing]) (:use [backtype.storm.daemon common]) (:require [backtype.storm.daemon [worker :as worker] [supervisor :as supervisor]]) (:use [conjure core]) + (:require [clojure.java.io :as io]) ) (bootstrap) @@ -252,7 +258,7 @@ opts topo-opts ["-Djava.library.path=" - (str "-Dlogfile.name=worker-" mock-port ".log") + (str "-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log") "-Dstorm.home=" "-Dlogback.configurationFile=/logback/cluster.xml" (str "-Dstorm.id=" mock-storm-id) @@ -274,8 +280,10 @@ topo-string-opts} add-to-classpath mock-cp supervisor-stormdist-root nil + launch-process nil + set-worker-user! nil supervisor/jlp nil - launch-process nil] + supervisor/write-log-metadata! nil] (supervisor/launch-worker mock-supervisor mock-storm-id mock-port @@ -293,8 +301,10 @@ topo-list-opts} add-to-classpath mock-cp supervisor-stormdist-root nil + launch-process nil + set-worker-user! nil supervisor/jlp nil - launch-process nil] + supervisor/write-log-metadata! nil] (supervisor/launch-worker mock-supervisor mock-storm-id mock-port @@ -303,6 +313,99 @@ [0] exp-args))))))) +(defn rm-r [f] + (if (.isDirectory f) + (for [sub (.listFiles f)] (rm-r sub)) + (.delete f) + )) + +(deftest test-worker-launch-command-run-as-user + (testing "*.worker.childopts configuration" + (let [mock-port "42" + mock-storm-id "fake-storm-id" + mock-worker-id "fake-worker-id" + mock-cp "mock-classpath'quote-on-purpose" + storm-local (str "/tmp/" (UUID/randomUUID)) + worker-script (str storm-local "/workers/" mock-worker-id "/storm-worker-script.sh") + exp-launch ["/bin/worker-launcher" + "me" + "worker" + (str storm-local "/workers/" mock-worker-id) + worker-script] + exp-script-fn (fn [opts topo-opts] + (str "#!/bin/bash\n'export' 'LD_LIBRARY_PATH=';\n\nexec 'java' '-server'" + " " (shell-cmd opts) + " " (shell-cmd topo-opts) + " '-Djava.library.path='" + " '-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log'" + " '-Dstorm.home='" + " '-Dlogback.configurationFile=/logback/cluster.xml'" + " '-Dstorm.id=" mock-storm-id "'" + " '-Dworker.id=" mock-worker-id "'" + " '-Dworker.port=" mock-port "'" + " '-cp' 'mock-classpath'\"'\"'quote-on-purpose'" + " 'backtype.storm.daemon.worker'" + " '" mock-storm-id "'" + " '" mock-port "'" + " '" mock-worker-id "';"))] + (.mkdirs (io/file storm-local "workers" mock-worker-id)) + (try + (testing "testing *.worker.childopts as strings with extra spaces" + (let [string-opts "-Dfoo=bar -Xmx1024m" + topo-string-opts "-Dkau=aux -Xmx2048m" + exp-script (exp-script-fn ["-Dfoo=bar" "-Xmx1024m"] + ["-Dkau=aux" "-Xmx2048m"]) + mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed + STORM-LOCAL-DIR storm-local + SUPERVISOR-RUN-WORKER-AS-USER true + WORKER-CHILDOPTS string-opts}}] + (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS + topo-string-opts + TOPOLOGY-SUBMITTER-USER "me"} + add-to-classpath mock-cp + supervisor-stormdist-root nil + launch-process nil + set-worker-user! nil + supervisor/java-cmd "java" + supervisor/jlp nil + supervisor/write-log-metadata! nil] + (supervisor/launch-worker mock-supervisor + mock-storm-id + mock-port + mock-worker-id) + (verify-first-call-args-for-indices launch-process + [0] + exp-launch)) + (is (= (slurp worker-script) exp-script)))) + (testing "testing *.worker.childopts as list of strings, with spaces in values" + (let [list-opts '("-Dopt1='this has a space in it'" "-Xmx1024m") + topo-list-opts '("-Dopt2='val with spaces'" "-Xmx2048m") + exp-script (exp-script-fn list-opts topo-list-opts) + mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed + STORM-LOCAL-DIR storm-local + SUPERVISOR-RUN-WORKER-AS-USER true + WORKER-CHILDOPTS list-opts}}] + (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS + topo-list-opts + TOPOLOGY-SUBMITTER-USER "me"} + add-to-classpath mock-cp + supervisor-stormdist-root nil + launch-process nil + set-worker-user! nil + supervisor/java-cmd "java" + supervisor/jlp nil + supervisor/write-log-metadata! nil] + (supervisor/launch-worker mock-supervisor + mock-storm-id + mock-port + mock-worker-id) + (verify-first-call-args-for-indices launch-process + [0] + exp-launch)) + (is (= (slurp worker-script) exp-script)))) +(finally (rm-r (io/file storm-local))) +)))) + (deftest test-workers-go-bananas ;; test that multiple workers are started for a port, and test that ;; supervisor shuts down propertly (doesn't shutdown the most @@ -319,3 +422,142 @@ ;; TODO just do reassign, and check that cleans up worker states after killing but doesn't get rid of downloaded code ) +(deftest test-supervisor-data-acls + (testing "supervisor-data uses correct ACLs" + (let [scheme "digest" + digest "storm:thisisapoorpassword" + auth-conf {STORM-ZOOKEEPER-AUTH-SCHEME scheme + STORM-ZOOKEEPER-AUTH-PAYLOAD digest} + expected-acls supervisor/SUPERVISOR-ZK-ACLS + fake-isupervisor (reify ISupervisor + (getSupervisorId [this] nil) + (getAssignmentId [this] nil))] + (stubbing [uptime-computer nil + cluster/mk-storm-cluster-state nil + supervisor-state nil + local-hostname nil + mk-timer nil] + (supervisor/supervisor-data auth-conf nil fake-isupervisor) + (verify-call-times-for cluster/mk-storm-cluster-state 1) + (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2] + expected-acls))))) + +(deftest test-write-log-metadata + (testing "supervisor writes correct data to logs metadata file" + (let [exp-owner "alice" + exp-worker-id "42" + exp-storm-id "0123456789" + exp-port 4242 + exp-logs-users ["bob" "charlie" "daryl"] + storm-conf {TOPOLOGY-SUBMITTER-USER "alice" + TOPOLOGY-USERS ["charlie" "bob"] + LOGS-USERS ["daryl"]} + exp-data {TOPOLOGY-SUBMITTER-USER exp-owner + "worker-id" exp-worker-id + LOGS-USERS exp-logs-users} + conf {}] + (mocking [supervisor/write-log-metadata-to-yaml-file!] + (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id + exp-storm-id exp-port conf) + (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file! + exp-storm-id exp-port exp-data conf))))) + +(deftest test-worker-launcher-requires-user + (testing "worker-launcher throws on blank user" + (mocking [launch-process] + (is (thrown-cause-with-msg? java.lang.IllegalArgumentException + #"(?i).*user cannot be blank.*" + (supervisor/worker-launcher {} nil "")))))) + +(defn found? [sub-str input-str] + (if (string? input-str) + (contrib-str/substring? sub-str (str input-str)) + (some? #(contrib-str/substring? sub-str %) input-str))) + +(defn not-found? [sub-str input-str] + (complement (found? sub-str input-str))) + +(deftest test-substitute-childopts-happy-path + (testing "worker-launcher replaces ids in childopts" + (let [ worker-id "w-01" + storm-id "s-01" + port 9999 + childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%STORM-ID%-%WORKER-ID%-%WORKER-PORT%.log" + ] + (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port)) + (is (not-found? "%WORKER-ID%" childopts-with-ids)) + (is (found? "w-01" childopts-with-ids)) + (is (not-found? "%STORM-ID%" childopts-with-ids)) + (is (found? "s-01" childopts-with-ids)) + (is (not-found? "%WORKER-PORT%" childopts-with-ids)) + (is (found? "-9999." childopts-with-ids)) + (is (not-found? "%ID%" childopts-with-ids)) + (is (found? "worker-9999" childopts-with-ids) (str childopts-with-ids)) + ))) + +(deftest test-substitute-childopts-storm-id-alone + (testing "worker-launcher replaces ids in childopts" + (let [ worker-id "w-01" + storm-id "s-01" + port 9999 + childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%STORM-ID%.log"] + (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port)) + (is (not-found? "%WORKER-ID%" childopts-with-ids)) + (is (not-found? "w-01" childopts-with-ids)) + (is (not-found? "%STORM-ID%" childopts-with-ids)) + (is (found? "s-01" childopts-with-ids)) + (is (not-found? "%WORKER-PORT%" childopts-with-ids)) + (is (not-found? "-9999." childopts-with-ids)) + (is (not-found? "%ID%" childopts-with-ids)) + (is (not-found? "worker-9999" childopts-with-ids) (str childopts-with-ids)) ))) + +(deftest test-substitute-childopts-no-keys + (testing "worker-launcher has no ids to replace in childopts" + (let [ worker-id "w-01" + storm-id "s-01" + port 9999 + childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"] + (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port)) + (is (not-found? "%WORKER-ID%" childopts-with-ids)) + (is (not-found? "w-01" childopts-with-ids)) + (is (not-found? "%STORM-ID%" childopts-with-ids)) + (is (not-found? "s-01" childopts-with-ids)) + (is (not-found? "%WORKER-PORT%" childopts-with-ids)) + (is (not-found? "-9999." childopts-with-ids)) + (is (not-found? "%ID%" childopts-with-ids)) + (is (not-found? "worker-9999" childopts-with-ids) (str childopts-with-ids)) ))) + +(deftest test-substitute-childopts-nil-childopts + (testing "worker-launcher has nil childopts" + (let [ worker-id "w-01" + storm-id "s-01" + port 9999 + childopts nil] + (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port)) + (is (not-found? "%WORKER-ID%" childopts-with-ids)) + (is (not-found? "w-01" childopts-with-ids)) + (is (not-found? "%STORM-ID%" childopts-with-ids)) + (is (not-found? "s-01" childopts-with-ids)) + (is (not-found? "%WORKER-PORT%" childopts-with-ids)) + (is (not-found? "-9999." childopts-with-ids)) + (is (not-found? "%ID%" childopts-with-ids)) + (is (not-found? "worker-9999" childopts-with-ids) (str childopts-with-ids)) + ))) + +(deftest test-substitute-childopts-nil-ids + (testing "worker-launcher has nil ids" + (let [ worker-id nil + storm-id "s-01" + port 9999 + childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%STORM-ID%-%WORKER-ID%-%WORKER-PORT%.log"] + (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port)) + (is (not-found? "%WORKER-ID%" childopts-with-ids)) + (is (not-found? "w-01" childopts-with-ids)) + (is (not-found? "%STORM-ID%" childopts-with-ids)) + (is (found? "s-01" childopts-with-ids)) + (is (not-found? "%WORKER-PORT%" childopts-with-ids)) + (is (found? "-9999." childopts-with-ids)) + (is (not-found? "%ID%" childopts-with-ids)) + (is (found? "worker-9999" childopts-with-ids) (str childopts-with-ids)) + ))) + diff --git a/storm-core/test/clj/backtype/storm/transactional_test.clj b/storm-core/test/clj/backtype/storm/transactional_test.clj index c633aa163e6..6c9ea44b0a2 100644 --- a/storm-core/test/clj/backtype/storm/transactional_test.clj +++ b/storm-core/test/clj/backtype/storm/transactional_test.clj @@ -18,10 +18,16 @@ (:import [backtype.storm.topology TopologyBuilder]) (:import [backtype.storm.transactional TransactionalSpoutCoordinator ITransactionalSpout ITransactionalSpout$Coordinator TransactionAttempt TransactionalTopologyBuilder]) - (:import [backtype.storm.transactional.state TransactionalState RotatingTransactionalState RotatingTransactionalState$StateInitializer]) + (:import [backtype.storm.transactional.state TransactionalState TestTransactionalState RotatingTransactionalState RotatingTransactionalState$StateInitializer]) (:import [backtype.storm.testing CountingBatchBolt MemoryTransactionalSpout KeyedCountingBatchBolt KeyedCountingCommitterBolt KeyedSummingBatchBolt IdentityBolt CountingCommitBolt OpaqueMemoryTransactionalSpout]) + (:import [backtype.storm.utils ZookeeperAuthInfo]) + (:import [org.apache.curator.framework CuratorFramework]) + (:import [org.apache.curator.framework.api CreateBuilder ProtectACLCreateModePathAndBytesable]) + (:import [org.apache.zookeeper CreateMode ZooDefs ZooDefs$Ids]) + (:import [org.mockito Matchers Mockito]) + (:import [org.mockito.exceptions.base MockitoAssertionError]) (:use [backtype.storm bootstrap testing]) (:use [backtype.storm.daemon common]) ) @@ -701,4 +707,21 @@ (verify! {"count" [[2 "apple" 1] [2 "dog" 1]]}) - )))) \ No newline at end of file + )))) + +(deftest test-create-node-acl + (testing "Creates ZooKeeper nodes with the correct ACLs" + (let [curator (Mockito/mock CuratorFramework) + builder0 (Mockito/mock CreateBuilder) + builder1 (Mockito/mock ProtectACLCreateModePathAndBytesable) + expectedAcls ZooDefs$Ids/CREATOR_ALL_ACL] + (. (Mockito/when (.create curator)) (thenReturn builder0)) + (. (Mockito/when (.creatingParentsIfNeeded builder0)) (thenReturn builder1)) + (. (Mockito/when (.withMode builder1 (Matchers/isA CreateMode))) (thenReturn builder1)) + (. (Mockito/when (.withACL builder1 (Mockito/anyList))) (thenReturn builder1)) + (TestTransactionalState/createNode curator "" (byte-array 0) expectedAcls nil) + (is (nil? + (try + (. (Mockito/verify builder1) (withACL expectedAcls)) + (catch MockitoAssertionError e + e))))))) diff --git a/storm-core/test/clj/backtype/storm/ui_test.clj b/storm-core/test/clj/backtype/storm/ui_test.clj new file mode 100644 index 00000000000..3a837c5237f --- /dev/null +++ b/storm-core/test/clj/backtype/storm/ui_test.clj @@ -0,0 +1,49 @@ +;; 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.ui-test + (:use [clojure test]) + (:use [backtype.storm config]) + (:use [backtype.storm testing]) + (:require [backtype.storm.ui [core :as core]]) + ) + +(deftest test-authorized-ui-user + (testing "allow cluster admin" + (let [conf {NIMBUS-ADMINS ["alice"]}] + (is (core/authorized-ui-user? "alice" conf {})))) + + (testing "ignore any cluster-set topology.users" + (let [conf {TOPOLOGY-USERS ["alice"]}] + (is (not (core/authorized-ui-user? "alice" conf {}))))) + + (testing "allow cluster ui user" + (let [conf {UI-USERS ["alice"]}] + (is (core/authorized-ui-user? "alice" conf {})))) + + (testing "allow submitted topology user" + (let [topo-conf {TOPOLOGY-USERS ["alice"]}] + (is (core/authorized-ui-user? "alice" {} topo-conf)))) + + (testing "allow submitted ui user" + (let [topo-conf {UI-USERS ["alice"]}] + (is (core/authorized-ui-user? "alice" {} topo-conf)))) + + (testing "disallow user not in nimbus admin, topo user, or ui user" + (is (not (core/authorized-ui-user? "alice" {} {})))) + + (testing "user cannot override nimbus admin" + (let [topo-conf {NIMBUS-ADMINS ["alice"]}] + (is (not (core/authorized-ui-user? "alice" {} topo-conf)))))) diff --git a/storm-core/test/clj/backtype/storm/utils/ZookeeperServerCnxnFactory_test.clj b/storm-core/test/clj/backtype/storm/utils/ZookeeperServerCnxnFactory_test.clj new file mode 100644 index 00000000000..0b2d1cdc4dc --- /dev/null +++ b/storm-core/test/clj/backtype/storm/utils/ZookeeperServerCnxnFactory_test.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.utils.ZookeeperServerCnxnFactory-test + (:import [backtype.storm.utils ZookeeperServerCnxnFactory]) + (:use [clojure test]) +) + +(deftest test-constructor-throws-runtimeexception-if-port-too-large + (is (thrown? RuntimeException (ZookeeperServerCnxnFactory. 65536 42))) +) + +(deftest test-factory + (let [zkcf-negative (ZookeeperServerCnxnFactory. -42 42) + next-port (+ (.port zkcf-negative) 1) + arbitrary-max-clients 42 + zkcf-next (ZookeeperServerCnxnFactory. next-port arbitrary-max-clients)] + ; Test handling negative port + (is (not (nil? zkcf-negative))) + ; Test max-clients is correctly set. + (is (= (-> zkcf-next .factory .getMaxClientCnxnsPerHost) arbitrary-max-clients)) + ) +) diff --git a/storm-core/test/clj/backtype/storm/utils_test.clj b/storm-core/test/clj/backtype/storm/utils_test.clj index 2294893bac8..b2d5958c22e 100644 --- a/storm-core/test/clj/backtype/storm/utils_test.clj +++ b/storm-core/test/clj/backtype/storm/utils_test.clj @@ -32,7 +32,7 @@ Config/STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING expected_ceiling}) servers ["bogus_server"] arbitrary_port 42 - curator (Utils/newCurator conf servers arbitrary_port) + curator (Utils/newCurator conf servers arbitrary_port nil) retry (-> curator .getZookeeperClient .getRetryPolicy) ] (is (.isAssignableFrom ExponentialBackoffRetry (.getClass retry))) @@ -49,20 +49,68 @@ "backtype.storm.security.auth.SimpleTransportPlugin" Config/NIMBUS_HOST "" Config/NIMBUS_THRIFT_PORT 65535 - })] - (is (thrown? RuntimeException + STORM-NIMBUS-RETRY-TIMES 0})] + (is (thrown-cause? RuntimeException (NimbusClient/getConfiguredClient storm-conf))) ) ) (deftest test-getConfiguredClient-throws-RunTimeException-on-bad-args - (let [storm-conf (read-storm-config)] - (is (thrown? TTransportException + (let [storm-conf (merge + (read-storm-config) + {STORM-NIMBUS-RETRY-TIMES 0})] + (is (thrown-cause? TTransportException (NimbusClient. storm-conf "" 65535) )) ) ) +(deftest test-isZkAuthenticationConfiguredTopology + (testing "Returns false on null config" + (is (not (Utils/isZkAuthenticationConfiguredTopology nil)))) + (testing "Returns false on scheme key missing" + (is (not (Utils/isZkAuthenticationConfiguredTopology + {STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME nil})))) + (testing "Returns false on scheme value null" + (is (not + (Utils/isZkAuthenticationConfiguredTopology + {STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME nil})))) + (testing "Returns true when scheme set to string" + (is + (Utils/isZkAuthenticationConfiguredTopology + {STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME "foobar"})))) + +(deftest test-isZkAuthenticationConfiguredStormServer + (let [k "java.security.auth.login.config" + oldprop (System/getProperty k)] + (try + (.remove (System/getProperties) k) + (testing "Returns false on null config" + (is (not (Utils/isZkAuthenticationConfiguredStormServer nil)))) + (testing "Returns false on scheme key missing" + (is (not (Utils/isZkAuthenticationConfiguredStormServer + {STORM-ZOOKEEPER-AUTH-SCHEME nil})))) + (testing "Returns false on scheme value null" + (is (not + (Utils/isZkAuthenticationConfiguredStormServer + {STORM-ZOOKEEPER-AUTH-SCHEME nil})))) + (testing "Returns true when scheme set to string" + (is + (Utils/isZkAuthenticationConfiguredStormServer + {STORM-ZOOKEEPER-AUTH-SCHEME "foobar"}))) + (testing "Returns true when java.security.auth.login.config is set" + (do + (System/setProperty k "anything") + (is (Utils/isZkAuthenticationConfiguredStormServer {})))) + (testing "Returns false when java.security.auth.login.config is set" + (do + (System/setProperty k "anything") + (is (Utils/isZkAuthenticationConfiguredStormServer {})))) + (finally + (if (not-nil? oldprop) + (System/setProperty k oldprop) + (.remove (System/getProperties) k)))))) + (deftest test-secs-to-millis-long (is (= 0 (secs-to-millis-long 0))) (is (= 2 (secs-to-millis-long 0.002))) diff --git a/storm-core/test/clj/storm/trident/state_test.clj b/storm-core/test/clj/storm/trident/state_test.clj index 63e38cab98d..c10c93cc371 100644 --- a/storm-core/test/clj/storm/trident/state_test.clj +++ b/storm-core/test/clj/storm/trident/state_test.clj @@ -19,10 +19,17 @@ (:import [storm.trident.operation.builtin Count]) (:import [storm.trident.state OpaqueValue]) (:import [storm.trident.state CombinerValueUpdater]) + (:import [storm.trident.topology.state TransactionalState TestTransactionalState]) (:import [storm.trident.state.map TransactionalMap OpaqueMap]) (:import [storm.trident.testing MemoryBackingMap MemoryMapState]) + (:import [backtype.storm.utils ZookeeperAuthInfo]) + (:import [org.apache.curator.framework CuratorFramework]) + (:import [org.apache.curator.framework.api CreateBuilder ProtectACLCreateModePathAndBytesable]) + (:import [org.apache.zookeeper CreateMode ZooDefs ZooDefs$Ids]) + (:import [org.mockito Matchers Mockito]) + (:import [org.mockito.exceptions.base MockitoAssertionError]) (:use [storm.trident testing]) - (:use [backtype.storm util])) + (:use [backtype.storm config util])) (defn single-remove [map key] (-> map (.multiRemove [[key]]))) @@ -103,6 +110,22 @@ (.commit map 2) )) +(deftest test-create-node-acl + (testing "Creates ZooKeeper nodes with the correct ACLs" + (let [curator (Mockito/mock CuratorFramework) + builder0 (Mockito/mock CreateBuilder) + builder1 (Mockito/mock ProtectACLCreateModePathAndBytesable) + expectedAcls ZooDefs$Ids/CREATOR_ALL_ACL] + (. (Mockito/when (.create curator)) (thenReturn builder0)) + (. (Mockito/when (.creatingParentsIfNeeded builder0)) (thenReturn builder1)) + (. (Mockito/when (.withMode builder1 (Matchers/isA CreateMode))) (thenReturn builder1)) + (. (Mockito/when (.withACL builder1 (Mockito/anyList))) (thenReturn builder1)) + (TestTransactionalState/createNode curator "" (byte-array 0) expectedAcls nil) + (is (nil? + (try + (. (Mockito/verify builder1) (withACL expectedAcls)) + (catch MockitoAssertionError e + e))))))) (deftest test-memory-map-state-remove (let [map (MemoryMapState. (uuid))] From ce6e5d26384d7f5d831b35e4eff126fe214981d1 Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Thu, 22 May 2014 18:34:23 +0000 Subject: [PATCH 02/61] rename test for consistent capitalization --- ...eAclAuthorizer_test.clj => DRPCSimpleACLAuthorizer_test.clj} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename storm-core/test/clj/backtype/storm/security/auth/authorizer/{DRPCSimpleAclAuthorizer_test.clj => DRPCSimpleACLAuthorizer_test.clj} (99%) diff --git a/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj b/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer_test.clj similarity index 99% rename from storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj rename to storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer_test.clj index d592a400fca..c70fa2a19e5 100644 --- a/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj +++ b/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer_test.clj @@ -1,4 +1,4 @@ -(ns backtype.storm.security.auth.authorizer.DRPCSimpleAclAuthorizer-test +(ns backtype.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer-test (:use [clojure test]) (:import [org.mockito Mockito]) (:import [backtype.storm Config]) From 6592b8209c49a98db15b3d6d228f488aa6c2e623 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Mon, 9 Jun 2014 15:42:39 +0000 Subject: [PATCH 03/61] Add missing ACLs to error znodes and remove auto vivification of error znodes --- storm-core/src/clj/backtype/storm/cluster.clj | 20 +++++++++++-------- .../clj/backtype/storm/daemon/executor.clj | 3 ++- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj index 7c43aea5f88..99698154f49 100644 --- a/storm-core/src/clj/backtype/storm/cluster.clj +++ b/storm-core/src/clj/backtype/storm/cluster.clj @@ -32,6 +32,7 @@ (get-data [this path watch?]) (get-children [this path watch?]) (mkdirs [this path acls]) + (exists-node? [this path watch?]) (close [this]) (register [this callback]) (unregister [this id]) @@ -110,6 +111,9 @@ (mkdirs [this path acls] (zk/mkdirs zk path acls)) + (exists-node? [this path watch?] + (zk/exists-node? zk path watch?)) + (close [this] (reset! active false) (.close zk)) @@ -388,14 +392,14 @@ (errors [this storm-id component-id] (let [path (error-path storm-id component-id) - _ (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) - maybe-deserialize)] - (when data - (struct TaskError (:error data) (:time-secs data)) - ))) + errors (if (exists-node? cluster-state path false) + (dofor [c (get-children cluster-state path false)] + (let [data (-> (get-data cluster-state (str path "/" c) false) + maybe-deserialize)] + (when data + (struct TaskError (:error data) (:time-secs data)) + ))) + ()) ] (->> (filter not-nil? errors) (sort-by (comp - :time-secs))))) diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj index d193a9fc0ac..29cce08f0c5 100644 --- a/storm-core/src/clj/backtype/storm/daemon/executor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj @@ -229,7 +229,8 @@ :batch-transfer-queue 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)) + :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker) + :acls (Utils/getWorkerACL storm-conf)) :type executor-type ;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field) :stats (mk-executor-stats <> (sampling-rate storm-conf)) From cf2e8b7ee06b455a90bd4b3bfd53facef1369612 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 10 Jun 2014 15:01:33 -0700 Subject: [PATCH 04/61] Storm 344. (Security) nimbus renew-credentials not calling ICredentialsRenewer.renew --- storm-core/src/clj/backtype/storm/daemon/nimbus.clj | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj index 0bd5337d35e..9502745c88e 100644 --- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj @@ -869,7 +869,8 @@ (if orig-creds (let [new-creds (HashMap. orig-creds)] (doseq [renewer renewers] - (log-message "Renewing Creds For " id " with " renewer)) + (log-message "Renewing Creds For " id " with " renewer) + (.renew renewer new-creds)) (when-not (= orig-creds new-creds) (.set-credentials! storm-cluster-state id new-creds) ))))))))) From 2131a0aeb9074b2c83a09d7515ff8e8ae86f6eaf Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 12 Jun 2014 10:58:13 -0500 Subject: [PATCH 05/61] Added back in the user to the web ui. --- storm-core/src/clj/backtype/storm/ui/core.clj | 20 +++++++++++-------- storm-core/src/ui/public/component.html | 7 +++++++ storm-core/src/ui/public/index.html | 8 +++++++- .../ui/public/templates/user-template.html | 9 +++++++++ storm-core/src/ui/public/topology.html | 7 +++++++ 5 files changed, 42 insertions(+), 9 deletions(-) create mode 100644 storm-core/src/ui/public/templates/user-template.html diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj index 216c3046adf..9f8f3b800d0 100644 --- a/storm-core/src/clj/backtype/storm/ui/core.clj +++ b/storm-core/src/clj/backtype/storm/ui/core.clj @@ -452,10 +452,10 @@ (.getNimbusConf ^Nimbus$Client nimbus))) (defn cluster-summary - ([] + ([user] (with-nimbus nimbus - (cluster-summary (.getClusterInfo ^Nimbus$Client nimbus)))) - ([^ClusterSummary summ] + (cluster-summary (.getClusterInfo ^Nimbus$Client nimbus) user))) + ([^ClusterSummary summ user] (let [sups (.get_supervisors summ) used-slots (reduce + (map #(.get_num_used_workers ^SupervisorSummary %) sups)) total-slots (reduce + (map #(.get_num_workers ^SupervisorSummary %) sups)) @@ -466,7 +466,8 @@ total-executors (->> (.get_topologies summ) (map #(.get_num_executors ^TopologySummary %)) (reduce +))] - { "stormVersion" (read-storm-version) + { "user" user + "stormVersion" (read-storm-version) "nimbusUptime" (pretty-uptime-sec (.get_nimbus_uptime_secs summ)) "supervisors" (count sups) "slotsTotal" total-slots @@ -619,7 +620,8 @@ (assert-authorized-ui-user user *STORM-CONF* topology-conf) (merge (topology-summary summ) - {"window" window + {"user" user + "window" window "windowHint" window-hint "msgTimeout" msg-timeout "topologyStats" (topology-stats id window (total-aggregate-stats spout-summs bolt-summs include-sys?)) @@ -778,7 +780,8 @@ errors (component-errors (get (.get_errors summ) component))] (assert-authorized-ui-user user *STORM-CONF* topology-conf) (merge - {"id" component + {"user" user + "id" component "name" (.get_name summ) "executors" (count summs) "tasks" (sum-tasks summs) @@ -802,8 +805,9 @@ (defroutes main-routes (GET "/api/v1/cluster/configuration" [] (cluster-configuration)) - (GET "/api/v1/cluster/summary" [] - (json-response (cluster-summary))) + (GET "/api/v1/cluster/summary" [:as {:keys [cookies servlet-request]}] + (let [user (.getUserName http-creds-handler servlet-request)] + (json-response (cluster-summary user)))) (GET "/api/v1/supervisor/summary" [] (json-response (supervisor-summary))) (GET "/api/v1/topology/summary" [] diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html index 90ca6306fd5..337d09b4d28 100644 --- a/storm-core/src/ui/public/component.html +++ b/storm-core/src/ui/public/component.html @@ -12,6 +12,8 @@ +
+

Storm UI

@@ -47,6 +49,11 @@

Storm UI

}); $.getJSON(url,function(response,status,jqXHR) { + var uiUser = $("#ui-user"); + $.get("/templates/user-template.html", function(template) { + uiUser.append(Mustache.render($(template).filter("#user-template").html(),response)); + }); + var componentSummary = $("#component-summary"); var componentStatsDetail = $("#component-stats-detail") var inputStats = $("#component-input-stats"); diff --git a/storm-core/src/ui/public/index.html b/storm-core/src/ui/public/index.html index fe83b2cc300..2bf616ff842 100644 --- a/storm-core/src/ui/public/index.html +++ b/storm-core/src/ui/public/index.html @@ -10,6 +10,8 @@ +
+

Storm UI

Cluster Summary

@@ -34,13 +36,17 @@

Nimbus Configuration

}); } }); - var template = $.get("/templates/index-page-template.html"); + var uiUser = $("#ui-user"); var clusterSummary = $("#cluster-summary"); var topologySummary = $("#topology-summary"); var supervisorSummary = $("#supervisor-summary"); var config = $("#nimbus-configuration"); $.getJSON("/api/v1/cluster/summary",function(response,status,jqXHR) { + $.get("/templates/user-template.html", function(template) { + uiUser.append(Mustache.render($(template).filter("#user-template").html(),response)); + }); + $.get("/templates/index-page-template.html", function(template) { clusterSummary.append(Mustache.render($(template).filter("#cluster-summary-template").html(),response)); }); diff --git a/storm-core/src/ui/public/templates/user-template.html b/storm-core/src/ui/public/templates/user-template.html new file mode 100644 index 00000000000..2b4eaa404a5 --- /dev/null +++ b/storm-core/src/ui/public/templates/user-template.html @@ -0,0 +1,9 @@ + diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index b2ac1e54af1..5f7480d0334 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -15,6 +15,8 @@ +
+

Storm UI

Topology summary

@@ -52,6 +54,11 @@

Topology summary

}); $.getJSON(url,function(response,status,jqXHR) { + var uiUser = $("#ui-user"); + $.get("/templates/user-template.html", function(template) { + uiUser.append(Mustache.render($(template).filter("#user-template").html(),response)); + }); + var topologySummary = $("#topology-summary"); var topologyStats = $("#topology-stats"); var spoutStats = $("#spout-stats"); From bc91ed88d77e392f38c406d143e7ac37bc634564 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 12 Jun 2014 11:01:35 -0500 Subject: [PATCH 06/61] Added license to UI template. --- .../src/ui/public/templates/user-template.html | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/storm-core/src/ui/public/templates/user-template.html b/storm-core/src/ui/public/templates/user-template.html index 2b4eaa404a5..9c0c5d8a3a4 100644 --- a/storm-core/src/ui/public/templates/user-template.html +++ b/storm-core/src/ui/public/templates/user-template.html @@ -1,3 +1,19 @@ + - diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index 74afb087d6a..db1a500c7de 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -89,7 +89,9 @@

Topology summary

topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response)); topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData)); topologyStats.append(Mustache.render($(template).filter("#topology-stats-template").html(),response)); - $("#topology-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + if(response["topologyStats"].length > 0) { + $("#topology-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + } spoutStats.append(Mustache.render($(template).filter("#spout-stats-template").html(),response)); if(response["spouts"].length > 0) { $("#spout-stats-table").tablesorter({sortList: [[0,0]], headers:{}}); From 28c168fd7d0272f88d586f6f572eab937b874f22 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Tue, 24 Jun 2014 19:12:56 +0000 Subject: [PATCH 13/61] Add check for empty table before sorting on security --- storm-core/src/ui/public/component.html | 22 ++++++++++++++++------ storm-core/src/ui/public/topology.html | 4 +++- 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html index 0951aed3004..092eb4066ab 100644 --- a/storm-core/src/ui/public/component.html +++ b/storm-core/src/ui/public/component.html @@ -80,22 +80,32 @@

Storm UI

componentSummary.append(Mustache.render($(template).filter("#component-summary-template").html(),response)); if(response["componentType"] == "spout") { componentStatsDetail.append(Mustache.render($(template).filter("#spout-stats-detail-template").html(),response)); - $("#spout-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + if (response["spoutSummary"].length > 0) { + $("#spout-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + } outputStats.append(Mustache.render($(template).filter("#output-stats-template").html(),response)); - $("#output-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + if (response["outputStats"].length > 0) { + $("#output-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + } executorStats.append(Mustache.render($(template).filter("#executor-stats-template").html(),response)); - $("#executor-stats-table").tablesorter({ sortList: [[0,0]], headers: {1: { sorter: "stormtimestr"}}}); + if (response["executorStats"].length > 0) { + $("#executor-stats-table").tablesorter({ sortList: [[0,0]], headers: {1: { sorter: "stormtimestr"}}}); + } } else { componentStatsDetail.append(Mustache.render($(template).filter("#bolt-stats-template").html(),response)); - $("#bolt-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + if (response["boltStats"].length > 0) { + $("#bolt-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + } inputStats.append(Mustache.render($(template).filter("#bolt-input-stats-template").html(),response)); if (response["inputStats"].length > 0) { $("#bolt-input-stats-table").tablesorter({ sortList: [[0,0]], headers: {}}); } outputStats.append(Mustache.render($(template).filter("#bolt-output-stats-template").html(),response)); - $("#bolt-output-stats-table").tablesorter({ sortList: [[0,0]], headers: {}}); + if (response["outputStats"].length > 0) { + $("#bolt-output-stats-table").tablesorter({ sortList: [[0,0]], headers: {}}); + } executorStats.append(Mustache.render($(template).filter("#bolt-executor-template").html(),response)); - if(response["outputStats"].length > 0) { + if(response["executorStats"].length > 0) { $("#bolt-executor-table").tablesorter({ sortList: [[0,0]], headers: {}}); } } diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index 74afb087d6a..db1a500c7de 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -89,7 +89,9 @@

Topology summary

topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response)); topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData)); topologyStats.append(Mustache.render($(template).filter("#topology-stats-template").html(),response)); - $("#topology-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + if(response["topologyStats"].length > 0) { + $("#topology-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); + } spoutStats.append(Mustache.render($(template).filter("#spout-stats-template").html(),response)); if(response["spouts"].length > 0) { $("#spout-stats-table").tablesorter({sortList: [[0,0]], headers:{}}); From 8325186b731259c17d4b06aa2c288ffc4f6dc59b Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 25 Jun 2014 15:32:17 +0000 Subject: [PATCH 14/61] revert tablesorter fix --- storm-core/src/ui/public/component.html | 22 ++++++---------------- storm-core/src/ui/public/topology.html | 4 +--- 2 files changed, 7 insertions(+), 19 deletions(-) diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html index 092eb4066ab..0951aed3004 100644 --- a/storm-core/src/ui/public/component.html +++ b/storm-core/src/ui/public/component.html @@ -80,32 +80,22 @@

Storm UI

componentSummary.append(Mustache.render($(template).filter("#component-summary-template").html(),response)); if(response["componentType"] == "spout") { componentStatsDetail.append(Mustache.render($(template).filter("#spout-stats-detail-template").html(),response)); - if (response["spoutSummary"].length > 0) { - $("#spout-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); - } + $("#spout-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); outputStats.append(Mustache.render($(template).filter("#output-stats-template").html(),response)); - if (response["outputStats"].length > 0) { - $("#output-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); - } + $("#output-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); executorStats.append(Mustache.render($(template).filter("#executor-stats-template").html(),response)); - if (response["executorStats"].length > 0) { - $("#executor-stats-table").tablesorter({ sortList: [[0,0]], headers: {1: { sorter: "stormtimestr"}}}); - } + $("#executor-stats-table").tablesorter({ sortList: [[0,0]], headers: {1: { sorter: "stormtimestr"}}}); } else { componentStatsDetail.append(Mustache.render($(template).filter("#bolt-stats-template").html(),response)); - if (response["boltStats"].length > 0) { - $("#bolt-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); - } + $("#bolt-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); inputStats.append(Mustache.render($(template).filter("#bolt-input-stats-template").html(),response)); if (response["inputStats"].length > 0) { $("#bolt-input-stats-table").tablesorter({ sortList: [[0,0]], headers: {}}); } outputStats.append(Mustache.render($(template).filter("#bolt-output-stats-template").html(),response)); - if (response["outputStats"].length > 0) { - $("#bolt-output-stats-table").tablesorter({ sortList: [[0,0]], headers: {}}); - } + $("#bolt-output-stats-table").tablesorter({ sortList: [[0,0]], headers: {}}); executorStats.append(Mustache.render($(template).filter("#bolt-executor-template").html(),response)); - if(response["executorStats"].length > 0) { + if(response["outputStats"].length > 0) { $("#bolt-executor-table").tablesorter({ sortList: [[0,0]], headers: {}}); } } diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index db1a500c7de..74afb087d6a 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -89,9 +89,7 @@

Topology summary

topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response)); topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData)); topologyStats.append(Mustache.render($(template).filter("#topology-stats-template").html(),response)); - if(response["topologyStats"].length > 0) { - $("#topology-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); - } + $("#topology-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}}); spoutStats.append(Mustache.render($(template).filter("#spout-stats-template").html(),response)); if(response["spouts"].length > 0) { $("#spout-stats-table").tablesorter({sortList: [[0,0]], headers:{}}); From 7bb8ec6d8068fe2eeec65085bfa0140aca0fb0a9 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 26 Jun 2014 00:13:03 +0000 Subject: [PATCH 15/61] Use javascript to color-code recent errors --- storm-core/src/clj/backtype/storm/ui/core.clj | 14 +++++++------- storm-core/src/ui/public/component.html | 10 ++++++++++ .../public/templates/component-page-template.html | 4 +--- .../public/templates/topology-page-template.html | 8 ++------ storm-core/src/ui/public/topology.html | 10 ++++++++++ 5 files changed, 30 insertions(+), 16 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj index bc3f5069b98..a7492f90624 100644 --- a/storm-core/src/clj/backtype/storm/ui/core.clj +++ b/storm-core/src/clj/backtype/storm/ui/core.clj @@ -319,11 +319,11 @@ (map nil-to-zero) (apply max))) -(defn is-error-recent? +(defn get-error-time [error] - (if error (< (time-delta (.get_error_time_secs ^ErrorInfo error)) - (* 60 30)) - false)) + (if error + (time-delta (.get_error_time_secs ^ErrorInfo error)) + "")) (defn get-error-data [error] @@ -596,7 +596,7 @@ "errorHost" error-host "errorPort" error-port "errorWorkerLogLink" (worker-log-link error-host error-port top-id) - (if (is-error-recent? last-error) "isRecent" "isNotRecent") true + "errorLapsedSecs" (get-error-time last-error) "lastError" (get-error-data last-error)})) (defn bolt-comp [top-id summ-map errors window include-sys?] @@ -622,7 +622,7 @@ "errorHost" error-host "errorPort" error-port "errorWorkerLogLink" (worker-log-link error-host error-port top-id) - (if (is-error-recent? last-error) "isRecent" "isNotRecent") true + "errorLapsedSecs" (get-error-time last-error) "lastError" (get-error-data last-error)})) (defn topology-summary [^TopologyInfo summ] @@ -733,7 +733,7 @@ "errorHost" (.get_host e) "errorPort" (.get_port e) "errorWorkerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id) - (if (is-error-recent? e) "isRecent" "isNotRecent") true + "errorLapsedSecs" (get-error-time e) "error" (.get_error e)})})) (defn spout-stats diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html index 0951aed3004..d0e0869370a 100644 --- a/storm-core/src/ui/public/component.html +++ b/storm-core/src/ui/public/component.html @@ -103,6 +103,16 @@

Storm UI

if(response["componentErrors"].length > 0) { $("#component-errors-table").tablesorter({ sortList: [[0,0]], headers: {1: { sorter: "stormtimestr"}}}); } + + var errorCells = document.getElementsByClassName("errorSpan"); + for (i =0; i < errorCells.length; i++) + { + var timeLapsedInSecs = errorCells[i].id; + if (parseInt(timeLapsedInSecs) < 1800) { + errorCells[i].style.color = "#9d261d"; + errorCells[i].style.borderBottomColor = "#9d261d"; + } + } }); }); }); diff --git a/storm-core/src/ui/public/templates/component-page-template.html b/storm-core/src/ui/public/templates/component-page-template.html index 4aac1e83f17..561dc4a779a 100644 --- a/storm-core/src/ui/public/templates/component-page-template.html +++ b/storm-core/src/ui/public/templates/component-page-template.html @@ -164,9 +164,7 @@

Errors

{{errorHost}} {{errorPort}} - -{{error}} - +{{error}} {{/componentErrors}} diff --git a/storm-core/src/ui/public/templates/topology-page-template.html b/storm-core/src/ui/public/templates/topology-page-template.html index 188a6f1f20c..c2490539e0a 100644 --- a/storm-core/src/ui/public/templates/topology-page-template.html +++ b/storm-core/src/ui/public/templates/topology-page-template.html @@ -125,9 +125,7 @@

Spouts ({{windowHint}})

{{errorHost}} {{errorPort}} - - {{lastError}} - + {{lastError}} {{/spouts}} @@ -155,9 +153,7 @@

Bolts ({{windowHint}})

{{errorHost}} {{errorPort}} - - {{lastError}} - + {{lastError}} {{/bolts}} diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index 74afb087d6a..b7d78552e7d 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -104,6 +104,16 @@

Topology summary

config.append(Mustache.render($(template).filter("#topology-configuration-template").html(),formattedConfig)); $("#topology-configuration-table").tablesorter({ sortList: [[0,0]], headers: {}}); + + var errorCells = document.getElementsByClassName("errorSpan"); + for (i =0; i < errorCells.length; i++) + { + var timeLapsedInSecs = errorCells[i].id; + if (parseInt(timeLapsedInSecs) < 1800) { + errorCells[i].style.color = "#9d261d"; + errorCells[i].style.borderBottomColor = "#9d261d"; + } + } }); }); }); From f3112fa76e98038da83170513a476499dde0bb41 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 26 Jun 2014 15:05:21 -0700 Subject: [PATCH 16/61] STORM-347. (Security) authentication should allow for groups not just users. --- conf/defaults.yaml | 3 +- storm-core/src/jvm/backtype/storm/Config.java | 6 + .../storm/security/auth/AuthUtils.java | 49 +- .../auth/IGroupMappingServiceProvider.java | 54 ++ .../auth/ShellBasedUnixGroupsMapping.java | 88 ++++ .../auth/authorizer/SimpleACLAuthorizer.java | 29 +- .../jvm/backtype/storm/utils/ShellUtils.java | 498 ++++++++++++++++++ 7 files changed, 702 insertions(+), 25 deletions(-) create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/IGroupMappingServiceProvider.java create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java create mode 100644 storm-core/src/jvm/backtype/storm/utils/ShellUtils.java diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 7f17054e0eb..83b7b4d2358 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -38,6 +38,7 @@ 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.group.mapping.service: "backtype.storm.security.auth.ShellBasedUnixGroupsMapping" storm.messaging.transport: "backtype.storm.messaging.netty.Context" storm.nimbus.retry.times: 5 storm.nimbus.retry.interval.millis: 2000 @@ -128,7 +129,7 @@ worker.childopts: "-Xmx768m" worker.gc.childopts: "" worker.heartbeat.frequency.secs: 1 -# control how many worker receiver threads we need per worker +# control how many worker receiver threads we need per worker topology.worker.receiver.thread.count: 1 task.heartbeat.frequency.secs: 3 diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 3b3f7e54f15..fee5f6eabb0 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -157,6 +157,12 @@ public class Config extends HashMap { 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 plugin that will provide user groups service + */ + public static final String STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN = "storm.group.mapping.service"; + public static final Object STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN_SCHEMA = String.class; + /** * The default transport plug-in for Thrift client/server communication */ diff --git a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java index b5118c7f6f8..7cfc9272fd6 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java @@ -37,21 +37,21 @@ public class AuthUtils { private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class); - public static final String LOGIN_CONTEXT_SERVER = "StormServer"; - public static final String LOGIN_CONTEXT_CLIENT = "StormClient"; + public static final String LOGIN_CONTEXT_SERVER = "StormServer"; + public static final String LOGIN_CONTEXT_CLIENT = "StormClient"; public static final String SERVICE = "storm_thrift_server"; /** - * Construct a JAAS configuration object per storm configuration file - * @param storm_conf Storm configuration + * Construct a JAAS configuration object per storm configuration file + * @param storm_conf Storm configuration * @return JAAS configuration object */ public static Configuration GetConfiguration(Map storm_conf) { Configuration login_conf = null; - //find login file configuration from Storm configuration + //find login file configuration from Storm configuration String loginConfigurationFile = (String)storm_conf.get("java.security.auth.login.config"); - if ((loginConfigurationFile != null) && (loginConfigurationFile.length()>0)) { + if ((loginConfigurationFile != null) && (loginConfigurationFile.length()>0)) { File config_file = new File(loginConfigurationFile); if (! config_file.canRead()) { throw new RuntimeException("File " + loginConfigurationFile + @@ -64,7 +64,7 @@ public static Configuration GetConfiguration(Map storm_conf) { throw new RuntimeException(ex); } } - + return login_conf; } @@ -86,11 +86,29 @@ public static IPrincipalToLocal GetPrincipalToLocalPlugin(Map storm_conf) { return ptol; } + /** + * Construct a group mapping service provider plugin + * @param conf storm configuration + * @return the plugin + */ + public static IGroupMappingServiceProvider GetGroupMappingServiceProviderPlugin(Map storm_conf) { + IGroupMappingServiceProvider gmsp = null; + try { + String gmsp_klassName = (String) storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN); + Class klass = Class.forName(gmsp_klassName); + gmsp = (IGroupMappingServiceProvider)klass.newInstance(); + gmsp.prepare(storm_conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + return gmsp; + } + /** * Get all of the configured Credential Renwer Plugins. * @param storm_conf the storm configuration to use. * @return the configured credential renewers. - */ + */ public static Collection GetCredentialRenewers(Map conf) { try { Set ret = new HashSet(); @@ -112,7 +130,7 @@ public static Collection GetCredentialRenewers(Map conf) { * Get all of the configured AutoCredential Plugins. * @param storm_conf the storm configuration to use. * @return the configured auto credentials. - */ + */ public static Collection GetAutoCredentials(Map storm_conf) { try { Set autos = new HashSet(); @@ -137,7 +155,7 @@ public static Collection GetAutoCredentials(Map storm_conf) { * @param autos the IAutoCredentials to call to populate the subject. * @param credentials the credentials to pull from * @return the populated subject. - */ + */ public static Subject populateSubject(Subject subject, Collection autos, Map credentials) { try { if (subject == null) { @@ -154,10 +172,10 @@ public static Subject populateSubject(Subject subject, Collection autos, Map credentials) { if (subject == null) { throw new RuntimeException("The subject cannot be null when updating a subject with credentials"); @@ -186,7 +204,7 @@ public static ITransportPlugin GetTransportPlugin(ThriftConnectionType type, Map transportPlugin.prepare(type, storm_conf, login_conf); } catch(Exception e) { throw new RuntimeException(e); - } + } return transportPlugin; } @@ -233,11 +251,10 @@ public static String get(Configuration configuration, String section, String key } for(AppConfigurationEntry entry: configurationEntries) { - Object val = entry.getOptions().get(key); - if (val != null) + Object val = entry.getOptions().get(key); + if (val != null) return (String)val; } return null; } } - diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IGroupMappingServiceProvider.java b/storm-core/src/jvm/backtype/storm/security/auth/IGroupMappingServiceProvider.java new file mode 100644 index 00000000000..0b49deccf4c --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/IGroupMappingServiceProvider.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.io.IOException; +import java.util.Set; +import java.util.Map; + +public interface IGroupMappingServiceProvider { + + /** + * Invoked once immediately after construction + * @param storm_conf Storm configuration + */ + void prepare(Map storm_conf); + + /** + * Get all various group memberships of a given user. + * Returns EMPTY list in case of non-existing user + * @param user User's name + * @return group memberships of user + * @throws IOException + */ + public Set getGroups(String user) throws IOException; + + /** + * Refresh the cache of groups and user mapping + * @throws IOException + */ + public void cacheGroupsRefresh() throws IOException; + /** + * Caches the group user information + * @param groups list of groups to add to cache + * @throws IOException + */ + public void cacheGroupsAdd(Set groups) throws IOException; + +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java new file mode 100644 index 00000000000..b8c83237a99 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.io.IOException; +import java.util.Set; +import java.util.HashSet; +import java.util.Map; +import java.util.StringTokenizer; +import backtype.storm.utils.ShellUtils; +import backtype.storm.utils.ShellUtils.ExitCodeException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class ShellBasedUnixGroupsMapping implements + IGroupMappingServiceProvider { + + public static Logger LOG = LoggerFactory.getLogger(ShellBasedUnixGroupsMapping.class); + + /** + * Invoked once immediately after construction + * @param storm_conf Storm configuration + */ + public void prepare(Map storm_conf) {} + + /** + * Returns list of groups for a user + * + * @param user get groups for this user + * @return list of groups for a given user + */ + @Override + public Set getGroups(String user) throws IOException { + return getUnixGroups(user); + } + + @Override + public void cacheGroupsRefresh() throws IOException { + } + + @Override + public void cacheGroupsAdd(Set groups) throws IOException { + } + + /** + * Get the current user's group list from Unix by running the command 'groups' + * NOTE. For non-existing user it will return EMPTY list + * @param user user name + * @return the groups set that the user belongs to + * @throws IOException if encounter any error when running the command + */ + private static Set getUnixGroups(final String user) throws IOException { + String result = ""; + try { + result = ShellUtils.execCommand(ShellUtils.getGroupsForUserCommand(user)); + } catch (ExitCodeException e) { + // if we didn't get the group - just return empty list; + LOG.warn("got exception trying to get groups for user " + user, e); + return new HashSet(); + } + + StringTokenizer tokenizer = + new StringTokenizer(result, ShellUtils.TOKEN_SEPARATOR_REGEX); + Set groups = new HashSet(); + while (tokenizer.hasMoreTokens()) { + groups.add(tokenizer.nextToken()); + } + return groups; + } + +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java index ef13750f6f4..0b867e8db05 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java @@ -23,12 +23,14 @@ import java.util.Set; import java.util.HashSet; import java.util.Collection; +import java.io.IOException; import backtype.storm.Config; import backtype.storm.security.auth.IAuthorizer; import backtype.storm.security.auth.ReqContext; import backtype.storm.security.auth.AuthUtils; import backtype.storm.security.auth.IPrincipalToLocal; +import backtype.storm.security.auth.IGroupMappingServiceProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,10 +49,10 @@ public class SimpleACLAuthorizer implements IAuthorizer { protected Set _admins; protected Set _supervisors; protected IPrincipalToLocal _ptol; - + protected IGroupMappingServiceProvider _groups; /** * Invoked once immediately after construction - * @param conf Storm configuration + * @param conf Storm configuration */ @Override public void prepare(Map conf) { @@ -64,27 +66,26 @@ public void prepare(Map conf) { _supervisors.addAll((Collection)conf.get(Config.NIMBUS_SUPERVISOR_USERS)); } _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); + _groups = AuthUtils.GetGroupMappingServiceProviderPlugin(conf); } /** * permit() method is invoked for each incoming Thrift request - * @param context request context includes info about + * @param context request context includes info about * @param operation operation name - * @param topology_storm configuration of targeted topology + * @param topology_storm configuration of targeted topology * @return true if the request is authorized, false if reject */ @Override public boolean permit(ReqContext context, String operation, Map topology_conf) { - LOG.info("[req "+ context.requestID()+ "] Access " + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) + (context.principal() == null? "" : (" principal:"+ context.principal())) +" op:"+operation + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)))); - + String principal = context.principal().getName(); String user = _ptol.toLocal(context.principal()); - if (_admins.contains(principal) || _admins.contains(user)) { return true; } @@ -106,8 +107,20 @@ public boolean permit(ReqContext context, String operation, Map topology_conf) { if (topoUsers.contains(principal) || topoUsers.contains(user)) { return true; } + if(_groups != null) { + try { + String topologySubmitterUser = (String) topology_conf.get(Config.TOPOLOGY_SUBMITTER_USER); + Set userGroups = _groups.getGroups(user); + Set topoUserGroups = _groups.getGroups(topologySubmitterUser); + for (String tgroup : topoUserGroups) { + if(userGroups.contains(tgroup)) + return true; + } + } catch(IOException e) { + LOG.warn("Error while trying to fetch user groups",e); + } + } } - return false; } } diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java b/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java new file mode 100644 index 00000000000..1065ff92f48 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java @@ -0,0 +1,498 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.utils; + +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.InputStream; +import java.util.Map; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + + +abstract public class ShellUtils { + public static Logger LOG = LoggerFactory.getLogger(ShellUtils.class); + + // OSType detection + public enum OSType { + OS_TYPE_LINUX, + OS_TYPE_WIN, + OS_TYPE_SOLARIS, + OS_TYPE_MAC, + OS_TYPE_FREEBSD, + OS_TYPE_OTHER + } + + public static final OSType osType = getOSType(); + + static private OSType getOSType() { + String osName = System.getProperty("os.name"); + if (osName.startsWith("Windows")) { + return OSType.OS_TYPE_WIN; + } else if (osName.contains("SunOS") || osName.contains("Solaris")) { + return OSType.OS_TYPE_SOLARIS; + } else if (osName.contains("Mac")) { + return OSType.OS_TYPE_MAC; + } else if (osName.contains("FreeBSD")) { + return OSType.OS_TYPE_FREEBSD; + } else if (osName.startsWith("Linux")) { + return OSType.OS_TYPE_LINUX; + } else { + // Some other form of Unix + return OSType.OS_TYPE_OTHER; + } + } + + // Helper static vars for each platform + public static final boolean WINDOWS = (osType == OSType.OS_TYPE_WIN); + public static final boolean SOLARIS = (osType == OSType.OS_TYPE_SOLARIS); + public static final boolean MAC = (osType == OSType.OS_TYPE_MAC); + public static final boolean FREEBSD = (osType == OSType.OS_TYPE_FREEBSD); + public static final boolean LINUX = (osType == OSType.OS_TYPE_LINUX); + public static final boolean OTHER = (osType == OSType.OS_TYPE_OTHER); + + + /** Token separator regex used to parse Shell tool outputs */ + public static final String TOKEN_SEPARATOR_REGEX + = WINDOWS ? "[|\n\r]" : "[ \t\n\r\f]"; + + private long interval; // refresh interval in msec + private long lastTime; // last time the command was performed + final private boolean redirectErrorStream; // merge stdout and stderr + private Map environment; // env for the command execution + private File dir; + private Process process; // sub process used to execute the command + private int exitCode; + /**Time after which the executing script would be timedout*/ + protected long timeOutInterval = 0L; + /** If or not script timed out*/ + private AtomicBoolean timedOut; + + /**If or not script finished executing*/ + private volatile AtomicBoolean completed; + + public ShellUtils() { + this(0L); + } + + public ShellUtils(long interval) { + this(interval, false); + } + + /** + * @param interval the minimum duration to wait before re-executing the + * command. + */ + public ShellUtils(long interval, boolean redirectErrorStream) { + this.interval = interval; + this.lastTime = (interval<0) ? 0 : -interval; + this.redirectErrorStream = redirectErrorStream; + } + + /** set the environment for the command + * @param env Mapping of environment variables + */ + protected void setEnvironment(Map env) { + this.environment = env; + } + + /** set the working directory + * @param dir The directory where the command would be executed + */ + protected void setWorkingDirectory(File dir) { + this.dir = dir; + } + + /** a Unix command to get the current user's groups list */ + public static String[] getGroupsCommand() { + return (WINDOWS)? new String[]{"cmd", "/c", "groups"} + : new String[]{"bash", "-c", "groups"}; + } + + /** + * a Unix command to get a given user's groups list. + * If the OS is not WINDOWS, the command will get the user's primary group + * first and finally get the groups list which includes the primary group. + * i.e. the user's primary group will be included twice. + */ + public static String[] getGroupsForUserCommand(final String user) { + //'groups username' command return is non-consistent across different unixes + return new String [] {"bash", "-c", "id -gn " + user + + "&& id -Gn " + user}; + } + + + /** check to see if a command needs to be executed and execute if needed */ + protected void run() throws IOException { + if (lastTime + interval > System.currentTimeMillis()) + return; + exitCode = 0; // reset for next run + runCommand(); + } + + /** Run a command */ + private void runCommand() throws IOException { + ProcessBuilder builder = new ProcessBuilder(getExecString()); + Timer timeOutTimer = null; + ShellTimeoutTimerTask timeoutTimerTask = null; + timedOut = new AtomicBoolean(false); + completed = new AtomicBoolean(false); + + if (environment != null) { + builder.environment().putAll(this.environment); + } + if (dir != null) { + builder.directory(this.dir); + } + + builder.redirectErrorStream(redirectErrorStream); + process = builder.start(); + + if (timeOutInterval > 0) { + timeOutTimer = new Timer("Shell command timeout"); + timeoutTimerTask = new ShellTimeoutTimerTask(this); + //One time scheduling. + timeOutTimer.schedule(timeoutTimerTask, timeOutInterval); + } + final BufferedReader errReader = + new BufferedReader(new InputStreamReader(process + .getErrorStream())); + BufferedReader inReader = + new BufferedReader(new InputStreamReader(process + .getInputStream())); + final StringBuffer errMsg = new StringBuffer(); + + // read error and input streams as this would free up the buffers + // free the error stream buffer + Thread errThread = new Thread() { + @Override + public void run() { + try { + String line = errReader.readLine(); + while((line != null) && !isInterrupted()) { + errMsg.append(line); + errMsg.append(System.getProperty("line.separator")); + line = errReader.readLine(); + } + } catch(IOException ioe) { + LOG.warn("Error reading the error stream", ioe); + } + } + }; + try { + errThread.start(); + } catch (IllegalStateException ise) { } + try { + parseExecResult(inReader); // parse the output + // clear the input stream buffer + String line = inReader.readLine(); + while(line != null) { + line = inReader.readLine(); + } + // wait for the process to finish and check the exit code + exitCode = process.waitFor(); + // make sure that the error thread exits + joinThread(errThread); + completed.set(true); + //the timeout thread handling + //taken care in finally block + if (exitCode != 0) { + throw new ExitCodeException(exitCode, errMsg.toString()); + } + } catch (InterruptedException ie) { + throw new IOException(ie.toString()); + } finally { + if (timeOutTimer != null) { + timeOutTimer.cancel(); + } + // close the input stream + try { + // JDK 7 tries to automatically drain the input streams for us + // when the process exits, but since close is not synchronized, + // it creates a race if we close the stream first and the same + // fd is recycled. the stream draining thread will attempt to + // drain that fd!! it may block, OOM, or cause bizarre behavior + // see: https://bugs.openjdk.java.net/browse/JDK-8024521 + // issue is fixed in build 7u60 + InputStream stdout = process.getInputStream(); + synchronized (stdout) { + inReader.close(); + } + } catch (IOException ioe) { + LOG.warn("Error while closing the input stream", ioe); + } + if (!completed.get()) { + errThread.interrupt(); + joinThread(errThread); + } + try { + InputStream stderr = process.getErrorStream(); + synchronized (stderr) { + errReader.close(); + } + } catch (IOException ioe) { + LOG.warn("Error while closing the error stream", ioe); + } + process.destroy(); + lastTime = System.currentTimeMillis(); + } + } + + private static void joinThread(Thread t) { + while (t.isAlive()) { + try { + t.join(); + } catch (InterruptedException ie) { + if (LOG.isWarnEnabled()) { + LOG.warn("Interrupted while joining on: " + t, ie); + } + t.interrupt(); // propagate interrupt + } + } + } + + /** return an array containing the command name & its parameters */ + protected abstract String[] getExecString(); + + /** Parse the execution result */ + protected abstract void parseExecResult(BufferedReader lines) + throws IOException; + + /** get the current sub-process executing the given command + * @return process executing the command + */ + public Process getProcess() { + return process; + } + + /** + * This is an IOException with exit code added. + */ + public static class ExitCodeException extends IOException { + int exitCode; + + public ExitCodeException(int exitCode, String message) { + super(message); + this.exitCode = exitCode; + } + + public int getExitCode() { + return exitCode; + } + } + + /** + * A simple shell command executor. + * + * ShellCommandExecutorshould be used in cases where the output + * of the command needs no explicit parsing and where the command, working + * directory and the environment remains unchanged. The output of the command + * is stored as-is and is expected to be small. + */ + public static class ShellCommandExecutor extends ShellUtils { + + private String[] command; + private StringBuffer output; + + + public ShellCommandExecutor(String[] execString) { + this(execString, null); + } + + public ShellCommandExecutor(String[] execString, File dir) { + this(execString, dir, null); + } + + public ShellCommandExecutor(String[] execString, File dir, + Map env) { + this(execString, dir, env , 0L); + } + + /** + * Create a new instance of the ShellCommandExecutor to execute a command. + * + * @param execString The command to execute with arguments + * @param dir If not-null, specifies the directory which should be set + * as the current working directory for the command. + * If null, the current working directory is not modified. + * @param env If not-null, environment of the command will include the + * key-value pairs specified in the map. If null, the current + * environment is not modified. + * @param timeout Specifies the time in milliseconds, after which the + * command will be killed and the status marked as timedout. + * If 0, the command will not be timed out. + */ + public ShellCommandExecutor(String[] execString, File dir, + Map env, long timeout) { + command = execString.clone(); + if (dir != null) { + setWorkingDirectory(dir); + } + if (env != null) { + setEnvironment(env); + } + timeOutInterval = timeout; + } + + + /** Execute the shell command. */ + public void execute() throws IOException { + this.run(); + } + + @Override + public String[] getExecString() { + return command; + } + + @Override + protected void parseExecResult(BufferedReader lines) throws IOException { + output = new StringBuffer(); + char[] buf = new char[512]; + int nRead; + while ( (nRead = lines.read(buf, 0, buf.length)) > 0 ) { + output.append(buf, 0, nRead); + } + } + + /** Get the output of the shell command.*/ + public String getOutput() { + return (output == null) ? "" : output.toString(); + } + + /** + * Returns the commands of this instance. + * Arguments with spaces in are presented with quotes round; other + * arguments are presented raw + * + * @return a string representation of the object. + */ + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + String[] args = getExecString(); + for (String s : args) { + if (s.indexOf(' ') >= 0) { + builder.append('"').append(s).append('"'); + } else { + builder.append(s); + } + builder.append(' '); + } + return builder.toString(); + } + } + + /** + * To check if the passed script to shell command executor timed out or + * not. + * + * @return if the script timed out. + */ + public boolean isTimedOut() { + return timedOut.get(); + } + + /** + * Set if the command has timed out. + * + */ + private void setTimedOut() { + this.timedOut.set(true); + } + + + /** + * Static method to execute a shell command. + * Covers most of the simple cases without requiring the user to implement + * the Shell interface. + * @param cmd shell command to execute. + * @return the output of the executed command. + */ + public static String execCommand(String ... cmd) throws IOException { + return execCommand(null, cmd, 0L); + } + + /** + * Static method to execute a shell command. + * Covers most of the simple cases without requiring the user to implement + * the Shell interface. + * @param env the map of environment key=value + * @param cmd shell command to execute. + * @param timeout time in milliseconds after which script should be marked timeout + * @return the output of the executed command.o + */ + + public static String execCommand(Map env, String[] cmd, + long timeout) throws IOException { + ShellCommandExecutor exec = new ShellCommandExecutor(cmd, null, env, + timeout); + exec.execute(); + return exec.getOutput(); + } + + /** + * Static method to execute a shell command. + * Covers most of the simple cases without requiring the user to implement + * the Shell interface. + * @param env the map of environment key=value + * @param cmd shell command to execute. + * @return the output of the executed command. + */ + public static String execCommand(Map env, String ... cmd) + throws IOException { + return execCommand(env, cmd, 0L); + } + + /** + * Timer which is used to timeout scripts spawned off by shell. + */ + private static class ShellTimeoutTimerTask extends TimerTask { + + private ShellUtils shell; + + public ShellTimeoutTimerTask(ShellUtils shell) { + this.shell = shell; + } + + @Override + public void run() { + Process p = shell.getProcess(); + try { + p.exitValue(); + } catch (Exception e) { + //Process has not terminated. + //So check if it has completed + //if not just destroy it. + if (p != null && !shell.completed.get()) { + shell.setTimedOut(); + p.destroy(); + } + } + } + } + +} From 3c6930dfe4447b6077916b9f9a07b062141b5305 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Mon, 7 Jul 2014 11:36:49 -0700 Subject: [PATCH 17/61] AutoHDFS for getting HDFS delegation token and auto renew. --- storm-core/src/jvm/backtype/storm/Config.java | 8 + .../security/auth/kerberos/AutoHDFS.java | 208 ++++++++++++++++++ 2 files changed, 216 insertions(+) create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 3b3f7e54f15..7805d89abbf 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -1186,6 +1186,14 @@ public class Config extends HashMap { */ public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines"; public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class; + + /** + * The path to use as the zookeeper dir when running a zookeeper server via + * "storm dev-zookeeper". This zookeeper instance is only intended for development; + * it is not a production grade zookeeper setup. + */ + public static final String HDFS_NAMENODE_URL = "dev.zookeeper.path"; + public static final Object HDFS_NAMENODE_URL_SCHEMA = ConfigValidation.StringsValidator; public static void setDebug(Map conf, boolean isOn) { conf.put(Config.TOPOLOGY_DEBUG, isOn); diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java new file mode 100644 index 00000000000..05da90f3add --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java @@ -0,0 +1,208 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import backtype.storm.Config; +import backtype.storm.security.auth.IAutoCredentials; +import backtype.storm.security.auth.ICredentialsRenewer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosTicket; +import javax.xml.bind.DatatypeConverter; +import java.io.*; +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; +import java.net.URI; +import java.util.List; +import java.util.Map; + +/** + * Automatically get HDFS delegation tokens and push it to user's topology. + */ +public class AutoHDFS implements IAutoCredentials, ICredentialsRenewer { + private static final Logger LOG = LoggerFactory.getLogger(AutoHDFS.class); + private static final float TICKET_RENEW_WINDOW = 0.80f; + public static final String HDFS_CREDENTIALS = "HDFS_CREDENTIALS"; + private Map conf; + + public void prepare(Map conf) { + this.conf = conf; + } + + private byte[] getHDFSCredsWithDelegationToken() throws Exception { + try { + /** + * What we want to do is following: + * FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); + * UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(stormUser, topologySubmitter); + * Credentials credential= proxyUser.getCredentials(); + * fs.addDelegationToken("stormUser", credential); + * + * and then return the credential object as a bytearray. + */ + final URI nameNodeURI = new URI((String) conf.get(Config.HDFS_NAMENODE_URL)); + final String topologySubmitterUser = (String) conf.get(Config.TOPOLOGY_SUBMITTER_USER); + final String processOwnerUser = System.getProperty("user.name"); + + /** + * FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); + */ + Class fileSystemClass = Class.forName("org.apache.hadoop.fs.FileSystem"); + Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + Method getMethod = fileSystemClass.getMethod("get", URI.class, configurationClass, String.class); + Object fileSystem = getMethod.invoke(null, nameNodeURI, configurationClass.newInstance(), topologySubmitterUser); + + //UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(stormUser, topologySubmitter); + Class ugiClass = Class.forName("org.apache.hadoop.security.UserGroupInformation"); + Method createProxyUserMethod = ugiClass.getMethod("createProxyUser",String.class, ugiClass); + Object proxyUser = createProxyUserMethod.invoke(processOwnerUser, topologySubmitterUser); + + //Credentials credential= proxyUser.getCredentials(); + Method getCredentialsMethod = ugiClass.getMethod("getCredentials"); + Object credentials = getCredentialsMethod.invoke(proxyUser); + + //fileSystem.addDelegationToken(renewerUser, credentials); + Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); + Method addDelegationTokensMethod = fileSystemClass.getMethod("addDelegationToken", String.class, + credentialClass); + addDelegationTokensMethod.invoke(fileSystem, processOwnerUser, credentials); + + //credentials.write(); + Method writeMethod = credentialClass.getMethod("write", DataOutput.class); + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bao); + writeMethod.invoke(credentials, out); + out.flush(); + out.close(); + + return bao.toByteArray(); + } catch (Exception ex) { + LOG.warn("Failed to get delegation tokens " , ex); + throw ex; + } + } + + @Override + public void populateCredentials(Map credentials) { + try { + credentials.put(HDFS_CREDENTIALS, DatatypeConverter.printBase64Binary(getHDFSCredsWithDelegationToken())); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static Object getHDFSCredential(Map credentials) { + Object credential = null; + if (credentials != null && credentials.containsKey(HDFS_CREDENTIALS)) { + try { + byte[] credBytes = DatatypeConverter.parseBase64Binary(credentials.get(HDFS_CREDENTIALS)); + ByteArrayInputStream bai = new ByteArrayInputStream(credBytes); + ObjectInputStream in = new ObjectInputStream(bai); + + Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); + credential = credentialClass.newInstance(); + Method readMethod = credentialClass.getMethod("readFields", DataInput.class); + readMethod.invoke(credential, in); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + return credential; + } + + @Override + public void updateSubject(Subject subject, Map credentials) { + getUGI(subject, credentials); + } + + @Override + public void populateSubject(Subject subject, Map credentials) { + getUGI(subject, credentials); + } + + private void getUGI(Subject subject, Map credentials) { + try { + Object credential = getHDFSCredential(credentials); + if (credential != null) { + Class ugiClass = Class.forName("org.apache.hadoop.security.UserGroupInformation"); + Constructor constructor = ugiClass.getConstructor(Subject.class); + Object ugi = constructor.newInstance(subject); + + Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); + Method addCredentialsMethod = ugiClass.getMethod("addCredentials", credentialClass); + addCredentialsMethod.invoke(credential); + } else { + LOG.info("No TGT found in credentials"); + } + } catch (Exception e) { + LOG.warn("Failed to initialize and get UserGroupInformation.", e); + } + } + + private long getRefreshTime(KerberosTicket tgt) { + long start = tgt.getStartTime().getTime(); + long end = tgt.getEndTime().getTime(); + return start + (long) ((end - start) * TICKET_RENEW_WINDOW); + } + + @Override + public void renew(Map credentials) { + Object credential = getHDFSCredential(credentials); + /** + * We are trying to do the following : + * List tokens = credential.getAllTokens(); + * for(Token token: tokens) { + * token.renew(configuration); + * } + */ + if (credential != null) { + try { + Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); + Class tokenClass = Class.forName("org.apache.hadoop.security.token.Token"); + Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + Object configuration = configurationClass.newInstance(); + + Method renewMethod = tokenClass.getMethod("renew", configurationClass); + Method getAllTokensMethod = credentialClass.getMethod("getAllTokens"); + + List tokens = (List) getAllTokensMethod.invoke(credential); + + for(Object token : tokens) { + renewMethod.invoke(token, configuration); + } + } catch(Exception e) { + LOG.warn("could not renew the credentials.", e); + } + } + } + + public static void main(String[] args) throws Exception { + AutoHDFS at = new AutoHDFS(); + Map conf = new java.util.HashMap(); + conf.put("java.security.auth.login.config", args[0]); + at.prepare(conf); + Map creds = new java.util.HashMap(); + at.populateCredentials(creds); + Subject s = new Subject(); + at.populateSubject(s, creds); + LOG.info("Got a Subject "+s); + } +} From e04c37356c96d9851c00542c739d053e4bf36481 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Mon, 7 Jul 2014 11:49:42 -0700 Subject: [PATCH 18/61] Revert "AutoHDFS for getting HDFS delegation token and auto renew." This reverts commit 3c6930dfe4447b6077916b9f9a07b062141b5305. --- storm-core/src/jvm/backtype/storm/Config.java | 8 - .../security/auth/kerberos/AutoHDFS.java | 208 ------------------ 2 files changed, 216 deletions(-) delete mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 7805d89abbf..3b3f7e54f15 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -1186,14 +1186,6 @@ public class Config extends HashMap { */ public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines"; public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class; - - /** - * The path to use as the zookeeper dir when running a zookeeper server via - * "storm dev-zookeeper". This zookeeper instance is only intended for development; - * it is not a production grade zookeeper setup. - */ - public static final String HDFS_NAMENODE_URL = "dev.zookeeper.path"; - public static final Object HDFS_NAMENODE_URL_SCHEMA = ConfigValidation.StringsValidator; public static void setDebug(Map conf, boolean isOn) { conf.put(Config.TOPOLOGY_DEBUG, isOn); diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java deleted file mode 100644 index 05da90f3add..00000000000 --- a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java +++ /dev/null @@ -1,208 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package backtype.storm.security.auth.kerberos; - -import backtype.storm.Config; -import backtype.storm.security.auth.IAutoCredentials; -import backtype.storm.security.auth.ICredentialsRenewer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.security.auth.Subject; -import javax.security.auth.kerberos.KerberosTicket; -import javax.xml.bind.DatatypeConverter; -import java.io.*; -import java.lang.reflect.Constructor; -import java.lang.reflect.Method; -import java.net.URI; -import java.util.List; -import java.util.Map; - -/** - * Automatically get HDFS delegation tokens and push it to user's topology. - */ -public class AutoHDFS implements IAutoCredentials, ICredentialsRenewer { - private static final Logger LOG = LoggerFactory.getLogger(AutoHDFS.class); - private static final float TICKET_RENEW_WINDOW = 0.80f; - public static final String HDFS_CREDENTIALS = "HDFS_CREDENTIALS"; - private Map conf; - - public void prepare(Map conf) { - this.conf = conf; - } - - private byte[] getHDFSCredsWithDelegationToken() throws Exception { - try { - /** - * What we want to do is following: - * FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); - * UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(stormUser, topologySubmitter); - * Credentials credential= proxyUser.getCredentials(); - * fs.addDelegationToken("stormUser", credential); - * - * and then return the credential object as a bytearray. - */ - final URI nameNodeURI = new URI((String) conf.get(Config.HDFS_NAMENODE_URL)); - final String topologySubmitterUser = (String) conf.get(Config.TOPOLOGY_SUBMITTER_USER); - final String processOwnerUser = System.getProperty("user.name"); - - /** - * FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); - */ - Class fileSystemClass = Class.forName("org.apache.hadoop.fs.FileSystem"); - Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); - Method getMethod = fileSystemClass.getMethod("get", URI.class, configurationClass, String.class); - Object fileSystem = getMethod.invoke(null, nameNodeURI, configurationClass.newInstance(), topologySubmitterUser); - - //UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(stormUser, topologySubmitter); - Class ugiClass = Class.forName("org.apache.hadoop.security.UserGroupInformation"); - Method createProxyUserMethod = ugiClass.getMethod("createProxyUser",String.class, ugiClass); - Object proxyUser = createProxyUserMethod.invoke(processOwnerUser, topologySubmitterUser); - - //Credentials credential= proxyUser.getCredentials(); - Method getCredentialsMethod = ugiClass.getMethod("getCredentials"); - Object credentials = getCredentialsMethod.invoke(proxyUser); - - //fileSystem.addDelegationToken(renewerUser, credentials); - Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); - Method addDelegationTokensMethod = fileSystemClass.getMethod("addDelegationToken", String.class, - credentialClass); - addDelegationTokensMethod.invoke(fileSystem, processOwnerUser, credentials); - - //credentials.write(); - Method writeMethod = credentialClass.getMethod("write", DataOutput.class); - ByteArrayOutputStream bao = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bao); - writeMethod.invoke(credentials, out); - out.flush(); - out.close(); - - return bao.toByteArray(); - } catch (Exception ex) { - LOG.warn("Failed to get delegation tokens " , ex); - throw ex; - } - } - - @Override - public void populateCredentials(Map credentials) { - try { - credentials.put(HDFS_CREDENTIALS, DatatypeConverter.printBase64Binary(getHDFSCredsWithDelegationToken())); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public static Object getHDFSCredential(Map credentials) { - Object credential = null; - if (credentials != null && credentials.containsKey(HDFS_CREDENTIALS)) { - try { - byte[] credBytes = DatatypeConverter.parseBase64Binary(credentials.get(HDFS_CREDENTIALS)); - ByteArrayInputStream bai = new ByteArrayInputStream(credBytes); - ObjectInputStream in = new ObjectInputStream(bai); - - Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); - credential = credentialClass.newInstance(); - Method readMethod = credentialClass.getMethod("readFields", DataInput.class); - readMethod.invoke(credential, in); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - return credential; - } - - @Override - public void updateSubject(Subject subject, Map credentials) { - getUGI(subject, credentials); - } - - @Override - public void populateSubject(Subject subject, Map credentials) { - getUGI(subject, credentials); - } - - private void getUGI(Subject subject, Map credentials) { - try { - Object credential = getHDFSCredential(credentials); - if (credential != null) { - Class ugiClass = Class.forName("org.apache.hadoop.security.UserGroupInformation"); - Constructor constructor = ugiClass.getConstructor(Subject.class); - Object ugi = constructor.newInstance(subject); - - Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); - Method addCredentialsMethod = ugiClass.getMethod("addCredentials", credentialClass); - addCredentialsMethod.invoke(credential); - } else { - LOG.info("No TGT found in credentials"); - } - } catch (Exception e) { - LOG.warn("Failed to initialize and get UserGroupInformation.", e); - } - } - - private long getRefreshTime(KerberosTicket tgt) { - long start = tgt.getStartTime().getTime(); - long end = tgt.getEndTime().getTime(); - return start + (long) ((end - start) * TICKET_RENEW_WINDOW); - } - - @Override - public void renew(Map credentials) { - Object credential = getHDFSCredential(credentials); - /** - * We are trying to do the following : - * List tokens = credential.getAllTokens(); - * for(Token token: tokens) { - * token.renew(configuration); - * } - */ - if (credential != null) { - try { - Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); - Class tokenClass = Class.forName("org.apache.hadoop.security.token.Token"); - Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); - Object configuration = configurationClass.newInstance(); - - Method renewMethod = tokenClass.getMethod("renew", configurationClass); - Method getAllTokensMethod = credentialClass.getMethod("getAllTokens"); - - List tokens = (List) getAllTokensMethod.invoke(credential); - - for(Object token : tokens) { - renewMethod.invoke(token, configuration); - } - } catch(Exception e) { - LOG.warn("could not renew the credentials.", e); - } - } - } - - public static void main(String[] args) throws Exception { - AutoHDFS at = new AutoHDFS(); - Map conf = new java.util.HashMap(); - conf.put("java.security.auth.login.config", args[0]); - at.prepare(conf); - Map creds = new java.util.HashMap(); - at.populateCredentials(creds); - Subject s = new Subject(); - at.populateSubject(s, creds); - LOG.info("Got a Subject "+s); - } -} From ae5d0c312d170c883e9758767808e1c8d33f58c8 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Tue, 8 Jul 2014 22:08:33 +0000 Subject: [PATCH 19/61] Use TBinaryProtocol without max buffer --- .../jvm/backtype/storm/security/auth/SaslTransportPlugin.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java index 348fae3025b..64bec193b53 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java @@ -67,14 +67,13 @@ public TServer getServer(TProcessor processor) throws IOException, TTransportExc TTransportFactory serverTransportFactory = getServerTransportFactory(); TServerSocket serverTransport = new TServerSocket(port); int numWorkerThreads = type.getNumThreads(storm_conf); - int maxBufferSize = type.getMaxBufferSize(storm_conf); Integer queueSize = type.getQueueSize(storm_conf); TThreadPoolServer.Args server_args = new TThreadPoolServer.Args(serverTransport). processor(new TUGIWrapProcessor(processor)). minWorkerThreads(numWorkerThreads). maxWorkerThreads(numWorkerThreads). - protocolFactory(new TBinaryProtocol.Factory(false, true, maxBufferSize)); + protocolFactory(new TBinaryProtocol.Factory(false, true)); if (serverTransportFactory != null) { server_args.transportFactory(serverTransportFactory); From 642ed743140acf7f9eec56cd98c65ac8435dfe9d Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 9 Jul 2014 17:49:59 +0000 Subject: [PATCH 20/61] Fixed merge error in drpc.clj --- storm-core/src/clj/backtype/storm/daemon/drpc.clj | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj index baba96b1303..1340e49423f 100644 --- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj +++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj @@ -208,10 +208,10 @@ ;; invocations that will unblock those threads handler-server (when (> drpc-port 0) (ThriftServer. conf - (DistributedRPC$Processor. service-handler) + (DistributedRPC$Processor. drpc-service-handler) ThriftConnectionType/DRPC)) invoke-server (ThriftServer. conf - (DistributedRPCInvocations$Processor. service-handler) + (DistributedRPCInvocations$Processor. drpc-service-handler) ThriftConnectionType/DRPC_INVOCATIONS) http-creds-handler (AuthUtils/GetDrpcHttpCredentialsPlugin conf)] (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] From 1094762bf9c3ae339500a3a4500d742367c33e63 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Fri, 11 Jul 2014 11:50:07 -0700 Subject: [PATCH 21/61] STORM-346: added AutoHDFS class that will get hdfs delegation tokens on behalf of users, push it to workers and renew the delegation tokens automatically. --- storm-core/src/jvm/backtype/storm/Config.java | 15 +- .../security/auth/kerberos/AutoHDFS.java | 298 ++++++++++++++++++ 2 files changed, 312 insertions(+), 1 deletion(-) create mode 100644 storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 3b3f7e54f15..785d1e2ecad 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -1186,7 +1186,20 @@ public class Config extends HashMap { */ public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines"; public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class; - + + /** + * HDFS information, used to get the delegation token on behalf of the topology + * submitter user and renew the tokens. see {@link backtype.storm.security.auth.kerberos.AutoHDFS} + */ + public static final String HDFS_NAMENODE_URL = "topology.hdfs.namenodeURI"; + public static final Object HDFS_NAMENODE_URL_SCHEMA = String.class; + + public static final Object HDFS_USER = "topology.hdfs.user"; + public static final Object HDFS_USER_SCHEMA = String.class; + + public static final Object HDFS_USER_KEYTAB = "topology.hdfs.userKeyTab"; + public static final Object HDFS_USER_KEYTAB_SCHEMA = String.class; + public static void setDebug(Map conf, boolean isOn) { conf.put(Config.TOPOLOGY_DEBUG, isOn); } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java new file mode 100644 index 00000000000..80b5916b609 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java @@ -0,0 +1,298 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import backtype.storm.Config; +import backtype.storm.security.auth.IAutoCredentials; +import backtype.storm.security.auth.ICredentialsRenewer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.Subject; +import javax.xml.bind.DatatypeConverter; +import java.io.*; +import java.lang.reflect.Method; +import java.net.URI; +import java.util.Collection; +import java.util.Map; + +/** + * Automatically get HDFS delegation tokens and push it to user's topology. The class + * assumes that HDFS configuration files are in your class path. + */ +public class AutoHDFS implements IAutoCredentials, ICredentialsRenewer { + private static final Logger LOG = LoggerFactory.getLogger(AutoHDFS.class); + public static final String HDFS_CREDENTIALS = "HDFS_CREDENTIALS"; + private static final String CONF_KEYTAB_KEY = "keytab"; + private static final String CONF_USER_KEY = "user"; + + private Map conf; + + public void prepare(Map conf) { + this.conf = conf; + } + + @SuppressWarnings("unchecked") + private Object getConfiguration() { + try { + final String hdfsUser = (String) conf.get(Config.HDFS_USER); + final String hdfsUserKeyTab = (String) conf.get(Config.HDFS_USER_KEYTAB); + + /** + * Configuration configuration = new Configuration(); + * configuration.set(CONF_KEYTAB_KEY, hdfsUserKeyTab); + * configuration.set(CONF_USER_KEY, hdfsUser); + */ + Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + Object configuration = configurationClass.newInstance(); + + Method setMethod = configurationClass.getMethod("set", String.class, String.class); + setMethod.invoke(configuration, CONF_KEYTAB_KEY, hdfsUserKeyTab); + setMethod.invoke(configuration, CONF_USER_KEY, hdfsUser); + /** + * Following are the minimum set of configuration that needs to be set, users should have hdfs-site.xml + * and core-site.xml in the class path which should set these configuration. + * setMethod.invoke(configuration, "hadoop.security.authentication", "KERBEROS"); + * setMethod.invoke(configuration,"dfs.namenode.kerberos.principal", + * "hdfs/zookeeper.witzend.com@WITZEND.COM"); + * setMethod.invoke(configuration, "hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002"); + */ + + setMethod.invoke(configuration, "hadoop.security.authentication", "KERBEROS"); + setMethod.invoke(configuration, "dfs.namenode.kerberos.principal","hdfs/zookeeper.witzend.com@WITZEND.COM"); + setMethod.invoke(configuration, "hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002"); + + //UserGroupInformation.setConfiguration(configuration); + final Class ugiClass = Class.forName("org.apache.hadoop.security.UserGroupInformation"); + Method setConfigurationMethod = ugiClass.getMethod("setConfiguration", configurationClass); + setConfigurationMethod.invoke(null, configuration); + return configuration; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @SuppressWarnings("unchecked") + private void login(Object configuration) { + try { + Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + final Class securityUtilClass = Class.forName("org.apache.hadoop.security.SecurityUtil"); + Method loginMethod = securityUtilClass.getMethod("login", configurationClass, String.class, String.class); + loginMethod.invoke(null, configuration, CONF_KEYTAB_KEY, CONF_USER_KEY); + } catch (Exception e) { + throw new RuntimeException("Failed to login to hdfs .", e); + } + } + + @SuppressWarnings("unchecked") + private byte[] getHDFSCredsWithDelegationToken() throws Exception { + + try { + /** + * What we want to do is following: + * Configuration configuration = new Configuration(); + * configuration.set(CONF_KEYTAB_KEY, hdfsUserKeyTab); + * configuration.set(CONF_USER_KEY, hdfsUser); + * UserGroupInformation.setConfiguration(configuration); + * if(UserGroupInformation.isSecurityEnabled) { + * SecurityUtil.login(configuration, CONF_KEYTAB_KEY, CONF_USER_KEY); + * FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); + * UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + * UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(topologySubmitterUser, ugi); + * Credentials credential= proxyUser.getCredentials(); + * fs.addDelegationToken(hdfsUser, credential); + * } + * and then return the credential object as a bytearray. + */ + Object configuration = getConfiguration(); + final Class ugiClass = Class.forName("org.apache.hadoop.security.UserGroupInformation"); + final Method isSecurityEnabledMethod = ugiClass.getDeclaredMethod("isSecurityEnabled"); + boolean isSecurityEnabled = (Boolean)isSecurityEnabledMethod.invoke(null); + if(isSecurityEnabled) { + login(configuration); + + final URI nameNodeURI = URI.create((String) conf.get(Config.HDFS_NAMENODE_URL)); + final String topologySubmitterUser = (String) conf.get(Config.TOPOLOGY_SUBMITTER_USER); + final String hdfsUser = (String) conf.get(Config.HDFS_USER); + + Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + + //FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); + Class fileSystemClass = Class.forName("org.apache.hadoop.fs.FileSystem"); + Method getMethod = fileSystemClass.getMethod("get", URI.class, configurationClass, String.class); + Object fileSystem = getMethod.invoke(null, nameNodeURI, configuration, topologySubmitterUser); + + //UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + Method getCurrentUserMethod = ugiClass.getMethod("getCurrentUser"); + final Object ugi = getCurrentUserMethod.invoke(null); + + //UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(topologySubmitterUser, ugi); + Method createProxyUserMethod = ugiClass.getMethod("createProxyUser", String.class, ugiClass); + Object proxyUGI = createProxyUserMethod.invoke(null, topologySubmitterUser, ugi); + + //Credentials credential= proxyUser.getCredentials(); + Method getCredentialsMethod = ugiClass.getMethod("getCredentials"); + Object credentials = getCredentialsMethod.invoke(proxyUGI); + + //fs.addDelegationToken(hdfsUser, credential); + Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); + Method addDelegationTokensMethod = fileSystemClass.getMethod("addDelegationTokens", String.class, + credentialClass); + addDelegationTokensMethod.invoke(fileSystem, hdfsUser, credentials); + + + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bao); + Method writeMethod = credentialClass.getMethod("write", DataOutput.class); + writeMethod.invoke(credentials, out); + out.flush(); + out.close(); + + LOG.info(bao.toString()); + return bao.toByteArray(); + } else { + throw new RuntimeException("Security is not enabled for HDFS"); + } + } catch (Exception ex) { + throw new RuntimeException("Failed to get delegation tokens." , ex); + } + } + + @Override + public void populateCredentials(Map credentials) { + try { + credentials.put(HDFS_CREDENTIALS, DatatypeConverter.printBase64Binary( getHDFSCredsWithDelegationToken())); + } catch (Exception e) { + LOG.warn("Could not populate HDFS credentials.", e); + } + } + + /** + * + * @param credentials map with creds. + * @return instance of org.apache.hadoop.security.Credentials, if the Map has HDFS_CREDENTIALS. + * this class's populateCredentials must have been called before. + */ + @SuppressWarnings("unchecked") + private static Object getHDFSCredential(Map credentials) { + Object credential = null; + if (credentials != null && credentials.containsKey(HDFS_CREDENTIALS)) { + try { + byte[] credBytes = DatatypeConverter.parseBase64Binary(credentials.get(HDFS_CREDENTIALS)); + ByteArrayInputStream bai = new ByteArrayInputStream(credBytes); + ObjectInputStream in = new ObjectInputStream(bai); + + Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); + credential = credentialClass.newInstance(); + Method readMethod = credentialClass.getMethod("readFields", DataInput.class); + readMethod.invoke(credential, in); + } catch (Exception e) { + LOG.warn("Could not obtain HDFS credentials from credentials map.", e); + } + } + return credential; + } + + @Override + public void updateSubject(Subject subject, Map credentials) { + addCredentialToSubject(subject, credentials); + } + + @Override + public void populateSubject(Subject subject, Map credentials) { + addCredentialToSubject(subject, credentials); + } + + @SuppressWarnings("unchecked") + private static void addCredentialToSubject(Subject subject, Map credentials) { + try { + Object credential = getHDFSCredential(credentials); + if (credential != null) { + subject.getPrivateCredentials().add(credential); + } else { + LOG.info("No HDFS credential found in credentials"); + } + } catch (Exception e) { + LOG.warn("Failed to initialize and get UserGroupInformation.", e); + } + } + + @Override + @SuppressWarnings("unchecked") + public void renew(Map credentials) { + Object credential = getHDFSCredential(credentials); + /** + * We are trying to do the following : + * List tokens = credential.getAllTokens(); + * for(Token token: tokens) { + * token.renew(configuration); + * } + * TODO: Need to talk to HDFS guys to check what is recommended way to identify a token that is beyond + * renew cycle, once we can identified tokens are not renewable we can repopulate credentials by asking HDFS + * to issue new tokens. Until we have a better way any exception during renewal would result in attempt to + * get new delegation tokens. + */ + if (credential != null) { + try { + Object configuration = getConfiguration(); + + Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); + Class tokenClass = Class.forName("org.apache.hadoop.security.token.Token"); + + Method renewMethod = tokenClass.getMethod("renew", configurationClass); + Method getAllTokensMethod = credentialClass.getMethod("getAllTokens"); + + Collection tokens = (Collection) getAllTokensMethod.invoke(credential); + + for(Object token : tokens) { + renewMethod.invoke(token, configuration); + } + } catch(Exception e) { + LOG.warn("could not renew the credentials, one of the possible reason is tokens are beyond " + + "renewal period so attempting to get new tokens.", e); + populateCredentials(credentials); + } + } + } + + @SuppressWarnings("unchecked") + public static void main(String[] args) throws Exception { + Map conf = new java.util.HashMap(); + conf.put(Config.HDFS_NAMENODE_URL, args[0]); + conf.put(Config.TOPOLOGY_SUBMITTER_USER, args[1]); //with realm e.g. storm@WITZEND.COM + conf.put(Config.HDFS_USER, args[2]); //with realm e.g. hdfs@WITZEND.COM + conf.put(Config.HDFS_USER_KEYTAB, args[3]); + + AutoHDFS autoHDFS = new AutoHDFS(); + autoHDFS.prepare(conf); + + Map creds = new java.util.HashMap(); + autoHDFS.populateCredentials(creds); + LOG.info("Got HDFS credentials", AutoHDFS.getHDFSCredential(creds)); + + Subject s = new Subject(); + autoHDFS.populateSubject(s, creds); + LOG.info("Got a Subject "+ s); + + autoHDFS.renew(creds); + LOG.info("renewed credentials", AutoHDFS.getHDFSCredential(creds)); + } +} + From 3635c88c7377df716b5e3d402ac6412a00dd6dfe Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 23 Jun 2014 16:05:41 -0700 Subject: [PATCH 22/61] STORM-367. Storm UI REST api documentation. --- STORM-UI-REST-API.md | 513 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 513 insertions(+) create mode 100644 STORM-UI-REST-API.md diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md new file mode 100644 index 00000000000..8a83e7f6cb4 --- /dev/null +++ b/STORM-UI-REST-API.md @@ -0,0 +1,513 @@ +# Storm UI REST API +Storm UI server provides a REST Api to access cluster, topology, component overview and metrics. +This api returns json response. + +## Using the UI REST Api + +### /api/v1/cluster/configuration (GET) + returns cluster configuration. + +Sample Response: +```json + { + "dev.zookeeper.path": "/tmp/dev-storm-zookeeper", + "topology.tick.tuple.freq.secs": null, + "topology.builtin.metrics.bucket.size.secs": 60, + "topology.fall.back.on.java.serialization": true, + "topology.max.error.report.per.interval": 5, + "zmq.linger.millis": 5000, + "topology.skip.missing.kryo.registrations": false, + "storm.messaging.netty.client_worker_threads": 1, + "ui.childopts": "-Xmx768m", + "storm.zookeeper.session.timeout": 20000, + "nimbus.reassign": true, + "topology.trident.batch.emit.interval.millis": 500, + "storm.messaging.netty.flush.check.interval.ms": 10, + "nimbus.monitor.freq.secs": 10, + "logviewer.childopts": "-Xmx128m", + "java.library.path": "/usr/local/lib:/opt/local/lib:/usr/lib", + "topology.executor.send.buffer.size": 1024, + } +``` + +### /api/v1/cluster/summary (GET) +returns cluster summary such as nimbus uptime,number of supervisors,slots etc.. + +Response Fields: + +|Field |Description| +|--- |--- | +|stormVersion| Storm version| +|nimbusUptime| Shows how long the cluster is running| +|supervisors| Number of supervisors running| +|slotsTotal| Total number of available worker slots| +|slotsUsed| Number of worker slots used| +|slotsFree| Number of worker slots available| +|executorsTotal| Total number of executors| +|tasksTotal| Total tasks| + +Sample Response: +```json + { + "stormVersion": "0.9.2-incubating-SNAPSHOT", + "nimbusUptime": "3m 53s", + "supervisors": 1, + "slotsTotal": 4, + "slotsUsed": 3, + "slotsFree": 1, + "executorsTotal": 28, + "tasksTotal": 28 + } +``` + +### /api/v1/supervisor/summary (GET) +returns all supervisors summary + +Response Fields: + +|Field |Description| +|--- |--- | +|id| Supervisor's id| +|host| Supervisor's host name| +|uptime| Shows how long the supervisor is runninge| +|slotsTotal| Total number of available worker slots for this supervisor| +|slotsUsed| Number of worker slots used on this supervisor| + +Sample Response: +```json +{ + "supervisors": [ + { + "id": "0b879808-2a26-442b-8f7d-23101e0c3696", + "host": "10.11.1.7", + "uptime": "5m 58s", + "slotsTotal": 4, + "slotsUsed": 3 + } + ] +} +``` + +### /api/v1/topology/summary (GET) +Returns all topologies summary + +Response Fields: + +|Field |Description| +|--- |--- | +|id| Topology Id| +|name| Topology Name| +|uptime| Shows how long the topology is running| +|tasksTotal| Total number of tasks for this topology| +|workersTotal| Number of workers used for this topology| +|executorsTotal| Number of executors used for this topology| + +Sample Response: +```json +{ + "topologies": [ + { + "id": "WordCount3-1-1402960825", + "name": "WordCount3", + "status": "ACTIVE", + "uptime": "6m 5s", + "tasksTotal": 28, + "workersTotal": 3, + "executorsTotal": 28 + } + ] +} +``` + +### /api/v1/topology/:id (GET) + Returns details topology information. Subsititute id with topology id. + +Request Parameters: + +|Parameter |Value |Description | +|----------|--------|-------------| +|id |String (required)| Topology Id | +|window |String. Default value :all-time| Window duration for metrics in ms| +|sys |String. Values 1 or 0. Default value 0| Controls including sys stats part of the response| + + +Response Fields: + +|Field |Description| +|--- |--- | +|id| Topology Id| +|name| Topology Name| +|uptime| Shows how long the topology is running| +|tasksTotal| Total number of tasks for this topology| +|workersTotal| Number of workers used for this topology| +|executorsTotal| Number of executors used for this topology| +|msgTimeout| | +|windowHint| | +|topologyStats.windowPretty| Duration passed in HH:MM:SS format| +|topologyStats.window| User requested time window for metrics| +|topologyStats.emitted| Number of messages emitted in given window| +|topologyStats.trasferred| Number messages transferred in given window| +|topologyStats.acked| Number of messages acked in given window| +|topologyStats.failed| Number of messages failed in given window| +|spouts| Array of all the spout components in the topology| +|spouts.spoutId| Spout id| +|spouts.executors| Number of executors for the spout| +|spouts.completeLatency| Total latency for processing the message| +|spouts.transferred| Total number of messages transferred| +|spouts.tasks| Total number of tasks for the spout| +|spouts.lastError| Shows the last error happened in a spout| +|spouts.acked| Number of messages acked| +|spouts.failed| Number of messages failed| +|bolts| Array of bolt components in the topology| +|bolts.boltId| Bolt id| +|bolts.capacity| This value indicates number of mesages executed * average execute latency / time window| +|bolts.processLatency| Bolt's average time to ack a message after it's received| +|bolts.executeLatency| Average time for bolt's execute method | +|bolts.executors| Number of executor tasks in the bolt component| +|bolts.tasks| Number of instances of bolt| +|bolts.acked| Number of tuples acked by the bolt| +|bolts.failed| Number of tuples failed by the bolt| +|bolts.lastError| Shows the last error occured in the bolt| +|bolts.emitted| Number of tuples emitted| + + + +Examples: +```no-highlight + 1. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825 + 2. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825?sys=1 + 3. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825?window=600 +``` + +Sample Response: +```json + { + "name": "WordCount3", + "id": "WordCount3-1-1402960825", + "workersTotal": 3, + "window": "600", + "status": "ACTIVE", + "tasksTotal": 28, + "executorsTotal": 28, + "uptime": "29m 19s", + "msgTimeout": 30, + "windowHint": "10m 0s", + "topologyStats": [ + { + "windowPretty": "10m 0s", + "window": "600", + "emitted": 397960, + "transferred": 213380, + "completeLatency": "0.000", + "acked": 213460, + "failed": 0 + }, + { + "windowPretty": "3h 0m 0s", + "window": "10800", + "emitted": 1190260, + "transferred": 638260, + "completeLatency": "0.000", + "acked": 638280, + "failed": 0 + }, + { + "windowPretty": "1d 0h 0m 0s", + "window": "86400", + "emitted": 1190260, + "transferred": 638260, + "completeLatency": "0.000", + "acked": 638280, + "failed": 0 + }, + { + "windowPretty": "All time", + "window": ":all-time", + "emitted": 1190260, + "transferred": 638260, + "completeLatency": "0.000", + "acked": 638280, + "failed": 0 + } + ], + "spouts": [ + { + "executors": 5, + "emitted": 28880, + "completeLatency": "0.000", + "transferred": 28880, + "acked": 0, + "spoutId": "spout", + "tasks": 5, + "lastError": "", + "failed": 0 + } + ], + "bolts": [ + { + "executors": 12, + "emitted": 184580, + "transferred": 0, + "acked": 184640, + "executeLatency": "0.048", + "tasks": 12, + "executed": 184620, + "processLatency": "0.043", + "boltId": "count", + "lastError": "", + "capacity": "0.003", + "failed": 0 + }, + { + "executors": 8, + "emitted": 184500, + "transferred": 184500, + "acked": 28820, + "executeLatency": "0.024", + "tasks": 8, + "executed": 28780, + "processLatency": "2.112", + "boltId": "split", + "lastError": "", + "capacity": "0.000", + "failed": 0 + } + ], + "configuration": { + "storm.id": "WordCount3-1-1402960825", + "dev.zookeeper.path": "/tmp/dev-storm-zookeeper", + "topology.tick.tuple.freq.secs": null, + "topology.builtin.metrics.bucket.size.secs": 60, + "topology.fall.back.on.java.serialization": true, + "topology.max.error.report.per.interval": 5, + "zmq.linger.millis": 5000, + "topology.skip.missing.kryo.registrations": false, + "storm.messaging.netty.client_worker_threads": 1, + "ui.childopts": "-Xmx768m", + "storm.zookeeper.session.timeout": 20000, + "nimbus.reassign": true, + "topology.trident.batch.emit.interval.millis": 500, + "storm.messaging.netty.flush.check.interval.ms": 10, + "nimbus.monitor.freq.secs": 10, + "logviewer.childopts": "-Xmx128m", + "java.library.path": "/usr/local/lib:/opt/local/lib:/usr/lib", + "topology.executor.send.buffer.size": 1024, + "storm.local.dir": "storm-local", + "storm.messaging.netty.buffer_size": 5242880, + "supervisor.worker.start.timeout.secs": 120, + "topology.enable.message.timeouts": true, + "nimbus.cleanup.inbox.freq.secs": 600, + "nimbus.inbox.jar.expiration.secs": 3600, + "drpc.worker.threads": 64, + "topology.worker.shared.thread.pool.size": 4, + "nimbus.host": "hw10843.local", + "storm.messaging.netty.min_wait_ms": 100, + "storm.zookeeper.port": 2181, + "transactional.zookeeper.port": null, + "topology.executor.receive.buffer.size": 1024, + "transactional.zookeeper.servers": null, + "storm.zookeeper.root": "/storm", + "storm.zookeeper.retry.intervalceiling.millis": 30000, + "supervisor.enable": true, + "storm.messaging.netty.server_worker_threads": 1 + }, +} +``` + + +### /api/v1/topology/:id/component/:component (GET) + +Returns detailed metrics and executor information + +|Parameter |Value |Description | +|----------|--------|-------------| +|id |String (required)| Topology Id | +|component |String (required)| Component Id | +|window |String. Default value :all-time| window duration for metrics in ms| +|sys |String. Values 1 or 0. Default value 0| controls including sys stats part of the response| + + +Examples: +```no-highlight +1. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout +2. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout?sys=1 +3. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout?window=600 +``` + +Sample Response: +```json +{ + "name": "WordCount3", + "id": "spout", + "componentType": "spout", + "windowHint": "10m 0s", + "executors": 5, + "componentErrors": [], + "topologyId": "WordCount3-1-1402960825", + "tasks": 5, + "window": "600", + "spoutSummary": [ + { + "windowPretty": "10m 0s", + "window": "600", + "emitted": 28500, + "transferred": 28460, + "completeLatency": "0.000", + "acked": 0, + "failed": 0 + }, + { + "windowPretty": "3h 0m 0s", + "window": "10800", + "emitted": 127640, + "transferred": 127440, + "completeLatency": "0.000", + "acked": 0, + "failed": 0 + }, + { + "windowPretty": "1d 0h 0m 0s", + "window": "86400", + "emitted": 127640, + "transferred": 127440, + "completeLatency": "0.000", + "acked": 0, + "failed": 0 + }, + { + "windowPretty": "All time", + "window": ":all-time", + "emitted": 127640, + "transferred": 127440, + "completeLatency": "0.000", + "acked": 0, + "failed": 0 + } + ], + "outputStats": [ + { + "stream": "__metrics", + "emitted": 40, + "transferred": 0, + "completeLatency": "0", + "acked": 0, + "failed": 0 + }, + { + "stream": "default", + "emitted": 28460, + "transferred": 28460, + "completeLatency": "0", + "acked": 0, + "failed": 0 + } + ] + "executorStats": [ + { + "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6701.log", + "emitted": 5720, + "port": 6701, + "completeLatency": "0.000", + "transferred": 5720, + "host": "10.11.1.7", + "acked": 0, + "uptime": "43m 4s", + "id": "[24-24]", + "failed": 0 + }, + { + "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6703.log", + "emitted": 5700, + "port": 6703, + "completeLatency": "0.000", + "transferred": 5700, + "host": "10.11.1.7", + "acked": 0, + "uptime": "42m 57s", + "id": "[25-25]", + "failed": 0 + }, + { + "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6702.log", + "emitted": 5700, + "port": 6702, + "completeLatency": "0.000", + "transferred": 5680, + "host": "10.11.1.7", + "acked": 0, + "uptime": "42m 57s", + "id": "[26-26]", + "failed": 0 + }, + { + "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6701.log", + "emitted": 5700, + "port": 6701, + "completeLatency": "0.000", + "transferred": 5680, + "host": "10.11.1.7", + "acked": 0, + "uptime": "43m 4s", + "id": "[27-27]", + "failed": 0 + }, + { + "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6703.log", + "emitted": 5680, + "port": 6703, + "completeLatency": "0.000", + "transferred": 5680, + "host": "10.11.1.7", + "acked": 0, + "uptime": "42m 57s", + "id": "[28-28]", + "failed": 0 + } + ] +} +``` + +### /api/v1/topology/:id/activate (POST) +activates a topology + +|Parameter |Value |Description | +|----------|--------|-------------| +|id |String (required)| Topology Id | + +### /api/v1/topology/:id/deactivate (POST) +deactivates a topology + +|Parameter |Value |Description | +|----------|--------|-------------| +|id |String (required)| Topology Id | + + +### /api/v1/topology/:id/rebalance/:wait-time (POST) +rebalances a topology + +|Parameter |Value |Description | +|----------|--------|-------------| +|id |String (required)| Topology Id | +|wait-time |String (required)| Wait time before rebalance happens | + +### /api/v1/topology/:id/kill/:wait-time (POST) +kills a topology + +|Parameter |Value |Description | +|----------|--------|-------------| +|id |String (required)| Topology Id | +|wait-time |String (required)| Wait time before rebalance happens | + + + +### ERRORS +on errors in any of the above api returns 500 http status code with +the following response. + +Sample Response: +```json +{ + "error": "Internal Server Error", + "errorMessage": "java.lang.NullPointerException\n\tat clojure.core$name.invoke(core.clj:1505)\n\tat backtype.storm.ui.core$component_page.invoke(core.clj:752)\n\tat backtype.storm.ui.core$fn__7766.invoke(core.clj:782)\n\tat compojure.core$make_route$fn__5755.invoke(core.clj:93)\n\tat compojure.core$if_route$fn__5743.invoke(core.clj:39)\n\tat compojure.core$if_method$fn__5736.invoke(core.clj:24)\n\tat compojure.core$routing$fn__5761.invoke(core.clj:106)\n\tat clojure.core$some.invoke(core.clj:2443)\n\tat compojure.core$routing.doInvoke(core.clj:106)\n\tat clojure.lang.RestFn.applyTo(RestFn.java:139)\n\tat clojure.core$apply.invoke(core.clj:619)\n\tat compojure.core$routes$fn__5765.invoke(core.clj:111)\n\tat ring.middleware.reload$wrap_reload$fn__6880.invoke(reload.clj:14)\n\tat backtype.storm.ui.core$catch_errors$fn__7800.invoke(core.clj:836)\n\tat ring.middleware.keyword_params$wrap_keyword_params$fn__6319.invoke(keyword_params.clj:27)\n\tat ring.middleware.nested_params$wrap_nested_params$fn__6358.invoke(nested_params.clj:65)\n\tat ring.middleware.params$wrap_params$fn__6291.invoke(params.clj:55)\n\tat ring.middleware.multipart_params$wrap_multipart_params$fn__6386.invoke(multipart_params.clj:103)\n\tat ring.middleware.flash$wrap_flash$fn__6675.invoke(flash.clj:14)\n\tat ring.middleware.session$wrap_session$fn__6664.invoke(session.clj:43)\n\tat ring.middleware.cookies$wrap_cookies$fn__6595.invoke(cookies.clj:160)\n\tat ring.adapter.jetty$proxy_handler$fn__6112.invoke(jetty.clj:16)\n\tat ring.adapter.jetty.proxy$org.mortbay.jetty.handler.AbstractHandler$0.handle(Unknown Source)\n\tat org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapper.java:152)\n\tat org.mortbay.jetty.Server.handle(Server.java:326)\n\tat org.mortbay.jetty.HttpConnection.handleRequest(HttpConnection.java:542)\n\tat org.mortbay.jetty.HttpConnection$RequestHandler.headerComplete(HttpConnection.java:928)\n\tat org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)\n\tat org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)\n\tat org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)\n\tat org.mortbay.jetty.bio.SocketConnector$Connection.run(SocketConnector.java:228)\n\tat org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582)\n" +} +``` From 37bfb6f5d8df29aca41715baff295011277f8ac2 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 26 Jun 2014 17:33:38 -0700 Subject: [PATCH 23/61] STORM-367. Storm UI REST api documentation. Added changes as per Bobby's recommendations. --- STORM-UI-REST-API.md | 163 ++++++++++++++++++++++++++----------------- 1 file changed, 98 insertions(+), 65 deletions(-) diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md index 8a83e7f6cb4..5222ca76ccd 100644 --- a/STORM-UI-REST-API.md +++ b/STORM-UI-REST-API.md @@ -1,11 +1,12 @@ # Storm UI REST API -Storm UI server provides a REST Api to access cluster, topology, component overview and metrics. -This api returns json response. +Storm UI server provides a REST Api to access cluster, topology, component overview and metrics. +This api returns json response. +Please ignore undocumented elements in the json repsonse. ## Using the UI REST Api ### /api/v1/cluster/configuration (GET) - returns cluster configuration. + returns cluster configuration. Below is a sample response but doesn't include all the config fileds. Sample Response: ```json @@ -35,16 +36,16 @@ returns cluster summary such as nimbus uptime,number of supervisors,slots etc.. Response Fields: -|Field |Description| -|--- |--- | -|stormVersion| Storm version| -|nimbusUptime| Shows how long the cluster is running| -|supervisors| Number of supervisors running| -|slotsTotal| Total number of available worker slots| -|slotsUsed| Number of worker slots used| -|slotsFree| Number of worker slots available| -|executorsTotal| Total number of executors| -|tasksTotal| Total tasks| +|Field |Value|Description +|--- |--- |--- +|stormVersion|String| Storm version| +|nimbusUptime|String| Shows how long the cluster is running| +|supervisors|Integer| Number of supervisors running| +|slotsTotal| Integer|Total number of available worker slots| +|slotsUsed| Integer| Number of worker slots used| +|slotsFree| Integer |Number of worker slots available| +|executorsTotal| Integer |Total number of executors| +|tasksTotal| Integer |Total tasks| Sample Response: ```json @@ -65,13 +66,13 @@ returns all supervisors summary Response Fields: -|Field |Description| -|--- |--- | -|id| Supervisor's id| -|host| Supervisor's host name| -|uptime| Shows how long the supervisor is runninge| -|slotsTotal| Total number of available worker slots for this supervisor| -|slotsUsed| Number of worker slots used on this supervisor| +|Field |Value|Description| +|--- |--- |--- +|id| String | Supervisor's id| +|host| String| Supervisor's host name| +|uptime| String| Shows how long the supervisor is running| +|slotsTotal| Integer| Total number of available worker slots for this supervisor| +|slotsUsed| Integer| Number of worker slots used on this supervisor| Sample Response: ```json @@ -93,14 +94,15 @@ Returns all topologies summary Response Fields: -|Field |Description| -|--- |--- | -|id| Topology Id| -|name| Topology Name| -|uptime| Shows how long the topology is running| -|tasksTotal| Total number of tasks for this topology| -|workersTotal| Number of workers used for this topology| -|executorsTotal| Number of executors used for this topology| +|Field |Value | Description| +|--- |--- |--- +|id| String| Topology Id| +|name| String| Topology Name| +|status| String| Topology Status| +|uptime| String| Shows how long the topology is running| +|tasksTotal| Integer |Total number of tasks for this topology| +|workersTotal| Integer |Number of workers used for this topology| +|executorsTotal| Integer |Number of executors used for this topology| Sample Response: ```json @@ -120,7 +122,7 @@ Sample Response: ``` ### /api/v1/topology/:id (GET) - Returns details topology information. Subsititute id with topology id. + Returns topology information and stats. Subsititute id with topology id. Request Parameters: @@ -133,42 +135,46 @@ Request Parameters: Response Fields: -|Field |Description| -|--- |--- | -|id| Topology Id| -|name| Topology Name| -|uptime| Shows how long the topology is running| -|tasksTotal| Total number of tasks for this topology| -|workersTotal| Number of workers used for this topology| -|executorsTotal| Number of executors used for this topology| -|msgTimeout| | -|windowHint| | -|topologyStats.windowPretty| Duration passed in HH:MM:SS format| -|topologyStats.window| User requested time window for metrics| -|topologyStats.emitted| Number of messages emitted in given window| -|topologyStats.trasferred| Number messages transferred in given window| -|topologyStats.acked| Number of messages acked in given window| -|topologyStats.failed| Number of messages failed in given window| -|spouts| Array of all the spout components in the topology| -|spouts.spoutId| Spout id| -|spouts.executors| Number of executors for the spout| -|spouts.completeLatency| Total latency for processing the message| -|spouts.transferred| Total number of messages transferred| -|spouts.tasks| Total number of tasks for the spout| -|spouts.lastError| Shows the last error happened in a spout| -|spouts.acked| Number of messages acked| -|spouts.failed| Number of messages failed| -|bolts| Array of bolt components in the topology| -|bolts.boltId| Bolt id| -|bolts.capacity| This value indicates number of mesages executed * average execute latency / time window| -|bolts.processLatency| Bolt's average time to ack a message after it's received| -|bolts.executeLatency| Average time for bolt's execute method | -|bolts.executors| Number of executor tasks in the bolt component| -|bolts.tasks| Number of instances of bolt| -|bolts.acked| Number of tuples acked by the bolt| -|bolts.failed| Number of tuples failed by the bolt| -|bolts.lastError| Shows the last error occured in the bolt| -|bolts.emitted| Number of tuples emitted| +|Field |Value |Description| +|--- |--- |--- +|id| String| Topology Id| +|name| String |Topology Name| +|uptime| String |Shows how long the topology is running| +|status| String |Shows Topology's current status| +|tasksTotal| Integer |Total number of tasks for this topology| +|workersTotal| Integer |Number of workers used for this topology| +|executorsTotal| Integer |Number of executors used for this topology| +|msgTimeout| Integer | Number of seconds a tuple has before the spout considers it failed | +|windowHint| String | window param value in "hh mm ss" format. Default value is "All Time"| +|topologyStats| Array | Array of all the topology related stats per time window| +|topologyStats.windowPretty| String |Duration passed in HH:MM:SS format| +|topologyStats.window| String |User requested time window for metrics| +|topologyStats.emitted| Long |Number of messages emitted in given window| +|topologyStats.trasferred| Long |Number messages transferred in given window| +|topologyStats.completeLatency| String (double value returned in String format) |Total latency for processing the message| +|topologyStats.acked| Long |Number of messages acked in given window| +|topologyStats.failed| Long |Number of messages failed in given window| +|spouts| Array | Array of all the spout components in the topology| +|spouts.spoutId| String |Spout id| +|spouts.executors| Integer |Number of executors for the spout| +|spouts.emitted| Long |Number of messages emitted in given window | +|spouts.completeLatency| String (double value returned in String format) |Total latency for processing the message| +|spouts.transferred| Long |Total number of messages transferred in given window| +|spouts.tasks| Integer |Total number of tasks for the spout| +|spouts.lastError| String |Shows the last error happened in a spout| +|spouts.acked| Long |Number of messages acked| +|spouts.failed| Long |Number of messages failed| +|bolts| Array | Array of bolt components in the topology| +|bolts.boltId| String |Bolt id| +|bolts.capacity| String (double value returned in String format) |This value indicates number of mesages executed * average execute latency / time window| +|bolts.processLatency| String (double value returned in String format) |Bolt's average time to ack a message after it's received| +|bolts.executeLatency| String (double value returned in String format) |Average time for bolt's execute method | +|bolts.executors| Integer |Number of executor tasks in the bolt component| +|bolts.tasks| Integer |Number of instances of bolt| +|bolts.acked| Long |Number of tuples acked by the bolt| +|bolts.failed| Long |Number of tuples failed by the bolt| +|bolts.lastError| String |Shows the last error occured in the bolt| +|bolts.emitted| Long |Number of tuples emitted| @@ -326,6 +332,33 @@ Returns detailed metrics and executor information |window |String. Default value :all-time| window duration for metrics in ms| |sys |String. Values 1 or 0. Default value 0| controls including sys stats part of the response| +Response Fields: + +|Field |Value |Description| +|--- |--- |--- +|id | String | Component's id| +|name | String | Topology name| +|componentType | String | component's type SPOUT or BOLT| +|windowHint| String | window param value in "hh mm ss" format. Default value is "All Time"| +|executors| Integer |Number of executor tasks in the component| +|componentErrors| Array of Strings | List of component errors| +|topologyId| String | Topology's Id| +|tasks| Integer |Number of instances of component| +|window |String. Default value "All Time" | window duration for metrics in seconds| +|spoutSummary or boltStats| Array |Array of component stats. **Please note this element tag can be spoutSummary or boltStats depending on the componentType**| +|spoutSummary.windowPretty| String |Duration passed in HH:MM:SS format| +|spoutSummary.window| String | window duration for metrics in seconds| +|spoutSummary.emitted| Long |Number of messages emitted in given window | +|spoutSummary.completeLatency| String (double value returned in String format) |Total latency for processing the message| +|spoutSummary.transferred| Long |Total number of messages transferred in given window| +|spoutSummary.acked| Long |Number of messages acked| +|spoutSummary.failed| Long |Number of messages failed| +|boltStats.windowPretty| String |Duration passed in HH:MM:SS format| +|boltStats..window| String | window duration for metrics in seconds| +|boltStats.transferred| Long |Total number of messages transferred in given window| +|boltStats.processLatency| String (double value returned in String format) |Bolt's average time to ack a message after it's received| +|boltStats.acked| Long |Number of messages acked| +|boltStats.failed| Long |Number of messages failed| Examples: ```no-highlight From 1ef85903fefbd9e5db497a078c247b2aa2e9a5d2 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 18 Jul 2014 20:43:52 +0000 Subject: [PATCH 24/61] Update Storm UI REST API for error lapsed time --- STORM-UI-REST-API.md | 23 +++++++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md index 5222ca76ccd..0c5cb1bd529 100644 --- a/STORM-UI-REST-API.md +++ b/STORM-UI-REST-API.md @@ -162,6 +162,8 @@ Response Fields: |spouts.transferred| Long |Total number of messages transferred in given window| |spouts.tasks| Integer |Total number of tasks for the spout| |spouts.lastError| String |Shows the last error happened in a spout| +|spouts.errorLapsedSecs| Integer | Number of seconds lapsed on nimbus since that last error happened in a spout| +|spouts.errorWorkerLogLink| String | Link to the worker log that reported the exception | |spouts.acked| Long |Number of messages acked| |spouts.failed| Long |Number of messages failed| |bolts| Array | Array of bolt components in the topology| @@ -174,6 +176,8 @@ Response Fields: |bolts.acked| Long |Number of tuples acked by the bolt| |bolts.failed| Long |Number of tuples failed by the bolt| |bolts.lastError| String |Shows the last error occured in the bolt| +|bolts.errorLapsedSecs| Integer |Number of seconds lapsed on nimbus since that last error happened in a bolt| +|bolts.errorWorkerLogLink| String | Link to the worker log that reported the exception | |bolts.emitted| Long |Number of tuples emitted| @@ -246,6 +250,7 @@ Sample Response: "spoutId": "spout", "tasks": 5, "lastError": "", + "errorLapsedSecs": "", "failed": 0 } ], @@ -261,6 +266,7 @@ Sample Response: "processLatency": "0.043", "boltId": "count", "lastError": "", + "errorLapsedSecs": "", "capacity": "0.003", "failed": 0 }, @@ -275,6 +281,7 @@ Sample Response: "processLatency": "2.112", "boltId": "split", "lastError": "", + "errorLapsedSecs": "", "capacity": "0.000", "failed": 0 } @@ -341,7 +348,13 @@ Response Fields: |componentType | String | component's type SPOUT or BOLT| |windowHint| String | window param value in "hh mm ss" format. Default value is "All Time"| |executors| Integer |Number of executor tasks in the component| -|componentErrors| Array of Strings | List of component errors| +|componentErrors| Array of Errors | List of component errors| +|componentErrors.time| String | Date/time timezone for the error| +|componentErrors.errorHost| String | host name for the error| +|componentErrors.errorPort| String | port for the error| +|componentErrors.error| String |Shows the error happened in a comopnent| +|componentErrors.errorLapsedSecs| Integer | Number of seconds lapsed on nimbus since the error happened in a component | +|componentErrors.errorWorkerLogLink| String | Link to the worker log that reported the exception | |topologyId| String | Topology's Id| |tasks| Integer |Number of instances of component| |window |String. Default value "All Time" | window duration for metrics in seconds| @@ -375,7 +388,13 @@ Sample Response: "componentType": "spout", "windowHint": "10m 0s", "executors": 5, - "componentErrors": [], + "componentErrors":[{"time": "Fri, 18 Jul 2014 19:00:16 +0000", + "errorHost": "10.11.1.70", + "errorPort": 6701, + "errorWorkerLogLink": "http://10.11.1.7:8000/log?file=worker-6701.log", + "errorLapsedSecs": 16, + "error": "java.lang.RuntimeException: java.lang.StringIndexOutOfBoundsException: Some Error\n\tat backtype.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:128)\n\tat backtype.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:99)\n\tat backtype.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:80)\n\tat backtype...more.." + }], "topologyId": "WordCount3-1-1402960825", "tasks": 5, "window": "600", From 9a41b50a6e3134a04c3ac1a267b4909facbaf924 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 18 Jul 2014 21:06:39 +0000 Subject: [PATCH 25/61] Fix STORM REST API --- STORM-UI-REST-API.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md index 0c5cb1bd529..22569cf225a 100644 --- a/STORM-UI-REST-API.md +++ b/STORM-UI-REST-API.md @@ -162,22 +162,22 @@ Response Fields: |spouts.transferred| Long |Total number of messages transferred in given window| |spouts.tasks| Integer |Total number of tasks for the spout| |spouts.lastError| String |Shows the last error happened in a spout| -|spouts.errorLapsedSecs| Integer | Number of seconds lapsed on nimbus since that last error happened in a spout| -|spouts.errorWorkerLogLink| String | Link to the worker log that reported the exception | +|spouts.errorLapsedSecs| Integer | Number of seconds elapsed since that last error happened in a spout| +|spouts.errorWorkerLogLink| String | Link to the worker log that reported the exception | |spouts.acked| Long |Number of messages acked| |spouts.failed| Long |Number of messages failed| |bolts| Array | Array of bolt components in the topology| |bolts.boltId| String |Bolt id| -|bolts.capacity| String (double value returned in String format) |This value indicates number of mesages executed * average execute latency / time window| +|bolts.capacity| String (double value returned in String format) |This value indicates number of messages executed * average execute latency / time window| |bolts.processLatency| String (double value returned in String format) |Bolt's average time to ack a message after it's received| |bolts.executeLatency| String (double value returned in String format) |Average time for bolt's execute method | |bolts.executors| Integer |Number of executor tasks in the bolt component| |bolts.tasks| Integer |Number of instances of bolt| |bolts.acked| Long |Number of tuples acked by the bolt| |bolts.failed| Long |Number of tuples failed by the bolt| -|bolts.lastError| String |Shows the last error occured in the bolt| -|bolts.errorLapsedSecs| Integer |Number of seconds lapsed on nimbus since that last error happened in a bolt| -|bolts.errorWorkerLogLink| String | Link to the worker log that reported the exception | +|bolts.lastError| String |Shows the last error occurred in the bolt| +|bolts.errorLapsedSecs| Integer |Number of seconds elapsed since that last error happened in a bolt| +|bolts.errorWorkerLogLink| String | Link to the worker log that reported the exception | |bolts.emitted| Long |Number of tuples emitted| @@ -352,9 +352,9 @@ Response Fields: |componentErrors.time| String | Date/time timezone for the error| |componentErrors.errorHost| String | host name for the error| |componentErrors.errorPort| String | port for the error| -|componentErrors.error| String |Shows the error happened in a comopnent| -|componentErrors.errorLapsedSecs| Integer | Number of seconds lapsed on nimbus since the error happened in a component | -|componentErrors.errorWorkerLogLink| String | Link to the worker log that reported the exception | +|componentErrors.error| String |Shows the error happened in a component| +|componentErrors.errorLapsedSecs| Integer | Number of seconds elapsed since the error happened in a component | +|componentErrors.errorWorkerLogLink| String | Link to the worker log that reported the exception | |topologyId| String | Topology's Id| |tasks| Integer |Number of instances of component| |window |String. Default value "All Time" | window duration for metrics in seconds| From 6d0311f82fa2acf8fbc292c381c7e3d507bd651c Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Mon, 21 Jul 2014 18:49:34 +0000 Subject: [PATCH 26/61] Make sure errorLapsedSecs is always integer --- STORM-UI-REST-API.md | 6 +++--- storm-core/src/clj/backtype/storm/ui/core.clj | 5 ++--- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md index 22569cf225a..72b5ea03ca2 100644 --- a/STORM-UI-REST-API.md +++ b/STORM-UI-REST-API.md @@ -250,7 +250,7 @@ Sample Response: "spoutId": "spout", "tasks": 5, "lastError": "", - "errorLapsedSecs": "", + "errorLapsedSecs": null "failed": 0 } ], @@ -266,7 +266,7 @@ Sample Response: "processLatency": "0.043", "boltId": "count", "lastError": "", - "errorLapsedSecs": "", + "errorLapsedSecs": null "capacity": "0.003", "failed": 0 }, @@ -281,7 +281,7 @@ Sample Response: "processLatency": "2.112", "boltId": "split", "lastError": "", - "errorLapsedSecs": "", + "errorLapsedSecs": null "capacity": "0.000", "failed": 0 } diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj index a7492f90624..a8773deeecc 100644 --- a/storm-core/src/clj/backtype/storm/ui/core.clj +++ b/storm-core/src/clj/backtype/storm/ui/core.clj @@ -321,9 +321,8 @@ (defn get-error-time [error] - (if error - (time-delta (.get_error_time_secs ^ErrorInfo error)) - "")) + (if error + (time-delta (.get_error_time_secs ^ErrorInfo error)))) (defn get-error-data [error] From 9e13a356268e1e8bf76ac29c42a441a0f49108be Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 21 Jul 2014 15:13:35 -0700 Subject: [PATCH 27/61] STORM-347. (Security) authentication should allow for groups not just users. Added Bobby's suggested changes. --- conf/defaults.yaml | 3 + storm-core/src/jvm/backtype/storm/Config.java | 170 +++++++++--------- .../auth/IGroupMappingServiceProvider.java | 12 -- .../auth/ShellBasedUnixGroupsMapping.java | 26 +-- 4 files changed, 107 insertions(+), 104 deletions(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 83b7b4d2358..b0ebb4dfe67 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -154,6 +154,9 @@ storm.messaging.netty.transfer.batch.size: 262144 # We check with this interval that whether the Netty channel is writable and try to write pending messages if it is. storm.messaging.netty.flush.check.interval.ms: 10 +# default number of seconds group mapping service will cache user group +storm.group.mapping.service.cache.duration.secs: 120 + ### topology.* configs are for specific executing storms topology.enable.message.timeouts: true topology.debug: false diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index fee5f6eabb0..ea54313e350 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -28,11 +28,11 @@ import java.util.Map; /** - * Topology configs are specified as a plain old map. This class provides a - * convenient way to create a topology config map by providing setter methods for - * all the configs that can be set. It also makes it easier to do things like add + * Topology configs are specified as a plain old map. This class provides a + * convenient way to create a topology config map by providing setter methods for + * all the configs that can be set. It also makes it easier to do things like add * serializations. - * + * *

This class also provides constants for all the configurations possible on * a Storm cluster and Storm topology. Each constant is paired with a schema * that defines the validity criterion of the corresponding field. Default @@ -40,7 +40,7 @@ * *

Note that you may put other configurations in any of the configs. Storm * will ignore anything it doesn't recognize, but your topologies are free to make - * use of them by reading them in the prepare method of Bolts or the open method of + * use of them by reading them in the prepare method of Bolts or the open method of * Spouts.

*/ public class Config extends HashMap { @@ -60,39 +60,39 @@ public class Config extends HashMap { /** * Netty based messaging: The buffer size for send/recv buffer */ - public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; + public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = Number.class; /** * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible */ - public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; + public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = Number.class; /** - * Netty based messaging: The min # of milliseconds that a peer will wait. + * Netty based messaging: The min # of milliseconds that a peer will wait. */ - public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; + public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = Number.class; /** - * Netty based messaging: The max # of milliseconds that a peer will wait. + * Netty based messaging: The max # of milliseconds that a peer will wait. */ - public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; + public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = Number.class; /** * Netty based messaging: The # of worker threads for the server. */ - public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads"; + public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads"; public static final Object STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS_SCHEMA = Number.class; /** * Netty based messaging: The # of worker threads for the client. */ - public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads"; + public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads"; public static final Object STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS_SCHEMA = Number.class; - + /** * If the Netty messaging layer is busy, the Netty client will try to batch message as more as possible up to the size of STORM_NETTY_MESSAGE_BATCH_SIZE bytes */ @@ -104,8 +104,8 @@ public class Config extends HashMap { */ public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms"; public static final Object STORM_NETTY_FLUSH_CHECK_INTERVAL_MS_SCHEMA = Number.class; - - + + /** * A list of hosts of ZooKeeper servers used to manage the cluster. */ @@ -128,7 +128,7 @@ public class Config extends HashMap { /** * A global task scheduler used to assign topologies's tasks to supervisors' wokers. - * + * * If this is not set, a default system scheduler will be used. */ public static final String STORM_SCHEDULER = "storm.scheduler"; @@ -141,9 +141,9 @@ public class Config extends HashMap { public static final Object STORM_CLUSTER_MODE_SCHEMA = String.class; /** - * The hostname the supervisors/workers should report to nimbus. If unset, Storm will + * The hostname the supervisors/workers should report to nimbus. If unset, Storm will * get the hostname to report by calling InetAddress.getLocalHost().getCanonicalHostName(). - * + * * You should set this config when you dont have a DNS which supervisors/workers * can utilize to find each other based on hostname got from calls to * InetAddress.getLocalHost().getCanonicalHostName(). @@ -163,6 +163,12 @@ public class Config extends HashMap { public static final String STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN = "storm.group.mapping.service"; public static final Object STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN_SCHEMA = String.class; + /** + * Max no.of seconds group mapping service will cache user groups + */ + public static final String STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS = "storm.group.mapping.service.cache.duration.secs"; + public static final Object STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS_SCHEMA = Number.class; + /** * The default transport plug-in for Thrift client/server communication */ @@ -170,7 +176,7 @@ public class Config extends HashMap { public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; /** - * The serializer class for ListDelegate (tuple payload). + * The serializer class for ListDelegate (tuple payload). * The default serializer will be ListDelegateSerializer */ public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer"; @@ -184,9 +190,9 @@ public class Config extends HashMap { 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 - * of this flag is to make it easy to run Storm in local mode by eliminating + * 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 + * of this flag is to make it easy to run Storm in local mode by eliminating * the need for native dependencies, which can be difficult to install. * * Defaults to false. @@ -271,7 +277,7 @@ public class Config extends HashMap { */ 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. */ @@ -311,15 +317,15 @@ public class Config extends HashMap { /** * 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 + * 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 + * 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 + * 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; @@ -392,7 +398,7 @@ public class Config extends HashMap { public static final Object NIMBUS_TASK_LAUNCH_SECS_SCHEMA = Number.class; /** - * Whether or not nimbus should reassign tasks if it detects that a task goes down. + * Whether or not nimbus should reassign tasks if it detects that a task goes down. * Defaults to true, and it's not recommended to change this value. */ public static final String NIMBUS_REASSIGN = "nimbus.reassign"; @@ -462,7 +468,7 @@ public class Config extends HashMap { public static final Object LOGVIEWER_CLEANUP_AGE_MINS_SCHEMA = ConfigValidation.PositiveIntegerValidator; /** - * A list of users allowed to view logs via the Log Viewer + * 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; @@ -594,7 +600,7 @@ public class Config extends HashMap { public static final Object DRPC_AUTHORIZER_ACL_STRICT_SCHEMA = Boolean.class; /** - * DRPC thrift server worker threads + * 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; @@ -606,7 +612,7 @@ public class Config extends HashMap { public static final Object DRPC_MAX_BUFFER_SIZE_SCHEMA = Number.class; /** - * DRPC thrift server queue size + * 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; @@ -618,13 +624,13 @@ public class Config extends HashMap { 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. + * 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 + * 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; @@ -667,7 +673,7 @@ 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. */ @@ -739,15 +745,15 @@ public class Config extends HashMap { 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. + * 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 + * 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; @@ -772,7 +778,7 @@ public class Config extends HashMap { */ public static final String WORKER_RECEIVER_THREAD_COUNT = "topology.worker.receiver.thread.count"; public static final Object WORKER_RECEIVER_THREAD_COUNT_SCHEMA = Number.class; - + /** * How often this worker should heartbeat to the supervisor. */ @@ -806,7 +812,7 @@ public class Config extends HashMap { /** * 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 + * 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; @@ -843,8 +849,8 @@ public class Config extends HashMap { /** * How many instances to create for a spout/bolt. A task runs on a thread with zero or more * other tasks for the same spout/bolt. The number of tasks for a spout/bolt is always - * the same throughout the lifetime of a topology, but the number of executors (threads) for - * a spout/bolt can change over time. This allows a topology to scale to more or less resources + * the same throughout the lifetime of a topology, but the number of executors (threads) for + * a spout/bolt can change over time. This allows a topology to scale to more or less resources * without redeploying the topology or violating the constraints of Storm (such as a fields grouping * guaranteeing that the same value goes to the same task). */ @@ -883,8 +889,8 @@ public class Config extends HashMap { /** * A list of classes that customize storm's kryo instance during start-up. - * Each listed class name must implement IKryoDecorator. During start-up the - * listed class is instantiated with 0 arguments, then its 'decorate' method + * Each listed class name must implement IKryoDecorator. During start-up the + * listed class is instantiated with 0 arguments, then its 'decorate' method * is called with storm's kryo instance as the only argument. */ public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators"; @@ -914,7 +920,7 @@ public class Config extends HashMap { /* * A list of classes implementing IMetricsConsumer (See storm.yaml.example for exact config format). - * Each listed class will be routed all the metrics data generated by the storm metrics API. + * Each listed class will be routed all the metrics data generated by the storm metrics API. * Each listed class maps 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is configurable. */ public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register"; @@ -930,24 +936,24 @@ public class Config extends HashMap { /** - * The maximum number of tuples that can be pending on a spout task at any given time. - * This config applies to individual tasks, not to spouts or topologies as a whole. - * + * The maximum number of tuples that can be pending on a spout task at any given time. + * This config applies to individual tasks, not to spouts or topologies as a whole. + * * A pending tuple is one that has been emitted from a spout but has not been acked or failed yet. - * Note that this config parameter has no effect for unreliable spouts that don't tag + * Note that this config parameter has no effect for unreliable spouts that don't tag * their tuples with a message id. */ - public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; + public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = Number.class; /** * A class that implements a strategy for what to do when a spout needs to wait. Waiting is * triggered in one of two conditions: - * + * * 1. nextTuple emits no tuples * 2. The spout has hit maxSpoutPending and can't emit any more tuples */ - public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy"; + public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy"; public static final Object TOPOLOGY_SPOUT_WAIT_STRATEGY_SCHEMA = String.class; /** @@ -970,7 +976,7 @@ public class Config extends HashMap { public static final Object TOPOLOGY_STATS_SAMPLE_RATE_SCHEMA = Number.class; /** - * The time period that builtin metrics data in bucketed into. + * The time period that builtin metrics data in bucketed into. */ public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS="topology.builtin.metrics.bucket.size.secs"; public static final Object TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS_SCHEMA = Number.class; @@ -1003,7 +1009,7 @@ public class Config extends HashMap { /** * A list of task hooks that are automatically added to every spout and bolt in the topology. An example - * of when you'd do this is to add a hook that integrates with your internal + * of when you'd do this is to add a hook that integrates with your internal * monitoring system. These hooks are instantiated using the zero-arg constructor. */ public static final String TOPOLOGY_AUTO_TASK_HOOKS="topology.auto.task.hooks"; @@ -1017,7 +1023,7 @@ public class Config extends HashMap { public static final Object TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; /** - * The maximum number of messages to batch from the thread receiving off the network to the + * The maximum number of messages to batch from the thread receiving off the network to the * executor queues. Must be a power of 2. */ public static final String TOPOLOGY_RECEIVER_BUFFER_SIZE="topology.receiver.buffer.size"; @@ -1051,14 +1057,14 @@ public class Config extends HashMap { public static final Object TOPOLOGY_DISRUPTOR_WAIT_STRATEGY_SCHEMA = String.class; /** - * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed + * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed * via the TopologyContext. */ public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE="topology.worker.shared.thread.pool.size"; public static final Object TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE_SCHEMA = Number.class; /** - * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example, + * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example, * an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be * reported to Zookeeper per task for every 10 second interval of time. */ @@ -1081,7 +1087,7 @@ public class Config extends HashMap { /** * Name of the topology. This config is automatically set by Storm when the topology is submitted. */ - public final static String TOPOLOGY_NAME="topology.name"; + public final static String TOPOLOGY_NAME="topology.name"; public static final Object TOPOLOGY_NAME_SCHEMA = String.class; /** @@ -1095,19 +1101,19 @@ public class Config extends HashMap { */ 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 */ @@ -1157,9 +1163,9 @@ public class Config extends HashMap { /** * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers) - * for the java.library.path value. java.library.path tells the JVM where + * for the java.library.path value. java.library.path tells the JVM where * to look for native libraries. It is necessary to set this config correctly since - * Storm uses the ZeroMQ and JZMQ native libs. + * Storm uses the ZeroMQ and JZMQ native libs. */ public static final String JAVA_LIBRARY_PATH = "java.library.path"; public static final Object JAVA_LIBRARY_PATH_SCHEMA = String.class; @@ -1178,7 +1184,7 @@ public class Config extends HashMap { */ public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines"; 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 @@ -1192,15 +1198,15 @@ public class Config extends HashMap { */ 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); - } + } public void setDebug(boolean isOn) { setDebug(this, isOn); } - + public static void setNumWorkers(Map conf, int workers) { conf.put(Config.TOPOLOGY_WORKERS, workers); } @@ -1216,7 +1222,7 @@ public static void setNumAckers(Map conf, int numExecutors) { public void setNumAckers(int numExecutors) { setNumAckers(this, numExecutors); } - + public static void setMessageTimeoutSecs(Map conf, int secs) { conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs); } @@ -1224,7 +1230,7 @@ public static void setMessageTimeoutSecs(Map conf, int secs) { public void setMessageTimeoutSecs(int secs) { setMessageTimeoutSecs(this, secs); } - + public static void registerSerialization(Map conf, Class klass) { getRegisteredSerializations(conf).add(klass.getName()); } @@ -1232,17 +1238,17 @@ public static void registerSerialization(Map conf, Class klass) { public void registerSerialization(Class klass) { registerSerialization(this, klass); } - + public static void registerSerialization(Map conf, Class klass, Class serializerClass) { Map register = new HashMap(); register.put(klass.getName(), serializerClass.getName()); - getRegisteredSerializations(conf).add(register); + getRegisteredSerializations(conf).add(register); } public void registerSerialization(Class klass, Class serializerClass) { registerSerialization(this, klass, serializerClass); } - + public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) { HashMap m = new HashMap(); m.put("class", klass.getCanonicalName()); @@ -1270,7 +1276,7 @@ public static void registerDecorator(Map conf, Class k public void registerDecorator(Class klass) { registerDecorator(this, klass); } - + public static void setKryoFactory(Map conf, Class klass) { conf.put(Config.TOPOLOGY_KRYO_FACTORY, klass.getName()); } @@ -1286,7 +1292,7 @@ public static void setSkipMissingKryoRegistrations(Map conf, boolean skip) { public void setSkipMissingKryoRegistrations(boolean skip) { setSkipMissingKryoRegistrations(this, skip); } - + public static void setMaxTaskParallelism(Map conf, int max) { conf.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, max); } @@ -1294,7 +1300,7 @@ public static void setMaxTaskParallelism(Map conf, int max) { public void setMaxTaskParallelism(int max) { setMaxTaskParallelism(this, max); } - + public static void setMaxSpoutPending(Map conf, int max) { conf.put(Config.TOPOLOGY_MAX_SPOUT_PENDING, max); } @@ -1302,23 +1308,23 @@ public static void setMaxSpoutPending(Map conf, int max) { public void setMaxSpoutPending(int max) { setMaxSpoutPending(this, max); } - + public static void setStatsSampleRate(Map conf, double rate) { conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, rate); - } + } public void setStatsSampleRate(double rate) { setStatsSampleRate(this, rate); - } + } public static void setFallBackOnJavaSerialization(Map conf, boolean fallback) { conf.put(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION, fallback); - } + } public void setFallBackOnJavaSerialization(boolean fallback) { setFallBackOnJavaSerialization(this, fallback); - } - + } + private static List getRegisteredSerializations(Map conf) { List ret; if(!conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) { @@ -1329,13 +1335,13 @@ private static List getRegisteredSerializations(Map conf) { conf.put(Config.TOPOLOGY_KRYO_REGISTER, ret); return ret; } - + private static List getRegisteredDecorators(Map conf) { List ret; if(!conf.containsKey(Config.TOPOLOGY_KRYO_DECORATORS)) { ret = new ArrayList(); } else { - ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS)); + ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS)); } conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret); return ret; diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IGroupMappingServiceProvider.java b/storm-core/src/jvm/backtype/storm/security/auth/IGroupMappingServiceProvider.java index 0b49deccf4c..5590b812af1 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/IGroupMappingServiceProvider.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/IGroupMappingServiceProvider.java @@ -39,16 +39,4 @@ public interface IGroupMappingServiceProvider { */ public Set getGroups(String user) throws IOException; - /** - * Refresh the cache of groups and user mapping - * @throws IOException - */ - public void cacheGroupsRefresh() throws IOException; - /** - * Caches the group user information - * @param groups list of groups to add to cache - * @throws IOException - */ - public void cacheGroupsAdd(Set groups) throws IOException; - } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java index b8c83237a99..438b9386636 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java @@ -23,7 +23,10 @@ import java.util.HashSet; import java.util.Map; import java.util.StringTokenizer; +import backtype.storm.Config; +import backtype.storm.utils.Utils; import backtype.storm.utils.ShellUtils; +import backtype.storm.utils.TimeCacheMap; import backtype.storm.utils.ShellUtils.ExitCodeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,12 +36,17 @@ public class ShellBasedUnixGroupsMapping implements IGroupMappingServiceProvider { public static Logger LOG = LoggerFactory.getLogger(ShellBasedUnixGroupsMapping.class); + public TimeCacheMap> cachedGroups; /** * Invoked once immediately after construction * @param storm_conf Storm configuration */ - public void prepare(Map storm_conf) {} + @Override + public void prepare(Map storm_conf) { + int timeout = Utils.getInt(storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS)); + cachedGroups = new TimeCacheMap>(timeout); + } /** * Returns list of groups for a user @@ -48,15 +56,13 @@ public void prepare(Map storm_conf) {} */ @Override public Set getGroups(String user) throws IOException { - return getUnixGroups(user); - } - - @Override - public void cacheGroupsRefresh() throws IOException { - } - - @Override - public void cacheGroupsAdd(Set groups) throws IOException { + if(cachedGroups.containsKey(user)) { + return cachedGroups.get(user); + } + Set groups = getUnixGroups(user); + if(!groups.isEmpty()) + cachedGroups.put(user,groups); + return groups; } /** From 87fc5541aa617c52cca740bc1564a63f4601c91b Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Tue, 8 Jul 2014 22:08:33 +0000 Subject: [PATCH 28/61] Use TBinaryProtocol without max buffer --- .../jvm/backtype/storm/security/auth/SaslTransportPlugin.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java index 348fae3025b..64bec193b53 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java @@ -67,14 +67,13 @@ public TServer getServer(TProcessor processor) throws IOException, TTransportExc TTransportFactory serverTransportFactory = getServerTransportFactory(); TServerSocket serverTransport = new TServerSocket(port); int numWorkerThreads = type.getNumThreads(storm_conf); - int maxBufferSize = type.getMaxBufferSize(storm_conf); Integer queueSize = type.getQueueSize(storm_conf); TThreadPoolServer.Args server_args = new TThreadPoolServer.Args(serverTransport). processor(new TUGIWrapProcessor(processor)). minWorkerThreads(numWorkerThreads). maxWorkerThreads(numWorkerThreads). - protocolFactory(new TBinaryProtocol.Factory(false, true, maxBufferSize)); + protocolFactory(new TBinaryProtocol.Factory(false, true)); if (serverTransportFactory != null) { server_args.transportFactory(serverTransportFactory); From 8973e30d70aa615c13ee02f10e20c4690ec1e43e Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 9 Jul 2014 17:49:59 +0000 Subject: [PATCH 29/61] Fixed merge error in drpc.clj --- storm-core/src/clj/backtype/storm/daemon/drpc.clj | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj index baba96b1303..1340e49423f 100644 --- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj +++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj @@ -208,10 +208,10 @@ ;; invocations that will unblock those threads handler-server (when (> drpc-port 0) (ThriftServer. conf - (DistributedRPC$Processor. service-handler) + (DistributedRPC$Processor. drpc-service-handler) ThriftConnectionType/DRPC)) invoke-server (ThriftServer. conf - (DistributedRPCInvocations$Processor. service-handler) + (DistributedRPCInvocations$Processor. drpc-service-handler) ThriftConnectionType/DRPC_INVOCATIONS) http-creds-handler (AuthUtils/GetDrpcHttpCredentialsPlugin conf)] (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] From 2c78f911e495518b65241ee0904d37d0f854b8b7 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Fri, 11 Jul 2014 11:50:07 -0700 Subject: [PATCH 30/61] STORM-346: added AutoHDFS class that will get hdfs delegation tokens on behalf of users, push it to workers and renew the delegation tokens automatically. INimbusCredentialPlugin is a plugin interfae that will be executed as part of nimbu's submit topology operation and fetch credential from nimbus process on behalf of user. AutoHDFS will renew the tokens on nimbus host as long as credential has more then 24 hours on renew cycle, if tokens are going to expire in next 24 hours then new tokens will be fetched and pushed to all workers. --- .../src/clj/backtype/storm/daemon/nimbus.clj | 40 +++-- storm-core/src/jvm/backtype/storm/Config.java | 18 +- .../security/INimbusCredentialPlugin.java | 39 +++++ .../storm/security/auth/AuthUtils.java | 23 +++ .../security/auth/ICredentialsRenewer.java | 3 +- .../security/auth/kerberos/AutoHDFS.java | 154 +++++++----------- .../storm/security/auth/kerberos/AutoTGT.java | 2 +- 7 files changed, 152 insertions(+), 127 deletions(-) create mode 100644 storm-core/src/jvm/backtype/storm/security/INimbusCredentialPlugin.java diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj index e2952d0edd2..3fe1a0a1ec6 100644 --- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj @@ -857,24 +857,6 @@ (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) - (.renew renewer new-creds)) - (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))) @@ -952,6 +934,25 @@ ) ) +(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) + topology-conf (try-read-storm-conf (:conf nimbus) id)] + (if orig-creds + (let [new-creds (HashMap. orig-creds)] + (doseq [renewer renewers] + (log-message "Renewing Creds For " id " with " renewer) + (.renew renewer new-creds topology-conf)) + (when-not (= orig-creds new-creds) + (.set-credentials! storm-cluster-state id new-creds) + ))))))))) + (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) @@ -1046,7 +1047,10 @@ (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) + nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins total-storm-conf) storm-cluster-state (:storm-cluster-state nimbus)] + (doseq [nimbus-autocred-plugin nimbus-autocred-plugins] + (.populateCredentials nimbus-autocred-plugin credentials total-storm-conf)) (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 diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 785d1e2ecad..3c95f8a87ff 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -425,6 +425,13 @@ public class Config extends HashMap { public static final String NIMBUS_CREDENTIAL_RENEWERS = "nimbus.credential.renewers.classes"; public static final Object NIMBUS_CREDENTIAL_RENEWERS_SCHEMA = ConfigValidation.StringsValidator; + /** + * A list of plugins that nimbus should load during submit topology to populate + * credentials on user's behalf. + */ + public static final String NIMBUS_AUTO_CRED_PLUGINS = "nimbus.autocredential.plugins.classes"; + public static final Object NIMBUS_AUTO_CRED_PLUGINS_SCHEMA = ConfigValidation.StringsValidator; + /** * Storm UI binds to this port. */ @@ -1190,15 +1197,10 @@ public class Config extends HashMap { /** * HDFS information, used to get the delegation token on behalf of the topology * submitter user and renew the tokens. see {@link backtype.storm.security.auth.kerberos.AutoHDFS} + * kerberos principal name with realm should be provided. */ - public static final String HDFS_NAMENODE_URL = "topology.hdfs.namenodeURI"; - public static final Object HDFS_NAMENODE_URL_SCHEMA = String.class; - - public static final Object HDFS_USER = "topology.hdfs.user"; - public static final Object HDFS_USER_SCHEMA = String.class; - - public static final Object HDFS_USER_KEYTAB = "topology.hdfs.userKeyTab"; - public static final Object HDFS_USER_KEYTAB_SCHEMA = String.class; + public static final Object HDFS_PRINCIPAL = "topology.hdfs.user"; + public static final Object HDFS_PRINCIPAL_SCHEMA = String.class; public static void setDebug(Map conf, boolean isOn) { conf.put(Config.TOPOLOGY_DEBUG, isOn); diff --git a/storm-core/src/jvm/backtype/storm/security/INimbusCredentialPlugin.java b/storm-core/src/jvm/backtype/storm/security/INimbusCredentialPlugin.java new file mode 100644 index 00000000000..4c56863d264 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/security/INimbusCredentialPlugin.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.security; + +import java.util.Map; + +/** + * Nimbus auto credential plugin that will be called on nimbus host + * during submit topology option. User can specify a list of implementation using config key + * nimbus.autocredential.plugins.classes. + */ +public interface INimbusCredentialPlugin { + + /** + * Method that will be called on nimbus as part of submit topology. This plugin will be called + * at least once during the submit Topology action. It will be not be called during activate instead + * the credentials return by this method will be merged with the other credentials in the topology + * and stored in zookeeper. + * @param credentials credentials map where more credentials will be added. + * @param conf topology configuration + * @return + */ + void populateCredentials(Map credentials, Map conf); +} diff --git a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java index b5118c7f6f8..fc06698e075 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java @@ -23,6 +23,8 @@ import javax.security.auth.Subject; import java.security.NoSuchAlgorithmException; import java.security.URIParameter; + +import backtype.storm.security.INimbusCredentialPlugin; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; @@ -108,6 +110,27 @@ public static Collection GetCredentialRenewers(Map conf) { } } + /** + * Get all the Nimbus Auto cred plugins that users want to use. + * @param topologyConf topologyConfiguration to use. + * @return nimbus auto credential plugins. + */ + public static Collection getNimbusAutoCredPlugins(Map topologyConf) { + try { + Set ret = new HashSet(); + Collection clazzes = (Collection)topologyConf.get(Config.NIMBUS_AUTO_CRED_PLUGINS); + if (clazzes != null) { + for (String clazz : clazzes) { + INimbusCredentialPlugin inst = (INimbusCredentialPlugin)Class.forName(clazz).newInstance(); + ret.add(inst); + } + } + return ret; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + /** * Get all of the configured AutoCredential Plugins. * @param storm_conf the storm configuration to use. diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java b/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java index a84a9e853c9..3eaf6c4b36c 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java @@ -35,6 +35,7 @@ public interface ICredentialsRenewer { /** * Renew any credentials that need to be renewed. (Update the credentials if needed) * @param credentials the credentials that may have something to renew. + * @param topologyConf topology configuration. */ - public void renew(Map credentials); + public void renew(Map credentials, Map topologyConf); } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java index 80b5916b609..7685fb10b51 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java @@ -19,6 +19,7 @@ package backtype.storm.security.auth.kerberos; import backtype.storm.Config; +import backtype.storm.security.INimbusCredentialPlugin; import backtype.storm.security.auth.IAutoCredentials; import backtype.storm.security.auth.ICredentialsRenewer; import org.slf4j.Logger; @@ -30,88 +31,28 @@ import java.lang.reflect.Method; import java.net.URI; import java.util.Collection; +import java.util.HashMap; import java.util.Map; /** * Automatically get HDFS delegation tokens and push it to user's topology. The class * assumes that HDFS configuration files are in your class path. */ -public class AutoHDFS implements IAutoCredentials, ICredentialsRenewer { +public class AutoHDFS implements IAutoCredentials, ICredentialsRenewer, INimbusCredentialPlugin { private static final Logger LOG = LoggerFactory.getLogger(AutoHDFS.class); public static final String HDFS_CREDENTIALS = "HDFS_CREDENTIALS"; - private static final String CONF_KEYTAB_KEY = "keytab"; - private static final String CONF_USER_KEY = "user"; - - private Map conf; public void prepare(Map conf) { - this.conf = conf; + LOG.debug("no op."); } @SuppressWarnings("unchecked") - private Object getConfiguration() { - try { - final String hdfsUser = (String) conf.get(Config.HDFS_USER); - final String hdfsUserKeyTab = (String) conf.get(Config.HDFS_USER_KEYTAB); - - /** - * Configuration configuration = new Configuration(); - * configuration.set(CONF_KEYTAB_KEY, hdfsUserKeyTab); - * configuration.set(CONF_USER_KEY, hdfsUser); - */ - Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); - Object configuration = configurationClass.newInstance(); - - Method setMethod = configurationClass.getMethod("set", String.class, String.class); - setMethod.invoke(configuration, CONF_KEYTAB_KEY, hdfsUserKeyTab); - setMethod.invoke(configuration, CONF_USER_KEY, hdfsUser); - /** - * Following are the minimum set of configuration that needs to be set, users should have hdfs-site.xml - * and core-site.xml in the class path which should set these configuration. - * setMethod.invoke(configuration, "hadoop.security.authentication", "KERBEROS"); - * setMethod.invoke(configuration,"dfs.namenode.kerberos.principal", - * "hdfs/zookeeper.witzend.com@WITZEND.COM"); - * setMethod.invoke(configuration, "hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002"); - */ - - setMethod.invoke(configuration, "hadoop.security.authentication", "KERBEROS"); - setMethod.invoke(configuration, "dfs.namenode.kerberos.principal","hdfs/zookeeper.witzend.com@WITZEND.COM"); - setMethod.invoke(configuration, "hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002"); - - //UserGroupInformation.setConfiguration(configuration); - final Class ugiClass = Class.forName("org.apache.hadoop.security.UserGroupInformation"); - Method setConfigurationMethod = ugiClass.getMethod("setConfiguration", configurationClass); - setConfigurationMethod.invoke(null, configuration); - return configuration; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - @SuppressWarnings("unchecked") - private void login(Object configuration) { - try { - Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); - final Class securityUtilClass = Class.forName("org.apache.hadoop.security.SecurityUtil"); - Method loginMethod = securityUtilClass.getMethod("login", configurationClass, String.class, String.class); - loginMethod.invoke(null, configuration, CONF_KEYTAB_KEY, CONF_USER_KEY); - } catch (Exception e) { - throw new RuntimeException("Failed to login to hdfs .", e); - } - } - - @SuppressWarnings("unchecked") - private byte[] getHDFSCredsWithDelegationToken() throws Exception { + private byte[] getHDFSCredsWithDelegationToken(Map conf) throws Exception { try { /** * What we want to do is following: - * Configuration configuration = new Configuration(); - * configuration.set(CONF_KEYTAB_KEY, hdfsUserKeyTab); - * configuration.set(CONF_USER_KEY, hdfsUser); - * UserGroupInformation.setConfiguration(configuration); * if(UserGroupInformation.isSecurityEnabled) { - * SecurityUtil.login(configuration, CONF_KEYTAB_KEY, CONF_USER_KEY); * FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); * UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); * UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(topologySubmitterUser, ugi); @@ -119,24 +60,32 @@ private byte[] getHDFSCredsWithDelegationToken() throws Exception { * fs.addDelegationToken(hdfsUser, credential); * } * and then return the credential object as a bytearray. + * + * Following are the minimum set of configuration that needs to be set, users should have hdfs-site.xml + * and core-site.xml in the class path which should set these configuration. + * configuration.set("hadoop.security.authentication", "KERBEROS"); + * configuration.set("dfs.namenode.kerberos.principal", + * "hdfs/zookeeper.witzend.com@WITZEND.COM"); + * configuration.set("hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002"); + * anf the ticket cache must have the hdfs user's creds. */ - Object configuration = getConfiguration(); + Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + Object configuration = configurationClass.newInstance(); + + //UserGroupInformation.isSecurityEnabled final Class ugiClass = Class.forName("org.apache.hadoop.security.UserGroupInformation"); final Method isSecurityEnabledMethod = ugiClass.getDeclaredMethod("isSecurityEnabled"); boolean isSecurityEnabled = (Boolean)isSecurityEnabledMethod.invoke(null); - if(isSecurityEnabled) { - login(configuration); - final URI nameNodeURI = URI.create((String) conf.get(Config.HDFS_NAMENODE_URL)); + if(isSecurityEnabled) { final String topologySubmitterUser = (String) conf.get(Config.TOPOLOGY_SUBMITTER_USER); - final String hdfsUser = (String) conf.get(Config.HDFS_USER); - - Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + final String hdfsUser = (String) conf.get(Config.HDFS_PRINCIPAL); //FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); Class fileSystemClass = Class.forName("org.apache.hadoop.fs.FileSystem"); + Object defaultNameNodeURI = fileSystemClass.getMethod("getDefaultUri", configurationClass).invoke(null, configuration); Method getMethod = fileSystemClass.getMethod("get", URI.class, configurationClass, String.class); - Object fileSystem = getMethod.invoke(null, nameNodeURI, configuration, topologySubmitterUser); + Object fileSystem = getMethod.invoke(null, defaultNameNodeURI, configuration, topologySubmitterUser); //UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); Method getCurrentUserMethod = ugiClass.getMethod("getCurrentUser"); @@ -164,7 +113,6 @@ private byte[] getHDFSCredsWithDelegationToken() throws Exception { out.flush(); out.close(); - LOG.info(bao.toString()); return bao.toByteArray(); } else { throw new RuntimeException("Security is not enabled for HDFS"); @@ -175,14 +123,19 @@ private byte[] getHDFSCredsWithDelegationToken() throws Exception { } @Override - public void populateCredentials(Map credentials) { + public void populateCredentials(Map credentials, Map conf) { try { - credentials.put(HDFS_CREDENTIALS, DatatypeConverter.printBase64Binary( getHDFSCredsWithDelegationToken())); + credentials.put(HDFS_CREDENTIALS, DatatypeConverter.printBase64Binary( getHDFSCredsWithDelegationToken(conf))); } catch (Exception e) { LOG.warn("Could not populate HDFS credentials.", e); } } + @Override + public void populateCredentials(Map credentials) { + LOG.debug("populateCredentials is a noop, nimbus should populate the crdes."); + } + /** * * @param credentials map with creds. @@ -235,63 +188,66 @@ private static void addCredentialToSubject(Subject subject, Map @Override @SuppressWarnings("unchecked") - public void renew(Map credentials) { + public void renew(Map credentials, Map topologyConf) { Object credential = getHDFSCredential(credentials); + //maximum allowed expiration time until which tokens will keep renewing, + //currently set to 1 day. + final long MAX_ALLOWED_EXPIRATION_MILLIS = 24 * 60 * 60 * 1000; + /** * We are trying to do the following : * List tokens = credential.getAllTokens(); * for(Token token: tokens) { - * token.renew(configuration); + * long expiration = token.renew(configuration); * } - * TODO: Need to talk to HDFS guys to check what is recommended way to identify a token that is beyond - * renew cycle, once we can identified tokens are not renewable we can repopulate credentials by asking HDFS - * to issue new tokens. Until we have a better way any exception during renewal would result in attempt to - * get new delegation tokens. */ - if (credential != null) { - try { - Object configuration = getConfiguration(); - + try { + if (credential != null) { Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); + Object configuration = configurationClass.newInstance(); + Class credentialClass = Class.forName("org.apache.hadoop.security.Credentials"); Class tokenClass = Class.forName("org.apache.hadoop.security.token.Token"); - Method renewMethod = tokenClass.getMethod("renew", configurationClass); + Method renewMethod = tokenClass.getMethod("renew", configurationClass); Method getAllTokensMethod = credentialClass.getMethod("getAllTokens"); Collection tokens = (Collection) getAllTokensMethod.invoke(credential); - for(Object token : tokens) { - renewMethod.invoke(token, configuration); + for (Object token : tokens) { + long expiration = (Long) renewMethod.invoke(token, configuration); + if(expiration < MAX_ALLOWED_EXPIRATION_MILLIS) { + LOG.debug("expiration {} is less then MAX_ALLOWED_EXPIRATION_MILLIS {}, getting new tokens", + expiration, MAX_ALLOWED_EXPIRATION_MILLIS); + populateCredentials(credentials, topologyConf); + } } - } catch(Exception e) { - LOG.warn("could not renew the credentials, one of the possible reason is tokens are beyond " + - "renewal period so attempting to get new tokens.", e); - populateCredentials(credentials); } + } catch (Exception e) { + LOG.warn("could not renew the credentials, one of the possible reason is tokens are beyond " + + "renewal period so attempting to get new tokens.", e); + populateCredentials(credentials); } } @SuppressWarnings("unchecked") public static void main(String[] args) throws Exception { Map conf = new java.util.HashMap(); - conf.put(Config.HDFS_NAMENODE_URL, args[0]); - conf.put(Config.TOPOLOGY_SUBMITTER_USER, args[1]); //with realm e.g. storm@WITZEND.COM - conf.put(Config.HDFS_USER, args[2]); //with realm e.g. hdfs@WITZEND.COM - conf.put(Config.HDFS_USER_KEYTAB, args[3]); + conf.put(Config.TOPOLOGY_SUBMITTER_PRINCIPAL, args[0]); //with realm e.g. storm@WITZEND.COM + conf.put(Config.HDFS_PRINCIPAL, args[1]); //with realm e.g. hdfs@WITZEND.COM AutoHDFS autoHDFS = new AutoHDFS(); autoHDFS.prepare(conf); - Map creds = new java.util.HashMap(); - autoHDFS.populateCredentials(creds); + Map creds = new HashMap(); + autoHDFS.populateCredentials(creds, conf); LOG.info("Got HDFS credentials", AutoHDFS.getHDFSCredential(creds)); Subject s = new Subject(); autoHDFS.populateSubject(s, creds); LOG.info("Got a Subject "+ s); - autoHDFS.renew(creds); + autoHDFS.renew(creds, conf); LOG.info("renewed credentials", AutoHDFS.getHDFSCredential(creds)); } } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java index 52bf540e679..3bbb0d374bd 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java @@ -236,7 +236,7 @@ private long getRefreshTime(KerberosTicket tgt) { } @Override - public void renew(Map credentials) { + public void renew(Map credentials, Map topologyConf) { KerberosTicket tgt = getTGT(credentials); if (tgt != null) { long refreshTime = getRefreshTime(tgt); From 41986445fb89ff77c101fcdd6daccb945160e8a9 Mon Sep 17 00:00:00 2001 From: Raghavendra Nandagopal Date: Tue, 22 Jul 2014 13:20:54 -0700 Subject: [PATCH 31/61] STORM-348: Netty SASL Authentication --- storm-core/src/jvm/backtype/storm/Config.java | 5 + .../storm/messaging/netty/Client.java | 29 +-- .../storm/messaging/netty/ControlMessage.java | 4 +- .../storm/messaging/netty/MessageDecoder.java | 32 +++- .../storm/messaging/netty/MessageEncoder.java | 4 + .../messaging/netty/SaslMessageToken.java | 100 +++++++++++ .../messaging/netty/SaslNettyClient.java | 167 +++++++++++++++++ .../messaging/netty/SaslNettyClientState.java | 31 ++++ .../messaging/netty/SaslNettyServer.java | 168 ++++++++++++++++++ .../messaging/netty/SaslNettyServerState.java | 31 ++++ .../netty/SaslStormClientHandler.java | 159 +++++++++++++++++ .../SaslStormServerAuthorizeHandler.java | 83 +++++++++ .../netty/SaslStormServerHandler.java | 159 +++++++++++++++++ .../storm/messaging/netty/SaslUtils.java | 61 +++++++ .../storm/messaging/netty/Server.java | 27 +-- .../netty/StormClientPipelineFactory.java | 26 ++- .../netty/StormServerPipelineFactory.java | 31 +++- 17 files changed, 1077 insertions(+), 40 deletions(-) create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClientState.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServerState.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerAuthorizeHandler.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 3b3f7e54f15..46b120c3c7e 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -105,6 +105,11 @@ public class Config extends HashMap { public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms"; public static final Object STORM_NETTY_FLUSH_CHECK_INTERVAL_MS_SCHEMA = Number.class; + /** + * Netty based messaging: Is authentication required for Netty messaging from client worker process to server worker process. + */ + public static final String STORM_MESSAGING_NETTY_AUTHENTICATION = "storm.messaging.netty.authentication"; + public static final Object STORM_MESSAGING_NETTY_AUTHENTICATION_SCHEMA = Boolean.class; /** * A list of hosts of ZooKeeper servers used to manage the cluster. diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 8d2d22167de..64a1757ba28 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -17,17 +17,6 @@ */ package backtype.storm.messaging.netty; -import backtype.storm.Config; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.Utils; -import org.jboss.netty.bootstrap.ClientBootstrap; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelFutureListener; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Iterator; @@ -39,6 +28,19 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import org.jboss.netty.bootstrap.ClientBootstrap; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.ChannelFuture; +import org.jboss.netty.channel.ChannelFutureListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.utils.Utils; + public class Client implements IConnection { private static final Logger LOG = LoggerFactory.getLogger(Client.class); private static final String PREFIX = "Netty-Client-"; @@ -57,8 +59,10 @@ public class Client implements IConnection { private int messageBatchSize; private AtomicLong pendings; + + Map storm_conf; - MessageBatch messageBatch = null; + MessageBatch messageBatch = null; private AtomicLong flushCheckTimer; private int flushCheckInterval; private ScheduledExecutorService scheduler; @@ -66,6 +70,7 @@ public class Client implements IConnection { @SuppressWarnings("rawtypes") Client(Map storm_conf, ChannelFactory factory, ScheduledExecutorService scheduler, String host, int port) { + this.storm_conf = storm_conf; this.factory = factory; this.scheduler = scheduler; channelRef = new AtomicReference(null); diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java b/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java index b7335b3f57d..fb3efe612ef 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java @@ -27,7 +27,9 @@ enum ControlMessage { CLOSE_MESSAGE((short)-100), EOB_MESSAGE((short)-201), OK_RESPONSE((short)-200), - FAILURE_RESPONSE((short)-400); + FAILURE_RESPONSE((short)-400), + SASL_TOKEN_MESSAGE_REQUEST((short)-202), + SASL_COMPLETE_REQUEST((short)-203); private short code; diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java index 72c3cf7ceeb..7d8bf54d67d 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java @@ -68,8 +68,38 @@ protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffe return ctrl_msg; } } + + //case 2: SaslTokenMessageRequest + if(code==-500) { + // Make sure that we have received at least an integer (length) + if (buf.readableBytes() < 4) { + //need more data + buf.resetReaderIndex(); + return null; + } + + // Read the length field. + int length = buf.readInt(); + if (length<=0) { + return new SaslMessageToken(null); + } + + // Make sure if there's enough bytes in the buffer. + if (buf.readableBytes() < length) { + // The whole bytes were not received yet - return null. + buf.resetReaderIndex(); + return null; + } + + // There's enough bytes in the buffer. Read it. + ChannelBuffer payload = buf.readBytes(length); + + // Successfully decoded a frame. + // Return a SaslTokenMessageRequest object + return new SaslMessageToken(payload.array()); + } - // case 2: task Message + // case 3: task Message short task = code; // Make sure that we have received at least an integer (length) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageEncoder.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageEncoder.java index e6e65c3c130..e5dd22fcdca 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageEncoder.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageEncoder.java @@ -32,6 +32,10 @@ protected Object encode(ChannelHandlerContext ctx, Channel channel, Object obj) return ((MessageBatch)obj).buffer(); } + if (obj instanceof SaslMessageToken) { + return ((SaslMessageToken)obj).buffer(); + } + throw new RuntimeException("Unsupported encoding of object of class "+obj.getClass().getName()); } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java new file mode 100644 index 00000000000..8383d2c0d55 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java @@ -0,0 +1,100 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.buffer.ChannelBufferOutputStream; +import org.jboss.netty.buffer.ChannelBuffers; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Send and receive SASL tokens. + */ +public class SaslMessageToken { + /** Class logger */ + private static final Logger LOG = LoggerFactory + .getLogger(SaslMessageToken.class); + + /** Used for client or server's token to send or receive from each other. */ + private byte[] token; + + /** + * Constructor used for reflection only. + */ + public SaslMessageToken() { + } + + /** + * Constructor used to send request. + * + * @param token + * the SASL token, generated by a SaslClient or SaslServer. + */ + public SaslMessageToken(byte[] token) { + this.token = token; + } + + /** + * Read accessor for SASL token + * + * @return saslToken SASL token + */ + public byte[] getSaslToken() { + return token; + } + + /** + * Write accessor for SASL token + * + * @param token + * SASL token + */ + public void setSaslToken(byte[] token) { + this.token = token; + } + + int encodeLength() { + return 2+4+token.length; + } + + /** + * encode the current SaslToken Message into a channel buffer + * SaslTokenMessageRequest is encoded as: + * identifier .... short(2) always it is -500 + * payload length .... int + * payload .... byte[] + * @throws Exception + */ + ChannelBuffer buffer() throws Exception { + ChannelBufferOutputStream bout = new ChannelBufferOutputStream( + ChannelBuffers.directBuffer(encodeLength())); + short identifier = -500; + int payload_len = 0; + if (token != null) + payload_len = token.length; + + bout.writeShort((short)identifier); + bout.writeInt((int)payload_len); + if(payload_len>0) { + bout.write(token); + } + bout.close(); + return bout.buffer(); + } +} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java new file mode 100644 index 00000000000..a4f1b5eb6d8 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java @@ -0,0 +1,167 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import java.io.IOException; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.RealmCallback; +import javax.security.sasl.RealmChoiceCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements SASL logic for storm worker client processes. + */ +public class SaslNettyClient { + + private static final Logger LOG = LoggerFactory + .getLogger(SaslNettyClient.class); + + /** + * Used to respond to server's counterpart, SaslServer with SASL tokens + * represented as byte arrays. + */ + private SaslClient saslClient; + + /** + * Create a SaslNettyClient for authentication with servers. + */ + public SaslNettyClient(String topologyUser) { + try { + LOG.debug("SaslNettyClient: Creating SASL " + + SaslUtils.AUTH_DIGEST_MD5 + + " client to authenticate to server "); + + saslClient = Sasl.createSaslClient( + new String[] { SaslUtils.AUTH_DIGEST_MD5 }, null, null, + SaslUtils.DEFAULT_REALM, SaslUtils.getSaslProps(), + new SaslClientCallbackHandler(topologyUser)); + + } catch (IOException e) { + LOG.error("SaslNettyClient: Could not obtain topology token for Netty " + + "Client to use to authenticate with a Netty Server."); + saslClient = null; + } + } + + public boolean isComplete() { + return saslClient.isComplete(); + } + + /** + * Respond to server's SASL token. + * + * @param saslTokenMessage + * contains server's SASL token + * @return client's response SASL token + */ + public byte[] saslResponse(SaslMessageToken saslTokenMessage) { + try { + byte[] retval = saslClient.evaluateChallenge(saslTokenMessage + .getSaslToken()); + return retval; + } catch (SaslException e) { + LOG.error( + "saslResponse: Failed to respond to SASL server's token:", + e); + return null; + } + } + + /** + * Implementation of javax.security.auth.callback.CallbackHandler that works + * with Storm topology tokens. + */ + private static class SaslClientCallbackHandler implements CallbackHandler { + /** Generated username contained in TopologyToken */ + private final String userName; + /** Generated password contained in TopologyToken */ + private final char[] userPassword; + + /** + * Set private members using topology token. + * + * @param topologyToken + */ + public SaslClientCallbackHandler(String topologyToken) { + this.userName = SaslUtils + .encodeIdentifier(topologyToken.getBytes()); + this.userPassword = SaslUtils.encodePassword(topologyToken + .getBytes()); + } + + /** + * Implementation used to respond to SASL tokens from server. + * + * @param callbacks + * objects that indicate what credential information the + * server's SaslServer requires from the client. + * @throws UnsupportedCallbackException + */ + public void handle(Callback[] callbacks) + throws UnsupportedCallbackException { + NameCallback nc = null; + PasswordCallback pc = null; + RealmCallback rc = null; + for (Callback callback : callbacks) { + if (callback instanceof RealmChoiceCallback) { + continue; + } else if (callback instanceof NameCallback) { + nc = (NameCallback) callback; + } else if (callback instanceof PasswordCallback) { + pc = (PasswordCallback) callback; + } else if (callback instanceof RealmCallback) { + rc = (RealmCallback) callback; + } else { + throw new UnsupportedCallbackException(callback, + "handle: Unrecognized SASL client callback"); + } + } + if (nc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("handle: SASL client callback: setting username: " + + userName); + } + nc.setName(userName); + } + if (pc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("handle: SASL client callback: setting userPassword"); + } + pc.setPassword(userPassword); + } + if (rc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("handle: SASL client callback: setting realm: " + + rc.getDefaultText()); + } + rc.setText(rc.getDefaultText()); + } + } + } + +} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClientState.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClientState.java new file mode 100644 index 00000000000..6df6c538456 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClientState.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelLocal; + +final class SaslNettyClientState { + + public static final ChannelLocal getSaslNettyClient = new ChannelLocal() { + protected SaslNettyClient initialValue(Channel channel) { + return null; + } + }; + +} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java new file mode 100644 index 00000000000..1178bd65530 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.HashMap; +import java.util.Map; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; + +import org.apache.commons.codec.binary.Base64; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SaslNettyServer { + + private static final Logger LOG = LoggerFactory + .getLogger(SaslNettyServer.class); + + private SaslServer saslServer; + + SaslNettyServer(String topologyToken) throws IOException { + LOG.debug("SaslNettyServer: Topology token is: " + topologyToken + + " with authmethod " + SaslUtils.AUTH_DIGEST_MD5); + + try { + + SaslDigestCallbackHandler ch = new SaslNettyServer.SaslDigestCallbackHandler( + topologyToken); + + saslServer = Sasl.createSaslServer(SaslUtils.AUTH_DIGEST_MD5, null, + SaslUtils.DEFAULT_REALM, SaslUtils.getSaslProps(), ch); + + } catch (SaslException e) { + LOG.error("SaslNettyServer: Could not create SaslServer: " + e); + } + + } + + public boolean isComplete() { + return saslServer.isComplete(); + } + + public String getUserName() { + return saslServer.getAuthorizationID(); + } + + + + /** CallbackHandler for SASL DIGEST-MD5 mechanism */ + public static class SaslDigestCallbackHandler implements CallbackHandler { + + /** Used to authenticate the clients */ + private String topologyToken; + + public SaslDigestCallbackHandler(String topologyToken) { + LOG.debug("SaslDigestCallback: Creating SaslDigestCallback handler " + + "with topology token: " + topologyToken); + this.topologyToken = topologyToken; + } + + @Override + public void handle(Callback[] callbacks) throws IOException, + UnsupportedCallbackException { + NameCallback nc = null; + PasswordCallback pc = null; + AuthorizeCallback ac = null; + + for (Callback callback : callbacks) { + if (callback instanceof AuthorizeCallback) { + ac = (AuthorizeCallback) callback; + } else if (callback instanceof NameCallback) { + nc = (NameCallback) callback; + } else if (callback instanceof PasswordCallback) { + pc = (PasswordCallback) callback; + } else if (callback instanceof RealmCallback) { + continue; // realm is ignored + } else { + throw new UnsupportedCallbackException(callback, + "handle: Unrecognized SASL DIGEST-MD5 Callback"); + } + } + + if(nc!=null) { + LOG.debug("handle: SASL server DIGEST-MD5 callback: setting " + + "username for client: " + topologyToken); + + nc.setName(topologyToken); + } + + if (pc != null) { + char[] password = SaslUtils.encodePassword(topologyToken.getBytes()); + + LOG.debug("handle: SASL server DIGEST-MD5 callback: setting " + + "password for client: " + topologyToken); + + pc.setPassword(password); + } + if (ac != null) { + + String authid = ac.getAuthenticationID(); + String authzid = ac.getAuthorizationID(); + + if (authid.equals(authzid)) { + ac.setAuthorized(true); + } else { + ac.setAuthorized(false); + } + + if (ac.isAuthorized()) { + if (LOG.isDebugEnabled()) { + String username = topologyToken; + LOG.debug("handle: SASL server DIGEST-MD5 callback: setting " + + "canonicalized client ID: " + username); + } + ac.setAuthorizedID(authzid); + } + } + } + } + + /** + * Used by SaslTokenMessage::processToken() to respond to server SASL + * tokens. + * + * @param token + * Server's SASL token + * @return token to send back to the server. + */ + public byte[] response(byte[] token) { + try { + LOG.debug("response: Responding to input token of length: " + + token.length); + byte[] retval = saslServer.evaluateResponse(token); + LOG.debug("response: Response token length: " + retval.length); + return retval; + } catch (SaslException e) { + LOG.error("response: Failed to evaluate client token of length: " + + token.length + " : " + e); + return null; + } + } +} \ No newline at end of file diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServerState.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServerState.java new file mode 100644 index 00000000000..980095930ee --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServerState.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelLocal; + +final class SaslNettyServerState { + + public static final ChannelLocal getSaslNettyServer = new ChannelLocal() { + protected SaslNettyServer initialValue(Channel channel) { + return null; + } + }; + +} \ No newline at end of file diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java new file mode 100644 index 00000000000..ca38c962978 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import java.io.IOException; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.ChannelStateEvent; +import org.jboss.netty.channel.Channels; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelUpstreamHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +public class SaslStormClientHandler extends SimpleChannelUpstreamHandler { + + private static final Logger LOG = LoggerFactory + .getLogger(SaslStormClientHandler.class); + private Client client; + long start_time; + /** Used for client or server's token to send or receive from each other. */ + private byte[] token; + private String topologyUser; + + public SaslStormClientHandler(Client client) throws IOException { + this.client = client; + start_time = System.currentTimeMillis(); + loadTopologyToken(); + } + + @Override + public void channelConnected(ChannelHandlerContext ctx, + ChannelStateEvent event) { + // register the newly established channel + Channel channel = ctx.getChannel(); + + LOG.info("Connection established from " + channel.getLocalAddress() + + " to " + channel.getRemoteAddress()); + + try { + SaslNettyClient saslNettyClient = SaslNettyClientState.getSaslNettyClient + .get(channel); + + if (saslNettyClient == null) { + LOG.debug("Creating saslNettyClient now " + "for channel: " + + channel); + saslNettyClient = new SaslNettyClient(topologyUser); + SaslNettyClientState.getSaslNettyClient.set(channel, + saslNettyClient); + } + channel.write(ControlMessage.SASL_TOKEN_MESSAGE_REQUEST); + } catch (Exception e) { + LOG.error("Failed to authenticate with server " + "due to error: " + + e); + } + return; + + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) + throws Exception { + LOG.debug("send/recv time (ms): {}", + (System.currentTimeMillis() - start_time)); + + Channel channel = ctx.getChannel(); + + // Generate SASL response to server using Channel-local SASL client. + SaslNettyClient saslNettyClient = SaslNettyClientState.getSaslNettyClient + .get(channel); + if (saslNettyClient == null) { + throw new Exception("saslNettyClient was unexpectedly " + + "null for channel: " + channel); + } + + // examine the response message from server + if (event.getMessage() instanceof ControlMessage) { + ControlMessage msg = (ControlMessage) event.getMessage(); + if (msg == ControlMessage.SASL_COMPLETE_REQUEST) { + LOG.debug("Server has sent us the SaslComplete " + + "message. Allowing normal work to proceed."); + + if (!saslNettyClient.isComplete()) { + LOG.error("Server returned a Sasl-complete message, " + + "but as far as we can tell, we are not authenticated yet."); + throw new Exception("Server returned a " + + "Sasl-complete message, but as far as " + + "we can tell, we are not authenticated yet."); + } + ctx.getPipeline().remove(this); + // We call fireMessageReceived since the client is allowed to + // perform this request. The client's request will now proceed + // to the next pipeline component namely StormClientHandler. + Channels.fireMessageReceived(ctx, msg); + return; + } + } + SaslMessageToken saslTokenMessage = (SaslMessageToken) event + .getMessage(); + LOG.debug("Responding to server's token of length: " + + saslTokenMessage.getSaslToken().length); + + // Generate SASL response (but we only actually send the response if + // it's non-null. + byte[] responseToServer = saslNettyClient + .saslResponse(saslTokenMessage); + if (responseToServer == null) { + // If we generate a null response, then authentication has completed + // (if not, warn), and return without sending a response back to the + // server. + LOG.debug("Response to server is null: " + + "authentication should now be complete."); + if (!saslNettyClient.isComplete()) { + LOG.warn("Generated a null response, " + + "but authentication is not complete."); + } + return; + } else { + LOG.debug("Response to server token has length:" + + responseToServer.length); + } + // Construct a message containing the SASL response and send it to the + // server. + SaslMessageToken saslResponse = new SaslMessageToken(responseToServer); + channel.write(saslResponse); + } + + /** + * Load Storm Topology Token. + * + * @param conf + * Configuration + * @throws IOException + */ + private void loadTopologyToken() throws IOException { + topologyUser = (String) this.client.storm_conf + .get(Config.TOPOLOGY_NAME); + LOG.debug("SASL credentials is the storm user name: " + topologyUser); + token = topologyUser.getBytes(); + } +} \ No newline at end of file diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerAuthorizeHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerAuthorizeHandler.java new file mode 100644 index 00000000000..04cd66e3753 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerAuthorizeHandler.java @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.Channels; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelUpstreamHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Authorize or deny client requests based on existence and completeness of + * client's SASL authentication. + */ +public class SaslStormServerAuthorizeHandler extends SimpleChannelUpstreamHandler { + + private static final Logger LOG = LoggerFactory + .getLogger(SaslStormServerHandler.class); + + /** + * Constructor. + */ + public SaslStormServerAuthorizeHandler() { + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) { + Object msg = e.getMessage(); + if (msg == null) + return; + + Channel channel = ctx.getChannel(); + LOG.debug("messageReceived: Checking whether the client is authorized to send messages to the server "); + + // Authorize: client is allowed to doRequest() if and only if the client + // has successfully authenticated with this server. + SaslNettyServer saslNettyServer = SaslNettyServerState.getSaslNettyServer + .get(channel); + + if (saslNettyServer == null) { + LOG.warn("messageReceived: This client is *NOT* authorized to perform " + + "this action since there's no saslNettyServer to " + + "authenticate the client: " + + "refusing to perform requested action: " + msg); + return; + } + + if (!saslNettyServer.isComplete()) { + LOG.warn("messageReceived: This client is *NOT* authorized to perform " + + "this action because SASL authentication did not complete: " + + "refusing to perform requested action: " + msg); + // Return now *WITHOUT* sending upstream here, since client + // not authorized. + return; + } + + LOG.debug("messageReceived: authenticated client: " + + saslNettyServer.getUserName() + + " is authorized to do request " + "on server."); + + // We call fireMessageReceived since the client is allowed to perform + // this request. The client's request will now proceed to the next + // pipeline component. + Channels.fireMessageReceived(ctx, msg); + } +} \ No newline at end of file diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java new file mode 100644 index 00000000000..2e8bcacd2c3 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import java.io.IOException; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.Channels; +import org.jboss.netty.channel.ExceptionEvent; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelUpstreamHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +public class SaslStormServerHandler extends SimpleChannelUpstreamHandler { + + Server server; + /** Used for client or server's token to send or receive from each other. */ + private byte[] token; + private String topologyUser; + + private static final Logger LOG = LoggerFactory + .getLogger(SaslStormServerHandler.class); + + public SaslStormServerHandler(Server server) throws IOException { + this.server = server; + loadTopologyToken(); + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) + throws Exception { + Object msg = e.getMessage(); + if (msg == null) + return; + + Channel channel = ctx.getChannel(); + LOG.debug("messageReceived: Got " + msg.getClass()); + + if (msg instanceof ControlMessage + && ((ControlMessage) e.getMessage()) == ControlMessage.SASL_TOKEN_MESSAGE_REQUEST) { + // initialize server-side SASL functionality, if we haven't yet + // (in which case we are looking at the first SASL message from the + // client). + SaslNettyServer saslNettyServer = SaslNettyServerState.getSaslNettyServer + .get(channel); + if (saslNettyServer == null) { + LOG.debug("No saslNettyServer for " + channel + + " yet; creating now, with topology token: "); + try { + saslNettyServer = new SaslNettyServer(topologyUser); + } catch (IOException ioe) { + LOG.error("Error occurred while creating saslNettyServer on server " + + channel.getLocalAddress() + + " for client " + + channel.getRemoteAddress()); + throw new IOException(ioe); + } + + SaslNettyServerState.getSaslNettyServer.set(channel, + saslNettyServer); + } else { + LOG.debug("Found existing saslNettyServer on server:" + + channel.getLocalAddress() + " for client " + + channel.getRemoteAddress()); + } + + LOG.debug("processToken: With nettyServer: " + saslNettyServer + + " and token length: " + token.length); + + SaslMessageToken saslTokenMessageRequest = null; + saslTokenMessageRequest = new SaslMessageToken( + saslNettyServer.response(new byte[0])); + // Send response to client. + channel.write(saslTokenMessageRequest); + // do not send upstream to other handlers: no further action needs + // to be done for SASL_TOKEN_MESSAGE_REQUEST requests. + return; + } + + if (msg instanceof SaslMessageToken) { + // initialize server-side SASL functionality, if we haven't yet + // (in which case we are looking at the first SASL message from the + // client). + SaslNettyServer saslNettyServer = SaslNettyServerState.getSaslNettyServer + .get(channel); + if (saslNettyServer == null) { + if (saslNettyServer == null) { + throw new Exception("saslNettyServer was unexpectedly " + + "null for channel: " + channel); + } + } + SaslMessageToken saslTokenMessageRequest = new SaslMessageToken( + saslNettyServer.response(((SaslMessageToken) msg) + .getSaslToken())); + + // Send response to client. + channel.write(saslTokenMessageRequest); + + if (saslNettyServer.isComplete()) { + // If authentication of client is complete, we will also send a + // SASL-Complete message to the client. + LOG.debug("SASL authentication is complete for client with " + + "username: " + saslNettyServer.getUserName()); + channel.write(ControlMessage.SASL_COMPLETE_REQUEST); + LOG.debug("Removing SaslServerHandler from pipeline since SASL " + + "authentication is complete."); + ctx.getPipeline().remove(this); + } + return; + } else { + // Client should not be sending other-than-SASL messages before + // SaslServerHandler has removed itself from the pipeline. Such + // non-SASL requests will be denied by the Authorize channel handler + // (the next handler upstream in the server pipeline) if SASL + // authentication has not completed. + LOG.warn("Sending upstream an unexpected non-SASL message : " + + msg); + Channels.fireMessageReceived(ctx, msg); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { + server.closeChannel(e.getChannel()); + } + + /** + * Load Storm Topology Token. + * + * @param conf + * Configuration + * @throws IOException + */ + private void loadTopologyToken() throws IOException { + topologyUser = (String) this.server.storm_conf + .get(Config.TOPOLOGY_NAME); + LOG.debug("SASL credentials for the storm topology: " + topologyUser); + token = topologyUser.getBytes(); + } +} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java new file mode 100644 index 00000000000..0077cf37074 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.messaging.netty; + +import java.nio.charset.Charset; +import java.util.HashMap; +import java.util.Map; + +import javax.security.sasl.Sasl; + +import org.apache.commons.codec.binary.Base64; + +class SaslUtils { + public static final String AUTH_DIGEST_MD5 = "DIGEST-MD5"; + public static final String DEFAULT_REALM = "default"; + + static Map getSaslProps() { + Map props = new HashMap(); + props.put(Sasl.POLICY_NOPLAINTEXT, "true"); + return props; + } + + /** + * Encode a password as a base64-encoded char[] array. + * + * @param password + * as a byte array. + * @return password as a char array. + */ + static char[] encodePassword(byte[] password) { + return new String(Base64.encodeBase64(password), + Charset.defaultCharset()).toCharArray(); + } + + /** + * Encode a identifier as a base64-encoded char[] array. + * + * @param identifier + * as a byte array. + * @return identifier as a char array. + */ + static String encodeIdentifier(byte[] identifier) { + return new String(Base64.encodeBase64(identifier), + Charset.defaultCharset()); + } +} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java index 20a147df254..1b2590a4bf2 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java @@ -17,19 +17,6 @@ */ package backtype.storm.messaging.netty; -import backtype.storm.Config; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.Utils; -import org.jboss.netty.bootstrap.ServerBootstrap; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.jboss.netty.channel.group.ChannelGroup; -import org.jboss.netty.channel.group.DefaultChannelGroup; -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Arrays; @@ -41,6 +28,20 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadFactory; +import org.jboss.netty.bootstrap.ServerBootstrap; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.group.ChannelGroup; +import org.jboss.netty.channel.group.DefaultChannelGroup; +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.utils.Utils; + class Server implements IConnection { private static final Logger LOG = LoggerFactory.getLogger(Server.class); @SuppressWarnings("rawtypes") diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java index e6e8b3dec89..4fdaee90d2b 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java @@ -21,6 +21,8 @@ import org.jboss.netty.channel.ChannelPipelineFactory; import org.jboss.netty.channel.Channels; +import backtype.storm.Config; + class StormClientPipelineFactory implements ChannelPipelineFactory { private Client client; @@ -32,12 +34,24 @@ public ChannelPipeline getPipeline() throws Exception { // Create a default pipeline implementation. ChannelPipeline pipeline = Channels.pipeline(); - // Decoder - pipeline.addLast("decoder", new MessageDecoder()); - // Encoder - pipeline.addLast("encoder", new MessageEncoder()); - // business logic. - pipeline.addLast("handler", new StormClientErrorHandler(client.name())); + boolean isNettyAuth = (Boolean) this.client.storm_conf.get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION); + if(isNettyAuth) { + // Decoder + pipeline.addLast("decoder", new MessageDecoder()); + // Encoder + pipeline.addLast("encoder", new MessageEncoder()); + // Authenticate: Removed after authentication completes + pipeline.addLast("saslClientHandler", new SaslStormClientHandler(client)); + // business logic. + pipeline.addLast("handler", new StormClientErrorHandler(client.name())); + } else { + // Decoder + pipeline.addLast("decoder", new MessageDecoder()); + // Encoder + pipeline.addLast("encoder", new MessageEncoder()); + // business logic. + pipeline.addLast("handler", new StormClientErrorHandler(client.name())); + } return pipeline; } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java index df29ba88880..c2b4c53a4e0 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java @@ -21,6 +21,9 @@ import org.jboss.netty.channel.ChannelPipelineFactory; import org.jboss.netty.channel.Channels; +import backtype.storm.Config; +import backtype.storm.utils.Utils; + class StormServerPipelineFactory implements ChannelPipelineFactory { private Server server; @@ -33,13 +36,27 @@ public ChannelPipeline getPipeline() throws Exception { // Create a default pipeline implementation. ChannelPipeline pipeline = Channels.pipeline(); - // Decoder - pipeline.addLast("decoder", new MessageDecoder()); - // Encoder - pipeline.addLast("encoder", new MessageEncoder()); - // business logic. - pipeline.addLast("handler", new StormServerHandler(server)); - + boolean isNettyAuth = (Boolean) this.server.storm_conf.get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION); + if(isNettyAuth) { + // Decoder + pipeline.addLast("decoder", new MessageDecoder()); + // Encoder + pipeline.addLast("encoder", new MessageEncoder()); + // Authenticate: Removed after authentication completes + pipeline.addLast("saslServerHandler", new SaslStormServerHandler(server)); + // Authorize + pipeline.addLast("authorizeServerHandler", new SaslStormServerAuthorizeHandler()); + // business logic. + pipeline.addLast("handler", new StormServerHandler(server)); + } else { + // Decoder + pipeline.addLast("decoder", new MessageDecoder()); + // Encoder + pipeline.addLast("encoder", new MessageEncoder()); + // business logic. + pipeline.addLast("handler", new StormServerHandler(server)); + } + return pipeline; } } From 5b68b9301412ae766fddfc22320f3890240b1779 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Fri, 11 Jul 2014 11:50:07 -0700 Subject: [PATCH 32/61] STORM-346: added AutoHDFS class that will get hdfs delegation tokens on behalf of users, push it to workers and renew the delegation tokens automatically. INimbusCredentialPlugin is a plugin interfae that will be executed as part of nimbu's submit topology operation and fetch credential from nimbus process on behalf of user. AutoHDFS will renew the tokens on nimbus host as long as credential has more then 24 hours on renew cycle, if tokens are going to expire in next 24 hours then new tokens will be fetched and pushed to all workers. --- .../src/clj/backtype/storm/MockAutoCred.clj | 58 +++++++++++++++++++ .../src/clj/backtype/storm/daemon/nimbus.clj | 6 +- .../test/clj/backtype/storm/nimbus_test.clj | 37 +++++++++++- 3 files changed, 97 insertions(+), 4 deletions(-) create mode 100644 storm-core/src/clj/backtype/storm/MockAutoCred.clj diff --git a/storm-core/src/clj/backtype/storm/MockAutoCred.clj b/storm-core/src/clj/backtype/storm/MockAutoCred.clj new file mode 100644 index 00000000000..5e37528c692 --- /dev/null +++ b/storm-core/src/clj/backtype/storm/MockAutoCred.clj @@ -0,0 +1,58 @@ +;; 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. + +;;mock implementation of INimbusCredentialPlugin,IAutoCredentials and ICredentialsRenewer for testing only. +(ns backtype.storm.MockAutoCred + (:use [backtype.storm testing config]) + (:import [backtype.storm.security.INimbusCredentialPlugin] + [backtype.storm.security.auth ICredentialsRenewer]) + (:gen-class + :implements [backtype.storm.security.INimbusCredentialPlugin + backtype.storm.security.auth.IAutoCredentials + backtype.storm.security.auth.ICredentialsRenewer])) + +(def nimbus-cred-key "nimbusCredTestKey") +(def nimbus-cred-val "nimbusTestCred") +(def nimbus-cred-renew-val "renewedNimbusTestCred") +(def gateway-cred-key "gatewayCredTestKey") +(def gateway-cred-val "gatewayTestCred") +(def gateway-cred-renew-val "renewedGatewayTestCred") + +(defn -populateCredentials + ([this creds conf] + (.put creds nimbus-cred-key nimbus-cred-val)) + ([this creds] + (.put creds gateway-cred-key gateway-cred-val))) + +(defn -prepare + [this conf]) + +(defn -renew + [this cred conf] + (.put cred nimbus-cred-key nimbus-cred-renew-val) + (.put cred gateway-cred-key gateway-cred-renew-val)) + +(defn -populateSubject + [subject credentials] + (.add (.getPublicCredentials subject) (.get credentials nimbus-cred-key)) + (.add (.getPublicCredentials subject) (.get credentials gateway-cred-key))) + +(defn -updateSubject + [subject credentials] + (-populateSubject subject credentials)) + + + diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj index 3fe1a0a1ec6..5a326d11514 100644 --- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj @@ -950,7 +950,7 @@ (log-message "Renewing Creds For " id " with " renewer) (.renew renewer new-creds topology-conf)) (when-not (= orig-creds new-creds) - (.set-credentials! storm-cluster-state id new-creds) + (.set-credentials! storm-cluster-state id new-creds topology-conf) ))))))))) (defn validate-topology-size [topo-conf nimbus-conf topology] @@ -1049,8 +1049,8 @@ topology (normalize-topology total-storm-conf topology) nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins total-storm-conf) storm-cluster-state (:storm-cluster-state nimbus)] - (doseq [nimbus-autocred-plugin nimbus-autocred-plugins] - (.populateCredentials nimbus-autocred-plugin credentials total-storm-conf)) + (when credentials (doseq [nimbus-autocred-plugin nimbus-autocred-plugins] + (.populateCredentials nimbus-autocred-plugin credentials total-storm-conf))) (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 diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj index c902119fa57..d33fc108995 100644 --- a/storm-core/test/clj/backtype/storm/nimbus_test.clj +++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj @@ -20,7 +20,7 @@ (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter]) (:import [backtype.storm.scheduler INimbus]) (:import [backtype.storm.generated Credentials]) - (:use [backtype.storm bootstrap testing]) + (:use [backtype.storm bootstrap testing MockAutoCred]) (:use [backtype.storm.daemon common]) (:require [conjure.core]) (:use [conjure core]) @@ -35,6 +35,10 @@ (storm-task-info (from-json (.getTopologyConf nimbus storm-id))) reverse-map))) +(defn getCredentials [cluster storm-name] + (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)] + (.credentials (:storm-cluster-state cluster) storm-id nil))) + (defn storm-component->executor-info [cluster storm-name] (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name) nimbus (:nimbus cluster) @@ -228,6 +232,37 @@ node-id )))) + +(deftest test-auto-credentials + (with-simulated-time-local-cluster [cluster :supervisors 6 + :ports-per-supervisor 3 + :daemon-conf {SUPERVISOR-ENABLE false + TOPOLOGY-ACKER-EXECUTORS 0 + NIMBUS-CREDENTIAL-RENEW-FREQ-SECS 10 + NIMBUS-CREDENTIAL-RENEWERS (list "backtype.storm.MockAutoCred") + }] + (let [state (:storm-cluster-state cluster) + nimbus (:nimbus cluster) + topology-name "test-auto-cred-storm" + submitOptions (SubmitOptions. TopologyInitialStatus/INACTIVE) + - (.set_creds submitOptions (Credentials. (HashMap.))) + topology (thrift/mk-topology + {"1" (thrift/mk-spout-spec (TestPlannerSpout. false) :parallelism-hint 3)} + {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 4) + "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.))}) + _ (submit-local-topology-with-opts nimbus topology-name {TOPOLOGY-WORKERS 4 + NIMBUS-AUTO-CRED-PLUGINS (list "backtype.storm.MockAutoCred") + TOPOLOGY-AUTO-CREDENTIALS (list "backtype.storm.MockAutoCred") + } topology submitOptions) + credentials (getCredentials cluster topology-name)] + ; check that the credentials have nimbus auto generated cred + (is (= (.get credentials nimbus-cred-key) nimbus-cred-val)) + ;advance cluster time so the renewers can execute + (advance-cluster-time cluster 20) + ;check that renewed credentials replace the original credential. + (is (= (.get (getCredentials cluster topology-name) nimbus-cred-key) nimbus-cred-renew-val)) + (is (= (.get (getCredentials cluster topology-name) gateway-cred-key) gateway-cred-renew-val))))) + (deftest test-isolated-assignment (with-simulated-time-local-cluster [cluster :supervisors 6 :ports-per-supervisor 3 From 133c398ee9f5799fb3d702e27c8f83b969fc0341 Mon Sep 17 00:00:00 2001 From: Raghavendra Nandagopal Date: Thu, 24 Jul 2014 15:47:51 -0700 Subject: [PATCH 33/61] STORM-348: Netty SASL Authentication --- .../messaging/netty/SaslNettyClient.java | 9 ++--- .../messaging/netty/SaslNettyServer.java | 37 +++++++++---------- .../netty/SaslStormClientHandler.java | 24 +++++------- .../netty/SaslStormServerHandler.java | 26 ++++++------- .../storm/messaging/netty/SaslUtils.java | 14 +++++++ 5 files changed, 56 insertions(+), 54 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java index a4f1b5eb6d8..fedcfff5e7f 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java @@ -50,7 +50,7 @@ public class SaslNettyClient { /** * Create a SaslNettyClient for authentication with servers. */ - public SaslNettyClient(String topologyUser) { + public SaslNettyClient(String topologyName, byte[] token) { try { LOG.debug("SaslNettyClient: Creating SASL " + SaslUtils.AUTH_DIGEST_MD5 @@ -59,7 +59,7 @@ public SaslNettyClient(String topologyUser) { saslClient = Sasl.createSaslClient( new String[] { SaslUtils.AUTH_DIGEST_MD5 }, null, null, SaslUtils.DEFAULT_REALM, SaslUtils.getSaslProps(), - new SaslClientCallbackHandler(topologyUser)); + new SaslClientCallbackHandler(topologyName, token)); } catch (IOException e) { LOG.error("SaslNettyClient: Could not obtain topology token for Netty " @@ -107,11 +107,10 @@ private static class SaslClientCallbackHandler implements CallbackHandler { * * @param topologyToken */ - public SaslClientCallbackHandler(String topologyToken) { + public SaslClientCallbackHandler(String topologyToken, byte[] token) { this.userName = SaslUtils .encodeIdentifier(topologyToken.getBytes()); - this.userPassword = SaslUtils.encodePassword(topologyToken - .getBytes()); + this.userPassword = SaslUtils.encodePassword(token); } /** diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java index 1178bd65530..2cb47d90b9a 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java @@ -44,14 +44,14 @@ class SaslNettyServer { private SaslServer saslServer; - SaslNettyServer(String topologyToken) throws IOException { - LOG.debug("SaslNettyServer: Topology token is: " + topologyToken + SaslNettyServer(String topologyName, byte[] token) throws IOException { + LOG.debug("SaslNettyServer: Topology token is: " + topologyName + " with authmethod " + SaslUtils.AUTH_DIGEST_MD5); try { SaslDigestCallbackHandler ch = new SaslNettyServer.SaslDigestCallbackHandler( - topologyToken); + topologyName, token); saslServer = Sasl.createSaslServer(SaslUtils.AUTH_DIGEST_MD5, null, SaslUtils.DEFAULT_REALM, SaslUtils.getSaslProps(), ch); @@ -70,18 +70,18 @@ public String getUserName() { return saslServer.getAuthorizationID(); } - - /** CallbackHandler for SASL DIGEST-MD5 mechanism */ public static class SaslDigestCallbackHandler implements CallbackHandler { /** Used to authenticate the clients */ - private String topologyToken; + private byte[] userPassword; + private String userName; - public SaslDigestCallbackHandler(String topologyToken) { + public SaslDigestCallbackHandler(String topologyName, byte[] token) { LOG.debug("SaslDigestCallback: Creating SaslDigestCallback handler " - + "with topology token: " + topologyToken); - this.topologyToken = topologyToken; + + "with topology token: " + topologyName); + this.userName = topologyName; + this.userPassword = token; } @Override @@ -105,19 +105,19 @@ public void handle(Callback[] callbacks) throws IOException, "handle: Unrecognized SASL DIGEST-MD5 Callback"); } } - - if(nc!=null) { + + if (nc != null) { LOG.debug("handle: SASL server DIGEST-MD5 callback: setting " - + "username for client: " + topologyToken); + + "username for client: " + userName); - nc.setName(topologyToken); + nc.setName(userName); } if (pc != null) { - char[] password = SaslUtils.encodePassword(topologyToken.getBytes()); + char[] password = SaslUtils.encodePassword(userPassword); LOG.debug("handle: SASL server DIGEST-MD5 callback: setting " - + "password for client: " + topologyToken); + + "password for client: " + userPassword); pc.setPassword(password); } @@ -133,11 +133,8 @@ public void handle(Callback[] callbacks) throws IOException, } if (ac.isAuthorized()) { - if (LOG.isDebugEnabled()) { - String username = topologyToken; - LOG.debug("handle: SASL server DIGEST-MD5 callback: setting " - + "canonicalized client ID: " + username); - } + LOG.debug("handle: SASL server DIGEST-MD5 callback: setting " + + "canonicalized client ID: " + userName); ac.setAuthorizedID(authzid); } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java index ca38c962978..59c4abd51e2 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java @@ -38,12 +38,12 @@ public class SaslStormClientHandler extends SimpleChannelUpstreamHandler { long start_time; /** Used for client or server's token to send or receive from each other. */ private byte[] token; - private String topologyUser; + private String topologyName; public SaslStormClientHandler(Client client) throws IOException { this.client = client; start_time = System.currentTimeMillis(); - loadTopologyToken(); + getSASLCredentials(); } @Override @@ -62,7 +62,7 @@ public void channelConnected(ChannelHandlerContext ctx, if (saslNettyClient == null) { LOG.debug("Creating saslNettyClient now " + "for channel: " + channel); - saslNettyClient = new SaslNettyClient(topologyUser); + saslNettyClient = new SaslNettyClient(topologyName, token); SaslNettyClientState.getSaslNettyClient.set(channel, saslNettyClient); } @@ -143,17 +143,13 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) channel.write(saslResponse); } - /** - * Load Storm Topology Token. - * - * @param conf - * Configuration - * @throws IOException - */ - private void loadTopologyToken() throws IOException { - topologyUser = (String) this.client.storm_conf + private void getSASLCredentials() throws IOException { + topologyName = (String) this.client.storm_conf .get(Config.TOPOLOGY_NAME); - LOG.debug("SASL credentials is the storm user name: " + topologyUser); - token = topologyUser.getBytes(); + String secretKey = SaslUtils.getSecretKey(this.client.storm_conf); + if(secretKey!=null) { + token = secretKey.getBytes(); + } + LOG.debug("SASL credentials for storm topology "+topologyName+ " is "+secretKey); } } \ No newline at end of file diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java index 2e8bcacd2c3..d06e96096e7 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java @@ -35,14 +35,14 @@ public class SaslStormServerHandler extends SimpleChannelUpstreamHandler { Server server; /** Used for client or server's token to send or receive from each other. */ private byte[] token; - private String topologyUser; + private String topologyName; private static final Logger LOG = LoggerFactory .getLogger(SaslStormServerHandler.class); public SaslStormServerHandler(Server server) throws IOException { this.server = server; - loadTopologyToken(); + getSASLCredentials(); } @Override @@ -53,7 +53,6 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) return; Channel channel = ctx.getChannel(); - LOG.debug("messageReceived: Got " + msg.getClass()); if (msg instanceof ControlMessage && ((ControlMessage) e.getMessage()) == ControlMessage.SASL_TOKEN_MESSAGE_REQUEST) { @@ -66,7 +65,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) LOG.debug("No saslNettyServer for " + channel + " yet; creating now, with topology token: "); try { - saslNettyServer = new SaslNettyServer(topologyUser); + saslNettyServer = new SaslNettyServer(topologyName, token); } catch (IOException ioe) { LOG.error("Error occurred while creating saslNettyServer on server " + channel.getLocalAddress() @@ -143,17 +142,14 @@ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { server.closeChannel(e.getChannel()); } - /** - * Load Storm Topology Token. - * - * @param conf - * Configuration - * @throws IOException - */ - private void loadTopologyToken() throws IOException { - topologyUser = (String) this.server.storm_conf + private void getSASLCredentials() throws IOException { + topologyName = (String) this.server.storm_conf .get(Config.TOPOLOGY_NAME); - LOG.debug("SASL credentials for the storm topology: " + topologyUser); - token = topologyUser.getBytes(); + String secretKey = SaslUtils.getSecretKey(this.server.storm_conf); + if (secretKey != null) { + token = secretKey.getBytes(); + } + LOG.debug("SASL credentials for storm topology " + topologyName + + " is " + secretKey); } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java index 0077cf37074..0f962338beb 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java @@ -25,6 +25,8 @@ import org.apache.commons.codec.binary.Base64; +import backtype.storm.Config; + class SaslUtils { public static final String AUTH_DIGEST_MD5 = "DIGEST-MD5"; public static final String DEFAULT_REALM = "default"; @@ -58,4 +60,16 @@ static String encodeIdentifier(byte[] identifier) { return new String(Base64.encodeBase64(identifier), Charset.defaultCharset()); } + + static String getSecretKey(Map conf) { + if (conf == null || conf.isEmpty()) + return null; + + String secretPayLoad = (String) conf + .get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD); + + return secretPayLoad; + } + + } From 63e90822056daf0a41c0778c0984b7371459a3f2 Mon Sep 17 00:00:00 2001 From: Raghavendra Nandagopal Date: Thu, 24 Jul 2014 15:57:31 -0700 Subject: [PATCH 34/61] STORM-348: Netty SASL Authentication --- conf/defaults.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 7f17054e0eb..ee66717010f 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -153,6 +153,9 @@ storm.messaging.netty.transfer.batch.size: 262144 # We check with this interval that whether the Netty channel is writable and try to write pending messages if it is. storm.messaging.netty.flush.check.interval.ms: 10 +# By default, the Netty SASL authentication is set to false. Users can override and set it true for a specific topology. +storm.messaging.netty.authentication: false + ### topology.* configs are for specific executing storms topology.enable.message.timeouts: true topology.debug: false From 1fedcd5b53649ba1f404429d6fa3c9cc6c296047 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Fri, 11 Jul 2014 11:50:07 -0700 Subject: [PATCH 35/61] STORM-346: added AutoHDFS class that will get hdfs delegation tokens on behalf of users, push it to workers and renew the delegation tokens automatically. INimbusCredentialPlugin is a plugin interfae that will be executed as part of nimbu's submit topology operation and fetch credential from nimbus process on behalf of user. AutoHDFS will renew the tokens on nimbus host as long as credential has more then 24 hours on renew cycle, if tokens are going to expire in next 24 hours then new tokens will be fetched and pushed to all workers. --- SECURITY.md | 12 ++++ .../src/clj/backtype/storm/daemon/nimbus.clj | 12 ++-- storm-core/src/jvm/backtype/storm/Config.java | 16 +++-- .../security/INimbusCredentialPlugin.java | 10 ++- .../storm/security/auth/AuthUtils.java | 9 +-- .../auth/{kerberos => hadoop}/AutoHDFS.java | 70 +++++++++++-------- .../test/clj/backtype/storm/nimbus_test.clj | 2 +- 7 files changed, 84 insertions(+), 47 deletions(-) rename storm-core/src/jvm/backtype/storm/security/auth/{kerberos => hadoop}/AutoHDFS.java (81%) diff --git a/SECURITY.md b/SECURITY.md index 882954a768f..6413e2387d7 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -316,6 +316,18 @@ On a kerberos secure cluster they should be set by default to point to backtype. 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. +#### Automatic HDFS credential push and renewal +If your topology is going to use secure HDFS , your administrator can configure nimbus to automatically get delegation tokens on behalf of the topology submitter user. The nimbus need to start with +nimbus.autocredential.plugins.classes=backtype.storm.security.auth.hadoop.AutoHDFS and nimbus.credential.renewers.classes=backtype.storm.security.auth.hadoop.AutoHDFS. Your topology configuration +should have topology.auto-credentials=backtype.storm.security.auth.hadoop.AutoHDFS so workers can automatically get the credentials in the Subject. + +If nimbus did not have the above configuration you need to add it and then restart it. Ensure all the hadoop configuration files are present in the nimbus' classpath. Please read more about setting up +secure hadoop on http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/SecureMode.html. + +You also need to ensure that nimbus user is allowed to act as a super user and get delegation tokens on behalf of other users. To achieve this you need to follow configuration directions listed on this link +http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/Superusers.html. + + ### 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. diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj index 5a326d11514..431b74d8f2b 100644 --- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj @@ -14,7 +14,8 @@ ;; See the License for the specific language governing permissions and ;; limitations under the License. (ns backtype.storm.daemon.nimbus - (:import [java.nio ByteBuffer]) + (:import [java.nio ByteBuffer] + [java.util Collections]) (:import [java.io FileNotFoundException]) (:import [java.nio.channels Channels WritableByteChannel]) (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils]) @@ -86,6 +87,7 @@ :scheduler (mk-scheduler conf inimbus) :id->sched-status (atom {}) :cred-renewers (AuthUtils/GetCredentialRenewers conf) + :nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf) })) (defn inbox [nimbus] @@ -948,7 +950,7 @@ (let [new-creds (HashMap. orig-creds)] (doseq [renewer renewers] (log-message "Renewing Creds For " id " with " renewer) - (.renew renewer new-creds topology-conf)) + (.renew renewer new-creds (Collections/unmodifiableMap topology-conf))) (when-not (= orig-creds new-creds) (.set-credentials! storm-cluster-state id new-creds topology-conf) ))))))))) @@ -1047,10 +1049,10 @@ (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) - nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins total-storm-conf) + storm-cluster-state (:storm-cluster-state nimbus)] - (when credentials (doseq [nimbus-autocred-plugin nimbus-autocred-plugins] - (.populateCredentials nimbus-autocred-plugin credentials total-storm-conf))) + (when credentials (doseq [nimbus-autocred-plugin (:nimbus-autocred-plugins nimbus)] + (.populateCredentials nimbus-autocred-plugin credentials (Collections/unmodifiableMap storm-conf)))) (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 diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 3c95f8a87ff..03bc438d221 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -17,10 +17,8 @@ */ package backtype.storm; -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; @@ -1196,11 +1194,19 @@ public class Config extends HashMap { /** * HDFS information, used to get the delegation token on behalf of the topology - * submitter user and renew the tokens. see {@link backtype.storm.security.auth.kerberos.AutoHDFS} + * submitter user and renew the tokens. see {@link backtype.storm.security.auth.hadoop.AutoHDFS} * kerberos principal name with realm should be provided. */ - public static final Object HDFS_PRINCIPAL = "topology.hdfs.user"; - public static final Object HDFS_PRINCIPAL_SCHEMA = String.class; + public static final Object TOPOLOGY_HDFS_PRINCIPAL = "topology.hdfs.user"; + public static final Object TOPOLOGY_HDFS_PRINCIPAL_SCHEMA = String.class; + + /** + * The HDFS URI to be used by AutoHDFS.java to grab the delegation token on topology + * submitter user's behalf by the nimbus. If this is not provided the default URI provided + * in the hdfs configuration files will be used. + */ + public static final Object TOPOLOGY_HDFS_URI = "topology.hdfs.uri"; + public static final Object TOPOLOGY_HDFS_URI_SCHEMA = String.class; public static void setDebug(Map conf, boolean isOn) { conf.put(Config.TOPOLOGY_DEBUG, isOn); diff --git a/storm-core/src/jvm/backtype/storm/security/INimbusCredentialPlugin.java b/storm-core/src/jvm/backtype/storm/security/INimbusCredentialPlugin.java index 4c56863d264..9670045eb76 100644 --- a/storm-core/src/jvm/backtype/storm/security/INimbusCredentialPlugin.java +++ b/storm-core/src/jvm/backtype/storm/security/INimbusCredentialPlugin.java @@ -17,6 +17,8 @@ */ package backtype.storm.security; +import backtype.storm.daemon.Shutdownable; + import java.util.Map; /** @@ -24,7 +26,13 @@ * during submit topology option. User can specify a list of implementation using config key * nimbus.autocredential.plugins.classes. */ -public interface INimbusCredentialPlugin { +public interface INimbusCredentialPlugin extends Shutdownable { + + /** + * this method will be called when nimbus initializes. + * @param conf + */ + void prepare(Map conf); /** * Method that will be called on nimbus as part of submit topology. This plugin will be called diff --git a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java index fc06698e075..4240769e590 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java @@ -111,17 +111,18 @@ public static Collection GetCredentialRenewers(Map conf) { } /** - * Get all the Nimbus Auto cred plugins that users want to use. - * @param topologyConf topologyConfiguration to use. + * Get all the Nimbus Auto cred plugins. + * @param conf nimbus configuration to use. * @return nimbus auto credential plugins. */ - public static Collection getNimbusAutoCredPlugins(Map topologyConf) { + public static Collection getNimbusAutoCredPlugins(Map conf) { try { Set ret = new HashSet(); - Collection clazzes = (Collection)topologyConf.get(Config.NIMBUS_AUTO_CRED_PLUGINS); + Collection clazzes = (Collection)conf.get(Config.NIMBUS_AUTO_CRED_PLUGINS); if (clazzes != null) { for (String clazz : clazzes) { INimbusCredentialPlugin inst = (INimbusCredentialPlugin)Class.forName(clazz).newInstance(); + inst.prepare(conf); ret.add(inst); } } diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java b/storm-core/src/jvm/backtype/storm/security/auth/hadoop/AutoHDFS.java similarity index 81% rename from storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java rename to storm-core/src/jvm/backtype/storm/security/auth/hadoop/AutoHDFS.java index 7685fb10b51..f61fb25568f 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoHDFS.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/hadoop/AutoHDFS.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package backtype.storm.security.auth.kerberos; +package backtype.storm.security.auth.hadoop; import backtype.storm.Config; import backtype.storm.security.INimbusCredentialPlugin; @@ -43,32 +43,31 @@ public class AutoHDFS implements IAutoCredentials, ICredentialsRenewer, INimbusC public static final String HDFS_CREDENTIALS = "HDFS_CREDENTIALS"; public void prepare(Map conf) { - LOG.debug("no op."); + //no op. } @SuppressWarnings("unchecked") private byte[] getHDFSCredsWithDelegationToken(Map conf) throws Exception { try { - /** - * What we want to do is following: - * if(UserGroupInformation.isSecurityEnabled) { - * FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); - * UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); - * UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(topologySubmitterUser, ugi); - * Credentials credential= proxyUser.getCredentials(); - * fs.addDelegationToken(hdfsUser, credential); - * } - * and then return the credential object as a bytearray. - * - * Following are the minimum set of configuration that needs to be set, users should have hdfs-site.xml - * and core-site.xml in the class path which should set these configuration. - * configuration.set("hadoop.security.authentication", "KERBEROS"); - * configuration.set("dfs.namenode.kerberos.principal", - * "hdfs/zookeeper.witzend.com@WITZEND.COM"); - * configuration.set("hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002"); - * anf the ticket cache must have the hdfs user's creds. - */ + // What we want to do is following: + // if(UserGroupInformation.isSecurityEnabled) { + // FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); + // UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + // UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(topologySubmitterUser, ugi); + // Credentials credential= proxyUser.getCredentials(); + // fs.addDelegationToken(hdfsUser, credential); + // } + // and then return the credential object as a bytearray. + // + // Following are the minimum set of configuration that needs to be set, users should have hdfs-site.xml + // and core-site.xml in the class path which should set these configuration. + // configuration.set("hadoop.security.authentication", "KERBEROS"); + // configuration.set("dfs.namenode.kerberos.principal", + // "hdfs/zookeeper.witzend.com@WITZEND.COM"); + // configuration.set("hadoop.security.kerberos.ticket.cache.path", "/tmp/krb5cc_1002"); + // and the ticket cache must have the hdfs user's creds. + Class configurationClass = Class.forName("org.apache.hadoop.conf.Configuration"); Object configuration = configurationClass.newInstance(); @@ -79,13 +78,15 @@ private byte[] getHDFSCredsWithDelegationToken(Map conf) throws Exception { if(isSecurityEnabled) { final String topologySubmitterUser = (String) conf.get(Config.TOPOLOGY_SUBMITTER_USER); - final String hdfsUser = (String) conf.get(Config.HDFS_PRINCIPAL); + final String hdfsUser = (String) conf.get(Config.TOPOLOGY_HDFS_PRINCIPAL); //FileSystem fs = FileSystem.get(nameNodeURI, configuration, topologySubmitterUser); Class fileSystemClass = Class.forName("org.apache.hadoop.fs.FileSystem"); - Object defaultNameNodeURI = fileSystemClass.getMethod("getDefaultUri", configurationClass).invoke(null, configuration); + + Object nameNodeURI = conf.containsKey(Config.TOPOLOGY_HDFS_URI) ? conf.get(Config.TOPOLOGY_HDFS_URI) + : fileSystemClass.getMethod("getDefaultUri", configurationClass).invoke(null, configuration); Method getMethod = fileSystemClass.getMethod("get", URI.class, configurationClass, String.class); - Object fileSystem = getMethod.invoke(null, defaultNameNodeURI, configuration, topologySubmitterUser); + Object fileSystem = getMethod.invoke(null, nameNodeURI, configuration, topologySubmitterUser); //UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); Method getCurrentUserMethod = ugiClass.getMethod("getCurrentUser"); @@ -124,19 +125,21 @@ private byte[] getHDFSCredsWithDelegationToken(Map conf) throws Exception { @Override public void populateCredentials(Map credentials, Map conf) { - try { - credentials.put(HDFS_CREDENTIALS, DatatypeConverter.printBase64Binary( getHDFSCredsWithDelegationToken(conf))); - } catch (Exception e) { - LOG.warn("Could not populate HDFS credentials.", e); + if(conf.containsKey(Config.TOPOLOGY_HDFS_PRINCIPAL)) { + try { + credentials.put(HDFS_CREDENTIALS, DatatypeConverter.printBase64Binary(getHDFSCredsWithDelegationToken(conf))); + } catch (Exception e) { + LOG.warn("Could not populate HDFS credentials.", e); + } } } @Override public void populateCredentials(Map credentials) { - LOG.debug("populateCredentials is a noop, nimbus should populate the crdes."); + //no op. } - /** + /* * * @param credentials map with creds. * @return instance of org.apache.hadoop.security.Credentials, if the Map has HDFS_CREDENTIALS. @@ -234,7 +237,7 @@ public void renew(Map credentials, Map topologyConf) { public static void main(String[] args) throws Exception { Map conf = new java.util.HashMap(); conf.put(Config.TOPOLOGY_SUBMITTER_PRINCIPAL, args[0]); //with realm e.g. storm@WITZEND.COM - conf.put(Config.HDFS_PRINCIPAL, args[1]); //with realm e.g. hdfs@WITZEND.COM + conf.put(Config.TOPOLOGY_HDFS_PRINCIPAL, args[1]); //with realm e.g. hdfs@WITZEND.COM AutoHDFS autoHDFS = new AutoHDFS(); autoHDFS.prepare(conf); @@ -250,5 +253,10 @@ public static void main(String[] args) throws Exception { autoHDFS.renew(creds, conf); LOG.info("renewed credentials", AutoHDFS.getHDFSCredential(creds)); } + + @Override + public void shutdown() { + + } } diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj index d33fc108995..efdad25f8c0 100644 --- a/storm-core/test/clj/backtype/storm/nimbus_test.clj +++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj @@ -240,6 +240,7 @@ TOPOLOGY-ACKER-EXECUTORS 0 NIMBUS-CREDENTIAL-RENEW-FREQ-SECS 10 NIMBUS-CREDENTIAL-RENEWERS (list "backtype.storm.MockAutoCred") + NIMBUS-AUTO-CRED-PLUGINS (list "backtype.storm.MockAutoCred") }] (let [state (:storm-cluster-state cluster) nimbus (:nimbus cluster) @@ -251,7 +252,6 @@ {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 4) "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.))}) _ (submit-local-topology-with-opts nimbus topology-name {TOPOLOGY-WORKERS 4 - NIMBUS-AUTO-CRED-PLUGINS (list "backtype.storm.MockAutoCred") TOPOLOGY-AUTO-CREDENTIALS (list "backtype.storm.MockAutoCred") } topology submitOptions) credentials (getCredentials cluster topology-name)] From fb882ca18f124c7f6b65b7edeb2c0a652e254c3a Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 28 Jul 2014 12:51:13 -0700 Subject: [PATCH 36/61] STORM-347. (Security) authentication should allow for groups not just users. Added Unit test for ShellBaasedGroupsMapping. --- conf/defaults.yaml | 2 +- ...roupsMapping.java => ShellBasedGroupsMapping.java} | 4 ++-- .../clj/backtype/storm/security/auth/auth_test.clj | 11 ++++++++++- 3 files changed, 13 insertions(+), 4 deletions(-) rename storm-core/src/jvm/backtype/storm/security/auth/{ShellBasedUnixGroupsMapping.java => ShellBasedGroupsMapping.java} (97%) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index b0ebb4dfe67..d4283a42394 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -38,7 +38,7 @@ 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.group.mapping.service: "backtype.storm.security.auth.ShellBasedUnixGroupsMapping" +storm.group.mapping.service: "backtype.storm.security.auth.ShellBasedGroupsMapping" storm.messaging.transport: "backtype.storm.messaging.netty.Context" storm.nimbus.retry.times: 5 storm.nimbus.retry.interval.millis: 2000 diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java similarity index 97% rename from storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java rename to storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java index 438b9386636..62a4c7e74c3 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedUnixGroupsMapping.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java @@ -32,10 +32,10 @@ import org.slf4j.LoggerFactory; -public class ShellBasedUnixGroupsMapping implements +public class ShellBasedGroupsMapping implements IGroupMappingServiceProvider { - public static Logger LOG = LoggerFactory.getLogger(ShellBasedUnixGroupsMapping.class); + public static Logger LOG = LoggerFactory.getLogger(ShellBasedGroupsMapping.class); public TimeCacheMap> cachedGroups; /** diff --git a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj index 6fdd485c039..12411e795eb 100644 --- a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj +++ b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj @@ -26,7 +26,7 @@ (:import [backtype.storm.generated AuthorizationException]) (:import [backtype.storm.utils NimbusClient]) (:import [backtype.storm.security.auth.authorizer SimpleWhitelistAuthorizer SimpleACLAuthorizer]) - (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient + (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient ShellBasedGroupsMapping ReqContext SimpleTransportPlugin KerberosPrincipalToLocal ThriftConnectionType]) (:use [backtype.storm bootstrap util]) (:use [backtype.storm.daemon common]) @@ -278,6 +278,15 @@ (is (= false (.permit authorizer (ReqContext. supervisor-user) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]}))) )) +(deftest shell-based-groups-mapping-test + (let [cluster-conf (merge (read-storm-config)) + groups (ShellBasedGroupsMapping. ) + user-name (System/getProperty "user.name")] + (.prepare groups cluster-conf) + (>= 0 (.size (.getGroups groups user-name))) + (>= 0 (.size (.getGroups groups "userDoesNotExist"))) + (>= 0 (.size (.getGroups groups nil))))) + (deftest simple-acl-same-user-auth-test (let [cluster-conf (merge (read-storm-config) {NIMBUS-ADMINS ["admin"] From 3bce04ceab9abfd6e6da211cb5a792d9df6c96c6 Mon Sep 17 00:00:00 2001 From: Raghavendra Nandagopal Date: Mon, 28 Jul 2014 16:28:45 -0700 Subject: [PATCH 37/61] STORM-348: Netty SASL Authentication --- .../storm/messaging/netty/Client.java | 2 +- .../messaging/netty/SaslMessageToken.java | 123 +++++---- .../messaging/netty/SaslNettyClient.java | 230 ++++++++--------- .../netty/SaslStormClientHandler.java | 243 +++++++++--------- .../netty/SaslStormServerHandler.java | 240 ++++++++--------- .../storm/messaging/netty/SaslUtils.java | 75 +++--- .../netty/StormClientPipelineFactory.java | 32 +-- .../netty/StormServerPipelineFactory.java | 43 ++-- 8 files changed, 490 insertions(+), 498 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 64a1757ba28..0cf18095ebe 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -62,7 +62,7 @@ public class Client implements IConnection { Map storm_conf; - MessageBatch messageBatch = null; + private MessageBatch messageBatch = null; private AtomicLong flushCheckTimer; private int flushCheckInterval; private ScheduledExecutorService scheduler; diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java index 8383d2c0d55..d0d3ca11401 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java @@ -27,74 +27,73 @@ * Send and receive SASL tokens. */ public class SaslMessageToken { - /** Class logger */ - private static final Logger LOG = LoggerFactory - .getLogger(SaslMessageToken.class); + /** Class logger */ + private static final Logger LOG = LoggerFactory + .getLogger(SaslMessageToken.class); - /** Used for client or server's token to send or receive from each other. */ - private byte[] token; + /** Used for client or server's token to send or receive from each other. */ + private byte[] token; - /** - * Constructor used for reflection only. - */ - public SaslMessageToken() { - } + /** + * Constructor used for reflection only. + */ + public SaslMessageToken() { + } - /** - * Constructor used to send request. - * - * @param token - * the SASL token, generated by a SaslClient or SaslServer. - */ - public SaslMessageToken(byte[] token) { - this.token = token; - } + /** + * Constructor used to send request. + * + * @param token + * the SASL token, generated by a SaslClient or SaslServer. + */ + public SaslMessageToken(byte[] token) { + this.token = token; + } - /** - * Read accessor for SASL token - * - * @return saslToken SASL token - */ - public byte[] getSaslToken() { - return token; - } + /** + * Read accessor for SASL token + * + * @return saslToken SASL token + */ + public byte[] getSaslToken() { + return token; + } - /** - * Write accessor for SASL token - * - * @param token - * SASL token - */ - public void setSaslToken(byte[] token) { - this.token = token; - } + /** + * Write accessor for SASL token + * + * @param token + * SASL token + */ + public void setSaslToken(byte[] token) { + this.token = token; + } - int encodeLength() { - return 2+4+token.length; - } + int encodeLength() { + return 2 + 4 + token.length; + } - /** - * encode the current SaslToken Message into a channel buffer - * SaslTokenMessageRequest is encoded as: - * identifier .... short(2) always it is -500 - * payload length .... int - * payload .... byte[] - * @throws Exception - */ - ChannelBuffer buffer() throws Exception { - ChannelBufferOutputStream bout = new ChannelBufferOutputStream( - ChannelBuffers.directBuffer(encodeLength())); - short identifier = -500; - int payload_len = 0; + /** + * encode the current SaslToken Message into a channel buffer + * SaslTokenMessageRequest is encoded as: identifier .... short(2) always it + * is -500 payload length .... int payload .... byte[] + * + * @throws Exception + */ + ChannelBuffer buffer() throws Exception { + ChannelBufferOutputStream bout = new ChannelBufferOutputStream( + ChannelBuffers.directBuffer(encodeLength())); + short identifier = -500; + int payload_len = 0; if (token != null) - payload_len = token.length; - - bout.writeShort((short)identifier); - bout.writeInt((int)payload_len); - if(payload_len>0) { - bout.write(token); - } - bout.close(); - return bout.buffer(); - } + payload_len = token.length; + + bout.writeShort((short) identifier); + bout.writeInt((int) payload_len); + if (payload_len > 0) { + bout.write(token); + } + bout.close(); + return bout.buffer(); + } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java index fedcfff5e7f..023e95060d2 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java @@ -38,129 +38,129 @@ */ public class SaslNettyClient { - private static final Logger LOG = LoggerFactory - .getLogger(SaslNettyClient.class); + private static final Logger LOG = LoggerFactory + .getLogger(SaslNettyClient.class); - /** - * Used to respond to server's counterpart, SaslServer with SASL tokens - * represented as byte arrays. - */ - private SaslClient saslClient; + /** + * Used to respond to server's counterpart, SaslServer with SASL tokens + * represented as byte arrays. + */ + private SaslClient saslClient; - /** - * Create a SaslNettyClient for authentication with servers. - */ - public SaslNettyClient(String topologyName, byte[] token) { - try { - LOG.debug("SaslNettyClient: Creating SASL " - + SaslUtils.AUTH_DIGEST_MD5 - + " client to authenticate to server "); + /** + * Create a SaslNettyClient for authentication with servers. + */ + public SaslNettyClient(String topologyName, byte[] token) { + try { + LOG.debug("SaslNettyClient: Creating SASL " + + SaslUtils.AUTH_DIGEST_MD5 + + " client to authenticate to server "); - saslClient = Sasl.createSaslClient( - new String[] { SaslUtils.AUTH_DIGEST_MD5 }, null, null, - SaslUtils.DEFAULT_REALM, SaslUtils.getSaslProps(), - new SaslClientCallbackHandler(topologyName, token)); + saslClient = Sasl.createSaslClient( + new String[] { SaslUtils.AUTH_DIGEST_MD5 }, null, null, + SaslUtils.DEFAULT_REALM, SaslUtils.getSaslProps(), + new SaslClientCallbackHandler(topologyName, token)); - } catch (IOException e) { - LOG.error("SaslNettyClient: Could not obtain topology token for Netty " - + "Client to use to authenticate with a Netty Server."); - saslClient = null; - } - } + } catch (IOException e) { + LOG.error("SaslNettyClient: Could not obtain topology token for Netty " + + "Client to use to authenticate with a Netty Server."); + saslClient = null; + } + } - public boolean isComplete() { - return saslClient.isComplete(); - } + public boolean isComplete() { + return saslClient.isComplete(); + } - /** - * Respond to server's SASL token. - * - * @param saslTokenMessage - * contains server's SASL token - * @return client's response SASL token - */ - public byte[] saslResponse(SaslMessageToken saslTokenMessage) { - try { - byte[] retval = saslClient.evaluateChallenge(saslTokenMessage - .getSaslToken()); - return retval; - } catch (SaslException e) { - LOG.error( - "saslResponse: Failed to respond to SASL server's token:", - e); - return null; - } - } + /** + * Respond to server's SASL token. + * + * @param saslTokenMessage + * contains server's SASL token + * @return client's response SASL token + */ + public byte[] saslResponse(SaslMessageToken saslTokenMessage) { + try { + byte[] retval = saslClient.evaluateChallenge(saslTokenMessage + .getSaslToken()); + return retval; + } catch (SaslException e) { + LOG.error( + "saslResponse: Failed to respond to SASL server's token:", + e); + return null; + } + } - /** - * Implementation of javax.security.auth.callback.CallbackHandler that works - * with Storm topology tokens. - */ - private static class SaslClientCallbackHandler implements CallbackHandler { - /** Generated username contained in TopologyToken */ - private final String userName; - /** Generated password contained in TopologyToken */ - private final char[] userPassword; + /** + * Implementation of javax.security.auth.callback.CallbackHandler that works + * with Storm topology tokens. + */ + private static class SaslClientCallbackHandler implements CallbackHandler { + /** Generated username contained in TopologyToken */ + private final String userName; + /** Generated password contained in TopologyToken */ + private final char[] userPassword; - /** - * Set private members using topology token. - * - * @param topologyToken - */ - public SaslClientCallbackHandler(String topologyToken, byte[] token) { - this.userName = SaslUtils - .encodeIdentifier(topologyToken.getBytes()); - this.userPassword = SaslUtils.encodePassword(token); - } + /** + * Set private members using topology token. + * + * @param topologyToken + */ + public SaslClientCallbackHandler(String topologyToken, byte[] token) { + this.userName = SaslUtils + .encodeIdentifier(topologyToken.getBytes()); + this.userPassword = SaslUtils.encodePassword(token); + } - /** - * Implementation used to respond to SASL tokens from server. - * - * @param callbacks - * objects that indicate what credential information the - * server's SaslServer requires from the client. - * @throws UnsupportedCallbackException - */ - public void handle(Callback[] callbacks) - throws UnsupportedCallbackException { - NameCallback nc = null; - PasswordCallback pc = null; - RealmCallback rc = null; - for (Callback callback : callbacks) { - if (callback instanceof RealmChoiceCallback) { - continue; - } else if (callback instanceof NameCallback) { - nc = (NameCallback) callback; - } else if (callback instanceof PasswordCallback) { - pc = (PasswordCallback) callback; - } else if (callback instanceof RealmCallback) { - rc = (RealmCallback) callback; - } else { - throw new UnsupportedCallbackException(callback, - "handle: Unrecognized SASL client callback"); - } - } - if (nc != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("handle: SASL client callback: setting username: " - + userName); - } - nc.setName(userName); - } - if (pc != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("handle: SASL client callback: setting userPassword"); - } - pc.setPassword(userPassword); - } - if (rc != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("handle: SASL client callback: setting realm: " - + rc.getDefaultText()); - } - rc.setText(rc.getDefaultText()); - } - } - } + /** + * Implementation used to respond to SASL tokens from server. + * + * @param callbacks + * objects that indicate what credential information the + * server's SaslServer requires from the client. + * @throws UnsupportedCallbackException + */ + public void handle(Callback[] callbacks) + throws UnsupportedCallbackException { + NameCallback nc = null; + PasswordCallback pc = null; + RealmCallback rc = null; + for (Callback callback : callbacks) { + if (callback instanceof RealmChoiceCallback) { + continue; + } else if (callback instanceof NameCallback) { + nc = (NameCallback) callback; + } else if (callback instanceof PasswordCallback) { + pc = (PasswordCallback) callback; + } else if (callback instanceof RealmCallback) { + rc = (RealmCallback) callback; + } else { + throw new UnsupportedCallbackException(callback, + "handle: Unrecognized SASL client callback"); + } + } + if (nc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("handle: SASL client callback: setting username: " + + userName); + } + nc.setName(userName); + } + if (pc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("handle: SASL client callback: setting userPassword"); + } + pc.setPassword(userPassword); + } + if (rc != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("handle: SASL client callback: setting realm: " + + rc.getDefaultText()); + } + rc.setText(rc.getDefaultText()); + } + } + } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java index 59c4abd51e2..f94cbc3d8ea 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java @@ -32,124 +32,127 @@ public class SaslStormClientHandler extends SimpleChannelUpstreamHandler { - private static final Logger LOG = LoggerFactory - .getLogger(SaslStormClientHandler.class); - private Client client; - long start_time; - /** Used for client or server's token to send or receive from each other. */ - private byte[] token; - private String topologyName; - - public SaslStormClientHandler(Client client) throws IOException { - this.client = client; - start_time = System.currentTimeMillis(); - getSASLCredentials(); - } - - @Override - public void channelConnected(ChannelHandlerContext ctx, - ChannelStateEvent event) { - // register the newly established channel - Channel channel = ctx.getChannel(); - - LOG.info("Connection established from " + channel.getLocalAddress() - + " to " + channel.getRemoteAddress()); - - try { - SaslNettyClient saslNettyClient = SaslNettyClientState.getSaslNettyClient - .get(channel); - - if (saslNettyClient == null) { - LOG.debug("Creating saslNettyClient now " + "for channel: " - + channel); - saslNettyClient = new SaslNettyClient(topologyName, token); - SaslNettyClientState.getSaslNettyClient.set(channel, - saslNettyClient); - } - channel.write(ControlMessage.SASL_TOKEN_MESSAGE_REQUEST); - } catch (Exception e) { - LOG.error("Failed to authenticate with server " + "due to error: " - + e); - } - return; - - } - - @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) - throws Exception { - LOG.debug("send/recv time (ms): {}", - (System.currentTimeMillis() - start_time)); - - Channel channel = ctx.getChannel(); - - // Generate SASL response to server using Channel-local SASL client. - SaslNettyClient saslNettyClient = SaslNettyClientState.getSaslNettyClient - .get(channel); - if (saslNettyClient == null) { - throw new Exception("saslNettyClient was unexpectedly " - + "null for channel: " + channel); - } - - // examine the response message from server - if (event.getMessage() instanceof ControlMessage) { - ControlMessage msg = (ControlMessage) event.getMessage(); - if (msg == ControlMessage.SASL_COMPLETE_REQUEST) { - LOG.debug("Server has sent us the SaslComplete " - + "message. Allowing normal work to proceed."); - - if (!saslNettyClient.isComplete()) { - LOG.error("Server returned a Sasl-complete message, " - + "but as far as we can tell, we are not authenticated yet."); - throw new Exception("Server returned a " - + "Sasl-complete message, but as far as " - + "we can tell, we are not authenticated yet."); - } - ctx.getPipeline().remove(this); - // We call fireMessageReceived since the client is allowed to - // perform this request. The client's request will now proceed - // to the next pipeline component namely StormClientHandler. - Channels.fireMessageReceived(ctx, msg); - return; - } - } - SaslMessageToken saslTokenMessage = (SaslMessageToken) event - .getMessage(); - LOG.debug("Responding to server's token of length: " - + saslTokenMessage.getSaslToken().length); - - // Generate SASL response (but we only actually send the response if - // it's non-null. - byte[] responseToServer = saslNettyClient - .saslResponse(saslTokenMessage); - if (responseToServer == null) { - // If we generate a null response, then authentication has completed - // (if not, warn), and return without sending a response back to the - // server. - LOG.debug("Response to server is null: " - + "authentication should now be complete."); - if (!saslNettyClient.isComplete()) { - LOG.warn("Generated a null response, " - + "but authentication is not complete."); - } - return; - } else { - LOG.debug("Response to server token has length:" - + responseToServer.length); - } - // Construct a message containing the SASL response and send it to the - // server. - SaslMessageToken saslResponse = new SaslMessageToken(responseToServer); - channel.write(saslResponse); - } - - private void getSASLCredentials() throws IOException { - topologyName = (String) this.client.storm_conf - .get(Config.TOPOLOGY_NAME); - String secretKey = SaslUtils.getSecretKey(this.client.storm_conf); - if(secretKey!=null) { - token = secretKey.getBytes(); - } - LOG.debug("SASL credentials for storm topology "+topologyName+ " is "+secretKey); - } + private static final Logger LOG = LoggerFactory + .getLogger(SaslStormClientHandler.class); + private Client client; + long start_time; + /** Used for client or server's token to send or receive from each other. */ + private byte[] token; + private String topologyName; + + public SaslStormClientHandler(Client client) throws IOException { + this.client = client; + start_time = System.currentTimeMillis(); + getSASLCredentials(); + } + + @Override + public void channelConnected(ChannelHandlerContext ctx, + ChannelStateEvent event) { + // register the newly established channel + Channel channel = ctx.getChannel(); + + LOG.info("Connection established from " + channel.getLocalAddress() + + " to " + channel.getRemoteAddress()); + + try { + SaslNettyClient saslNettyClient = SaslNettyClientState.getSaslNettyClient + .get(channel); + + if (saslNettyClient == null) { + LOG.debug("Creating saslNettyClient now " + "for channel: " + + channel); + saslNettyClient = new SaslNettyClient(topologyName, token); + SaslNettyClientState.getSaslNettyClient.set(channel, + saslNettyClient); + } + channel.write(ControlMessage.SASL_TOKEN_MESSAGE_REQUEST); + } catch (Exception e) { + LOG.error("Failed to authenticate with server " + "due to error: ", + e); + } + return; + + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) + throws Exception { + LOG.debug("send/recv time (ms): {}", + (System.currentTimeMillis() - start_time)); + + Channel channel = ctx.getChannel(); + + // Generate SASL response to server using Channel-local SASL client. + SaslNettyClient saslNettyClient = SaslNettyClientState.getSaslNettyClient + .get(channel); + if (saslNettyClient == null) { + throw new Exception("saslNettyClient was unexpectedly " + + "null for channel: " + channel); + } + + // examine the response message from server + if (event.getMessage() instanceof ControlMessage) { + ControlMessage msg = (ControlMessage) event.getMessage(); + if (msg == ControlMessage.SASL_COMPLETE_REQUEST) { + LOG.debug("Server has sent us the SaslComplete " + + "message. Allowing normal work to proceed."); + + if (!saslNettyClient.isComplete()) { + LOG.error("Server returned a Sasl-complete message, " + + "but as far as we can tell, we are not authenticated yet."); + throw new Exception("Server returned a " + + "Sasl-complete message, but as far as " + + "we can tell, we are not authenticated yet."); + } + ctx.getPipeline().remove(this); + // We call fireMessageReceived since the client is allowed to + // perform this request. The client's request will now proceed + // to the next pipeline component namely StormClientHandler. + Channels.fireMessageReceived(ctx, msg); + return; + } + } + SaslMessageToken saslTokenMessage = (SaslMessageToken) event + .getMessage(); + LOG.debug("Responding to server's token of length: " + + saslTokenMessage.getSaslToken().length); + + // Generate SASL response (but we only actually send the response if + // it's non-null. + byte[] responseToServer = saslNettyClient + .saslResponse(saslTokenMessage); + if (responseToServer == null) { + // If we generate a null response, then authentication has completed + // (if not, warn), and return without sending a response back to the + // server. + LOG.debug("Response to server is null: " + + "authentication should now be complete."); + if (!saslNettyClient.isComplete()) { + LOG.warn("Generated a null response, " + + "but authentication is not complete."); + throw new Exception("Server reponse is null, but as far as " + + "we can tell, we are not authenticated yet."); + } + return; + } else { + LOG.debug("Response to server token has length:" + + responseToServer.length); + } + // Construct a message containing the SASL response and send it to the + // server. + SaslMessageToken saslResponse = new SaslMessageToken(responseToServer); + channel.write(saslResponse); + } + + private void getSASLCredentials() throws IOException { + topologyName = (String) this.client.storm_conf + .get(Config.TOPOLOGY_NAME); + String secretKey = SaslUtils.getSecretKey(this.client.storm_conf); + if (secretKey != null) { + token = secretKey.getBytes(); + } + LOG.debug("SASL credentials for storm topology " + topologyName + + " is " + secretKey); + } } \ No newline at end of file diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java index d06e96096e7..02448e2991f 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java @@ -32,124 +32,124 @@ public class SaslStormServerHandler extends SimpleChannelUpstreamHandler { - Server server; - /** Used for client or server's token to send or receive from each other. */ - private byte[] token; - private String topologyName; - - private static final Logger LOG = LoggerFactory - .getLogger(SaslStormServerHandler.class); - - public SaslStormServerHandler(Server server) throws IOException { - this.server = server; - getSASLCredentials(); - } - - @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) - throws Exception { - Object msg = e.getMessage(); - if (msg == null) - return; - - Channel channel = ctx.getChannel(); - - if (msg instanceof ControlMessage - && ((ControlMessage) e.getMessage()) == ControlMessage.SASL_TOKEN_MESSAGE_REQUEST) { - // initialize server-side SASL functionality, if we haven't yet - // (in which case we are looking at the first SASL message from the - // client). - SaslNettyServer saslNettyServer = SaslNettyServerState.getSaslNettyServer - .get(channel); - if (saslNettyServer == null) { - LOG.debug("No saslNettyServer for " + channel - + " yet; creating now, with topology token: "); - try { - saslNettyServer = new SaslNettyServer(topologyName, token); - } catch (IOException ioe) { - LOG.error("Error occurred while creating saslNettyServer on server " - + channel.getLocalAddress() - + " for client " - + channel.getRemoteAddress()); - throw new IOException(ioe); - } - - SaslNettyServerState.getSaslNettyServer.set(channel, - saslNettyServer); - } else { - LOG.debug("Found existing saslNettyServer on server:" - + channel.getLocalAddress() + " for client " - + channel.getRemoteAddress()); - } - - LOG.debug("processToken: With nettyServer: " + saslNettyServer - + " and token length: " + token.length); - - SaslMessageToken saslTokenMessageRequest = null; - saslTokenMessageRequest = new SaslMessageToken( - saslNettyServer.response(new byte[0])); - // Send response to client. - channel.write(saslTokenMessageRequest); - // do not send upstream to other handlers: no further action needs - // to be done for SASL_TOKEN_MESSAGE_REQUEST requests. - return; - } - - if (msg instanceof SaslMessageToken) { - // initialize server-side SASL functionality, if we haven't yet - // (in which case we are looking at the first SASL message from the - // client). - SaslNettyServer saslNettyServer = SaslNettyServerState.getSaslNettyServer - .get(channel); - if (saslNettyServer == null) { - if (saslNettyServer == null) { - throw new Exception("saslNettyServer was unexpectedly " - + "null for channel: " + channel); - } - } - SaslMessageToken saslTokenMessageRequest = new SaslMessageToken( - saslNettyServer.response(((SaslMessageToken) msg) - .getSaslToken())); - - // Send response to client. - channel.write(saslTokenMessageRequest); - - if (saslNettyServer.isComplete()) { - // If authentication of client is complete, we will also send a - // SASL-Complete message to the client. - LOG.debug("SASL authentication is complete for client with " - + "username: " + saslNettyServer.getUserName()); - channel.write(ControlMessage.SASL_COMPLETE_REQUEST); - LOG.debug("Removing SaslServerHandler from pipeline since SASL " - + "authentication is complete."); - ctx.getPipeline().remove(this); - } - return; - } else { - // Client should not be sending other-than-SASL messages before - // SaslServerHandler has removed itself from the pipeline. Such - // non-SASL requests will be denied by the Authorize channel handler - // (the next handler upstream in the server pipeline) if SASL - // authentication has not completed. - LOG.warn("Sending upstream an unexpected non-SASL message : " - + msg); - Channels.fireMessageReceived(ctx, msg); - } - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { - server.closeChannel(e.getChannel()); - } - - private void getSASLCredentials() throws IOException { - topologyName = (String) this.server.storm_conf - .get(Config.TOPOLOGY_NAME); - String secretKey = SaslUtils.getSecretKey(this.server.storm_conf); - if (secretKey != null) { - token = secretKey.getBytes(); - } - LOG.debug("SASL credentials for storm topology " + topologyName - + " is " + secretKey); - } + Server server; + /** Used for client or server's token to send or receive from each other. */ + private byte[] token; + private String topologyName; + + private static final Logger LOG = LoggerFactory + .getLogger(SaslStormServerHandler.class); + + public SaslStormServerHandler(Server server) throws IOException { + this.server = server; + getSASLCredentials(); + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) + throws Exception { + Object msg = e.getMessage(); + if (msg == null) + return; + + Channel channel = ctx.getChannel(); + + if (msg instanceof ControlMessage + && ((ControlMessage) e.getMessage()) == ControlMessage.SASL_TOKEN_MESSAGE_REQUEST) { + // initialize server-side SASL functionality, if we haven't yet + // (in which case we are looking at the first SASL message from the + // client). + SaslNettyServer saslNettyServer = SaslNettyServerState.getSaslNettyServer + .get(channel); + if (saslNettyServer == null) { + LOG.debug("No saslNettyServer for " + channel + + " yet; creating now, with topology token: "); + try { + saslNettyServer = new SaslNettyServer(topologyName, token); + } catch (IOException ioe) { + LOG.error("Error occurred while creating saslNettyServer on server " + + channel.getLocalAddress() + + " for client " + + channel.getRemoteAddress()); + saslNettyServer = null; + } + + SaslNettyServerState.getSaslNettyServer.set(channel, + saslNettyServer); + } else { + LOG.debug("Found existing saslNettyServer on server:" + + channel.getLocalAddress() + " for client " + + channel.getRemoteAddress()); + } + + LOG.debug("processToken: With nettyServer: " + saslNettyServer + + " and token length: " + token.length); + + SaslMessageToken saslTokenMessageRequest = null; + saslTokenMessageRequest = new SaslMessageToken( + saslNettyServer.response(new byte[0])); + // Send response to client. + channel.write(saslTokenMessageRequest); + // do not send upstream to other handlers: no further action needs + // to be done for SASL_TOKEN_MESSAGE_REQUEST requests. + return; + } + + if (msg instanceof SaslMessageToken) { + // initialize server-side SASL functionality, if we haven't yet + // (in which case we are looking at the first SASL message from the + // client). + SaslNettyServer saslNettyServer = SaslNettyServerState.getSaslNettyServer + .get(channel); + if (saslNettyServer == null) { + if (saslNettyServer == null) { + throw new Exception("saslNettyServer was unexpectedly " + + "null for channel: " + channel); + } + } + SaslMessageToken saslTokenMessageRequest = new SaslMessageToken( + saslNettyServer.response(((SaslMessageToken) msg) + .getSaslToken())); + + // Send response to client. + channel.write(saslTokenMessageRequest); + + if (saslNettyServer.isComplete()) { + // If authentication of client is complete, we will also send a + // SASL-Complete message to the client. + LOG.debug("SASL authentication is complete for client with " + + "username: " + saslNettyServer.getUserName()); + channel.write(ControlMessage.SASL_COMPLETE_REQUEST); + LOG.debug("Removing SaslServerHandler from pipeline since SASL " + + "authentication is complete."); + ctx.getPipeline().remove(this); + } + return; + } else { + // Client should not be sending other-than-SASL messages before + // SaslServerHandler has removed itself from the pipeline. Such + // non-SASL requests will be denied by the Authorize channel handler + // (the next handler upstream in the server pipeline) if SASL + // authentication has not completed. + LOG.warn("Sending upstream an unexpected non-SASL message : " + + msg); + Channels.fireMessageReceived(ctx, msg); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { + server.closeChannel(e.getChannel()); + } + + private void getSASLCredentials() throws IOException { + topologyName = (String) this.server.storm_conf + .get(Config.TOPOLOGY_NAME); + String secretKey = SaslUtils.getSecretKey(this.server.storm_conf); + if (secretKey != null) { + token = secretKey.getBytes(); + } + LOG.debug("SASL credentials for storm topology " + topologyName + + " is " + secretKey); + } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java index 0f962338beb..a4cc0ba6c13 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java @@ -28,48 +28,47 @@ import backtype.storm.Config; class SaslUtils { - public static final String AUTH_DIGEST_MD5 = "DIGEST-MD5"; - public static final String DEFAULT_REALM = "default"; + public static final String AUTH_DIGEST_MD5 = "DIGEST-MD5"; + public static final String DEFAULT_REALM = "default"; - static Map getSaslProps() { - Map props = new HashMap(); - props.put(Sasl.POLICY_NOPLAINTEXT, "true"); - return props; - } + static Map getSaslProps() { + Map props = new HashMap(); + props.put(Sasl.POLICY_NOPLAINTEXT, "true"); + return props; + } - /** - * Encode a password as a base64-encoded char[] array. - * - * @param password - * as a byte array. - * @return password as a char array. - */ - static char[] encodePassword(byte[] password) { - return new String(Base64.encodeBase64(password), - Charset.defaultCharset()).toCharArray(); - } + /** + * Encode a password as a base64-encoded char[] array. + * + * @param password + * as a byte array. + * @return password as a char array. + */ + static char[] encodePassword(byte[] password) { + return new String(Base64.encodeBase64(password), + Charset.defaultCharset()).toCharArray(); + } - /** - * Encode a identifier as a base64-encoded char[] array. - * - * @param identifier - * as a byte array. - * @return identifier as a char array. - */ - static String encodeIdentifier(byte[] identifier) { - return new String(Base64.encodeBase64(identifier), - Charset.defaultCharset()); - } + /** + * Encode a identifier as a base64-encoded char[] array. + * + * @param identifier + * as a byte array. + * @return identifier as a char array. + */ + static String encodeIdentifier(byte[] identifier) { + return new String(Base64.encodeBase64(identifier), + Charset.defaultCharset()); + } - static String getSecretKey(Map conf) { - if (conf == null || conf.isEmpty()) - return null; + static String getSecretKey(Map conf) { + if (conf == null || conf.isEmpty()) + return null; - String secretPayLoad = (String) conf - .get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD); + String secretPayLoad = (String) conf + .get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD); + + return secretPayLoad; + } - return secretPayLoad; - } - - } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java index 4fdaee90d2b..1ea382bd0f2 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java @@ -27,31 +27,27 @@ class StormClientPipelineFactory implements ChannelPipelineFactory { private Client client; StormClientPipelineFactory(Client client) { - this.client = client; + this.client = client; } public ChannelPipeline getPipeline() throws Exception { // Create a default pipeline implementation. ChannelPipeline pipeline = Channels.pipeline(); - boolean isNettyAuth = (Boolean) this.client.storm_conf.get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION); - if(isNettyAuth) { - // Decoder - pipeline.addLast("decoder", new MessageDecoder()); - // Encoder - pipeline.addLast("encoder", new MessageEncoder()); - // Authenticate: Removed after authentication completes - pipeline.addLast("saslClientHandler", new SaslStormClientHandler(client)); - // business logic. - pipeline.addLast("handler", new StormClientErrorHandler(client.name())); - } else { - // Decoder - pipeline.addLast("decoder", new MessageDecoder()); - // Encoder - pipeline.addLast("encoder", new MessageEncoder()); - // business logic. - pipeline.addLast("handler", new StormClientErrorHandler(client.name())); + // Decoder + pipeline.addLast("decoder", new MessageDecoder()); + // Encoder + pipeline.addLast("encoder", new MessageEncoder()); + + boolean isNettyAuth = (Boolean) this.client.storm_conf + .get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION); + if (isNettyAuth) { + // Authenticate: Removed after authentication completes + pipeline.addLast("saslClientHandler", new SaslStormClientHandler( + client)); } + // business logic. + pipeline.addLast("handler", new StormClientErrorHandler(client.name())); return pipeline; } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java index c2b4c53a4e0..f6e20c5b789 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerPipelineFactory.java @@ -22,41 +22,36 @@ import org.jboss.netty.channel.Channels; import backtype.storm.Config; -import backtype.storm.utils.Utils; - -class StormServerPipelineFactory implements ChannelPipelineFactory { +class StormServerPipelineFactory implements ChannelPipelineFactory { private Server server; - + StormServerPipelineFactory(Server server) { - this.server = server; + this.server = server; } - + public ChannelPipeline getPipeline() throws Exception { // Create a default pipeline implementation. ChannelPipeline pipeline = Channels.pipeline(); - boolean isNettyAuth = (Boolean) this.server.storm_conf.get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION); - if(isNettyAuth) { - // Decoder - pipeline.addLast("decoder", new MessageDecoder()); - // Encoder - pipeline.addLast("encoder", new MessageEncoder()); + // Decoder + pipeline.addLast("decoder", new MessageDecoder()); + // Encoder + pipeline.addLast("encoder", new MessageEncoder()); + + boolean isNettyAuth = (Boolean) this.server.storm_conf + .get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION); + if (isNettyAuth) { // Authenticate: Removed after authentication completes - pipeline.addLast("saslServerHandler", new SaslStormServerHandler(server)); + pipeline.addLast("saslServerHandler", new SaslStormServerHandler( + server)); // Authorize - pipeline.addLast("authorizeServerHandler", new SaslStormServerAuthorizeHandler()); - // business logic. - pipeline.addLast("handler", new StormServerHandler(server)); - } else { - // Decoder - pipeline.addLast("decoder", new MessageDecoder()); - // Encoder - pipeline.addLast("encoder", new MessageEncoder()); - // business logic. - pipeline.addLast("handler", new StormServerHandler(server)); + pipeline.addLast("authorizeServerHandler", + new SaslStormServerAuthorizeHandler()); } - + // business logic. + pipeline.addLast("handler", new StormServerHandler(server)); + return pipeline; } } From 19ad13510cb42963f868fd8bd785b4937dd348ab Mon Sep 17 00:00:00 2001 From: Raghavendra Nandagopal Date: Mon, 28 Jul 2014 17:42:05 -0700 Subject: [PATCH 38/61] STORM-348: Netty SASL Authentication --- .../src/jvm/backtype/storm/messaging/netty/SaslUtils.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java index a4cc0ba6c13..a2d0b26a1f2 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java @@ -24,6 +24,7 @@ import javax.security.sasl.Sasl; import org.apache.commons.codec.binary.Base64; +import org.apache.commons.io.Charsets; import backtype.storm.Config; @@ -45,8 +46,8 @@ static Map getSaslProps() { * @return password as a char array. */ static char[] encodePassword(byte[] password) { - return new String(Base64.encodeBase64(password), - Charset.defaultCharset()).toCharArray(); + return new String(Base64.encodeBase64(password), Charsets.UTF_8) + .toCharArray(); } /** @@ -57,8 +58,7 @@ static char[] encodePassword(byte[] password) { * @return identifier as a char array. */ static String encodeIdentifier(byte[] identifier) { - return new String(Base64.encodeBase64(identifier), - Charset.defaultCharset()); + return new String(Base64.encodeBase64(identifier), Charsets.UTF_8); } static String getSecretKey(Map conf) { From ee5bb1792afc28155bd8cdf87e97ebf719c031d1 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 29 Jul 2014 16:47:19 -0500 Subject: [PATCH 39/61] STORM-427: AutoTGT and HBase can expose JVM kerberos bug. --- .../storm/security/auth/kerberos/AutoTGT.java | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java index 52bf540e679..1e07daa8be2 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java @@ -33,12 +33,14 @@ import java.lang.reflect.Constructor; import java.security.Principal; import java.util.concurrent.atomic.AtomicReference; +import java.util.Iterator; import javax.security.auth.kerberos.KerberosTicket; import javax.security.auth.kerberos.KerberosPrincipal; import javax.security.auth.login.Configuration; import javax.security.auth.login.LoginException; import javax.security.auth.login.LoginContext; +import javax.security.auth.DestroyFailedException; import javax.security.auth.RefreshFailedException; import javax.security.auth.Subject; import javax.xml.bind.DatatypeConverter; @@ -152,10 +154,22 @@ public void populateSubject(Subject subject, Map credentials) { private void populateSubjectWithTGT(Subject subject, Map credentials) { KerberosTicket tgt = getTGT(credentials); if (tgt != null) { - KerberosTicket oldTGT = getTGT(subject); - subject.getPrivateCredentials().add(tgt); - if (oldTGT != null && !oldTGT.equals(tgt)) { - subject.getPrivateCredentials().remove(oldTGT); + Set creds = subject.getPrivateCredentials(); + synchronized(creds) { + Iterator iterator = creds.iterator(); + while (iterator.hasNext()) { + Object o = iterator.next(); + if (o instanceof KerberosTicket) { + KerberosTicket t = (KerberosTicket)o; + iterator.remove(); + try { + t.destroy(); + } catch (DestroyFailedException e) { + LOG.warn("Failed to destory ticket ", e); + } + } + } + creds.add(tgt); } subject.getPrincipals().add(tgt.getClient()); kerbTicket.set(tgt); From 90cf0e6c21df978462d29fe906ad0f2a6adcdb1f Mon Sep 17 00:00:00 2001 From: Raghavendra Nandagopal Date: Tue, 29 Jul 2014 17:32:38 -0700 Subject: [PATCH 40/61] STORM-348: Netty SASL Authentication --- .../clj/backtype/storm/messaging/netty_integration_test.clj | 1 + .../test/clj/backtype/storm/messaging/netty_unit_test.clj | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj index 8534c827b99..dea4abe0592 100644 --- a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj +++ b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj @@ -25,6 +25,7 @@ (with-simulated-time-local-cluster [cluster :supervisors 4 :supervisor-slot-port-min 6710 :daemon-conf {STORM-LOCAL-MODE-ZMQ true STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" + STORM_MESSAGING_NETTY_AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj index ea7b8dc0f5a..04d25ecc2bd 100644 --- a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj +++ b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj @@ -26,6 +26,7 @@ (deftest test-basic (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz") storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" + STORM_MESSAGING_NETTY_AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 1024 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 @@ -48,6 +49,7 @@ (deftest test-large-msg (let [req_msg (apply str (repeat 2048000 'c')) storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" + STORM_MESSAGING_NETTY_AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 102400 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 @@ -70,6 +72,7 @@ (deftest test-server-delayed (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz") storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" + STORM_MESSAGING_NETTY_AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 1024 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 @@ -99,6 +102,7 @@ (deftest test-batch (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" + STORM_MESSAGING_NETTY_AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 From 00f7d632eb48bf967b5a7b72f5460352d0a8dbd9 Mon Sep 17 00:00:00 2001 From: Raghavendra Nandagopal Date: Tue, 29 Jul 2014 17:50:24 -0700 Subject: [PATCH 41/61] STORM-348: Netty SASL Authentication --- .../backtype/storm/messaging/netty_integration_test.clj | 2 +- .../test/clj/backtype/storm/messaging/netty_unit_test.clj | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj index dea4abe0592..98144ccef03 100644 --- a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj +++ b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj @@ -25,7 +25,7 @@ (with-simulated-time-local-cluster [cluster :supervisors 4 :supervisor-slot-port-min 6710 :daemon-conf {STORM-LOCAL-MODE-ZMQ true STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" - STORM_MESSAGING_NETTY_AUTHENTICATION false + STORM-MESSAGING-NETTY-AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj index 04d25ecc2bd..8aaa7e5aff6 100644 --- a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj +++ b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj @@ -26,7 +26,7 @@ (deftest test-basic (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz") storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" - STORM_MESSAGING_NETTY_AUTHENTICATION false + STORM-MESSAGING-NETTY-AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 1024 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 @@ -49,7 +49,7 @@ (deftest test-large-msg (let [req_msg (apply str (repeat 2048000 'c')) storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" - STORM_MESSAGING_NETTY_AUTHENTICATION false + STORM-MESSAGING-NETTY-AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 102400 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 @@ -72,7 +72,7 @@ (deftest test-server-delayed (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz") storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" - STORM_MESSAGING_NETTY_AUTHENTICATION false + STORM-MESSAGING-NETTY-AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 1024 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 @@ -102,7 +102,7 @@ (deftest test-batch (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" - STORM_MESSAGING_NETTY_AUTHENTICATION false + STORM-MESSAGING-NETTY-AUTHENTICATION false STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 From cf5fc0c3f0a71d8e81d344f247a477694e83a2a7 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 30 Jul 2014 11:15:43 -0500 Subject: [PATCH 42/61] Updated README for STORM-348 --- README.markdown | 1 + 1 file changed, 1 insertion(+) diff --git a/README.markdown b/README.markdown index cf67f644943..586f1712f80 100644 --- a/README.markdown +++ b/README.markdown @@ -159,6 +159,7 @@ under the License. * Jo Liss ([@joliss](https://github.com/joliss)) * averykhoo ([@averykhoo](https://github.com/averykhoo)) * Curtis Allen ([@curtisallen](https://github.com/curtisallen)) +* Raghavendra Nandagopal ([@RaghavendraNandagopal](https://github.com/RaghavendraNandagopal)) ## Acknowledgements From 3208a8e3f39140227fe6be9c686b795a19777e64 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 4 Aug 2014 19:18:26 -0700 Subject: [PATCH 43/61] STORM-349. (Security) ui actions should have nimbus like authroization. Upgraded ring to 1.3.0. Added IAuthorizer instance to UI daemon. Added hadoop-auth as a dependency which allow users to configure AuthenticationFilter. --- SECURITY.md | 25 ++++- pom.xml | 8 +- storm-core/pom.xml | 21 +++- storm-core/src/clj/backtype/storm/ui/core.clj | 105 ++++++++---------- .../src/clj/backtype/storm/ui/helpers.clj | 15 +-- .../test/clj/backtype/storm/ui_test.clj | 49 -------- 6 files changed, 103 insertions(+), 120 deletions(-) delete mode 100644 storm-core/test/clj/backtype/storm/ui_test.clj diff --git a/SECURITY.md b/SECURITY.md index 882954a768f..f919de70eda 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -54,7 +54,26 @@ must have it set to the actual port that they are going to bind to. 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. +them. + +Storm UI can be configured to use AuthenticationFilter from hadoop-auth. +```yaml +ui.filter: "org.apache.hadoop.security.authentication.server.AuthenticationFilter" +ui.filter.params: + "type": "kerberos" + "kerberos.principal": "HTTP/nimbus.witzend.com" + "kerberos.keytab": "/vagrant/keytabs/http.keytab" + "kerberos.name.rules": "RULE:[2:$1@$0]([jt]t@.*EXAMPLE.COM)s/.*/$MAPRED_USER/ RULE:[2:$1@$0]([nd]n@.*EXAMPLE.COM)s/.*/$HDFS_USER/DEFAULT" +``` +make sure to create a prinicpal 'HTTP/{hostname}' (here hostname should be the one where UI daemon runs + +Once configured users needs to do kinit before accessing UI. +Ex: +curl -i --negotiate -u:anyUser -b ~/cookiejar.txt -c ~/cookiejar.txt http://storm-ui-hostname:8080/api/v1/cluster/summary + +1) Firefox: Goto about:config and search for network.negotiate-auth.trusted-uris double-click to add value "http://storm-ui-hostname:8080" +2) Google-chrome: start from command line with: google-chrome --auth-server-whitelist="*storm-ui-hostname" --auth-negotiate-delegate-whitelist="*storm-ui-hostname" +3) IE: Configure trusted websites to include "storm-ui-hostname" and allow negotiation for that website ## Authentication (Kerberos) @@ -335,7 +354,3 @@ The Logviewer deamon now is also responsible for cleaning up old log files for d ### DRPC Hopefully more on this soon - - - - diff --git a/pom.xml b/pom.xml index 87c52ea177d..04b6ef1d350 100644 --- a/pom.xml +++ b/pom.xml @@ -187,7 +187,8 @@ 0.4.1 2.4.0 1.1 - 0.3.11 + 1.3.0 + 7.6.13.v20130916 0.2.3 0.0.1 1.4.0 @@ -389,6 +390,11 @@ ring-jetty-adapter ${ring.version} + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + org.clojure tools.logging diff --git a/storm-core/pom.xml b/storm-core/pom.xml index c677b7ce73d..3ac8f230ae8 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -59,6 +59,10 @@ ring ring-jetty-adapter + + org.eclipse.jetty + jetty-servlet + org.clojure tools.logging @@ -96,7 +100,7 @@ data.codec test - + commons-io @@ -196,6 +200,21 @@ conjure test + + org.apache.hadoop + hadoop-auth + 2.4.0 + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + junit junit diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj index 8dd301e2a20..0aea7ecc14c 100644 --- a/storm-core/src/clj/backtype/storm/ui/core.clj +++ b/storm-core/src/clj/backtype/storm/ui/core.clj @@ -20,8 +20,8 @@ (:use [hiccup core page-helpers]) (:use [backtype.storm config util log]) (:use [backtype.storm.ui helpers]) - (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID - ACKER-ACK-STREAM-ID ACKER-FAIL-STREAM-ID system-id?]]]) + (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID + ACKER-FAIL-STREAM-ID system-id? mk-authorization-handler]]]) (:use [ring.adapter.jetty :only [run-jetty]]) (:use [clojure.string :only [blank? lower-case trim]]) (:import [backtype.storm.utils Utils]) @@ -30,6 +30,8 @@ ErrorInfo ClusterSummary SupervisorSummary TopologySummary Nimbus$Client StormTopology GlobalStreamId RebalanceOptions KillOptions]) + (:import [backtype.storm.security.auth AuthUtils ReqContext]) + (:import [backtype.storm.generated AuthorizationException]) (:import [backtype.storm.security.auth AuthUtils]) (:import [java.io File]) (:require [compojure.route :as route] @@ -40,6 +42,9 @@ (:gen-class)) (def ^:dynamic *STORM-CONF* (read-storm-config)) +(def ^:dynamic *UI-ACL-HANDLER* (mk-authorization-handler (*STORM-CONF* NIMBUS-AUTHORIZER) *STORM-CONF*)) + +(def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*)) (defmacro with-nimbus [nimbus-sym & body] @@ -47,33 +52,19 @@ [~nimbus-sym (*STORM-CONF* NIMBUS-HOST) (*STORM-CONF* NIMBUS-THRIFT-PORT)] ~@body)) -(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))] - (or (blank? (conf UI-FILTER)) - (and (not (blank? user)) - (some #(= % user) ui-users))))) - -(defn assert-authorized-ui-user - [user conf topology-conf] - (if (not (authorized-ui-user? user conf topology-conf)) - ;;TODO need a better exception here so the UI can appear better - (throw (RuntimeException. (str "User " user " is not authorized."))))) - -(defn- ui-actions-enabled? - [] - (= "true" (lower-case (*STORM-CONF* UI-ACTIONS-ENABLED)))) - -(defn assert-authorized-topology-user - [user] - ;;TODO eventually we will want to use the Authorizatin handler from nimbus, but for now - ;; Disable the calls conditionally - (if (not (ui-actions-enabled?)) - ;;TODO need a better exception here so the UI can appear better - (throw (RuntimeException. (str "Topology actions for the UI have been disabled"))))) +(defn assert-authorized-user + ([servlet-request op] + (assert-authorized-user servlet-request op nil)) + ([servlet-request op topology-conf] + (if http-creds-handler (.populateContext http-creds-handler (ReqContext/context) servlet-request)) + (if *UI-ACL-HANDLER* + (let [context (ReqContext/context)] + (if-not (.permit *UI-ACL-HANDLER* context op topology-conf) + (let [principal (.principal context) + user (if principal (.getName principal) "unknown")] + (throw (AuthorizationException. + (str "UI request '" op "' for '" + user "' user is not authorized"))))))))) (defn get-filled-stats [summs] @@ -474,9 +465,9 @@ :checked (is-ack-stream (get m :stream))}))))))] (map (fn [row] {:row row}) (partition 4 4 nil streams)))) - + (defn mk-visualization-data - [id window include-sys? user] + [id window include-sys?] (with-nimbus nimbus (let [window (if window window ":all-time") @@ -491,10 +482,9 @@ bolt-comp-summs (group-by-comp bolt-summs) bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) bolt-comp-summs) - topology-conf (from-json + topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id))] - (assert-authorized-ui-user user *STORM-CONF* topology-conf) - (visualization-data + (visualization-data (merge (hashmap-to-persistent spouts) (hashmap-to-persistent bolts)) spout-comp-summs bolt-comp-summs window id)))) @@ -675,7 +665,6 @@ bolt-comp-summs window id)] - (assert-authorized-ui-user user *STORM-CONF* topology-conf) (merge (topology-summary summ) {"user" user @@ -686,8 +675,7 @@ "spouts" (spout-comp id spout-comp-summs (.get_errors summ) window include-sys?) "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys?) "configuration" topology-conf - "visualizationTable" (stream-boxes visualizer-data) - "uiActionsEnabled" (ui-actions-enabled?)})))) + "visualizationTable" (stream-boxes visualizer-data)})))) (defn spout-output-stats [stream-summary window] @@ -842,7 +830,6 @@ spec (cond (= type :spout) (spout-stats window summ component summs include-sys?) (= type :bolt) (bolt-stats window summ component summs include-sys?)) errors (component-errors (get (.get_errors summ) component) topology-id)] - (assert-authorized-ui-user user *STORM-CONF* topology-conf) (merge {"user" user "id" component @@ -855,6 +842,10 @@ "windowHint" (window-hint window)} spec errors)))) +(defn topology-config [topology-id] + (with-nimbus nimbus + (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id)))) + (defn check-include-sys? [sys?] (if (or (nil? sys?) (= "false" sys?)) false true)) @@ -871,42 +862,44 @@ (cluster-configuration)) (GET "/api/v1/cluster/summary" [:as {:keys [cookies servlet-request]}] (let [user (.getUserName http-creds-handler servlet-request)] + (assert-authorized-user servlet-request "getClusterInfo") (json-response (cluster-summary user)))) - (GET "/api/v1/supervisor/summary" [] + (GET "/api/v1/supervisor/summary" [:as {:keys [cookies servlet-request]}] + (assert-authorized-user servlet-request "getClusterInfo") (json-response (supervisor-summary))) - (GET "/api/v1/topology/summary" [] + (GET "/api/v1/topology/summary" [:as {:keys [cookies servlet-request]}] + (assert-authorized-user servlet-request "getClusterInfo") (json-response (all-topologies-summary))) (GET "/api/v1/topology/:id" [:as {:keys [cookies servlet-request]} id & m] (let [id (url-decode id) user (.getUserName http-creds-handler servlet-request)] + (assert-authorized-user servlet-request "getTopology" (topology-config id)) (json-response (topology-page id (:window m) (check-include-sys? (:sys m)) user)))) (GET "/api/v1/topology/:id/visualization" [:as {:keys [cookies servlet-request]} id & m] - (let [id (url-decode id) - user (.getUserName http-creds-handler servlet-request)] - (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m)) user)))) + (let [id (url-decode id)] + (assert-authorized-user servlet-request "getTopology" (topology-config id)) + (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m)))))) (GET "/api/v1/topology/:id/component/:component" [:as {:keys [cookies servlet-request]} id component & m] (let [id (url-decode id) component (url-decode component) user (.getUserName http-creds-handler servlet-request)] + (assert-authorized-user servlet-request "getTopology" (topology-config id)) (json-response (component-page id component (:window m) (check-include-sys? (:sys m)) user)))) (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id] (with-nimbus nimbus (let [id (url-decode id) tplg (.getTopologyInfo ^Nimbus$Client nimbus id) - name (.get_name tplg) - user (.getUserName http-creds-handler servlet-request)] - (assert-authorized-topology-user user) + name (.get_name tplg)] + (assert-authorized-user servlet-request "activate" (topology-config id)) (.activate nimbus name) (log-message "Activating topology '" name "'"))) (resp/redirect (str "/api/v1/topology/" id))) - (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id] (with-nimbus nimbus (let [id (url-decode id) tplg (.getTopologyInfo ^Nimbus$Client nimbus id) - name (.get_name tplg) - user (.getUserName http-creds-handler servlet-request)] - (assert-authorized-topology-user user) + name (.get_name tplg)] + (assert-authorized-user servlet-request "deactivate" (topology-config id)) (.deactivate nimbus name) (log-message "Deactivating topology '" name "'"))) (resp/redirect (str "/api/v1/topology/" id))) @@ -915,9 +908,8 @@ (let [id (url-decode id) tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg) - options (RebalanceOptions.) - user (.getUserName http-creds-handler servlet-request)] - (assert-authorized-topology-user user) + options (RebalanceOptions.)] + (assert-authorized-user servlet-request "rebalance" (topology-config id)) (.set_wait_secs options (Integer/parseInt wait-time)) (.rebalance nimbus name options) (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs"))) @@ -927,9 +919,8 @@ (let [id (url-decode id) tplg (.getTopologyInfo ^Nimbus$Client nimbus id) name (.get_name tplg) - options (KillOptions.) - user (.getUserName http-creds-handler servlet-request)] - (assert-authorized-topology-user user) + options (KillOptions.)] + (assert-authorized-user servlet-request "killTopology" (topology-config id)) (.set_wait_secs options (Integer/parseInt wait-time)) (.killTopologyWithOpts nimbus name options) (log-message "Killing topology '" name "' with wait time: " wait-time " secs"))) @@ -972,7 +963,7 @@ :join? false :configurator (fn [server] (doseq [connector (.getConnectors server)] - (.setHeaderBufferSize connector header-buffer-size)) + (.setRequestHeaderSize connector header-buffer-size)) (config-filter server app filters-confs))})) (catch Exception ex (log-error ex)))) diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj index 60e7bd21db8..8033c5022a5 100644 --- a/storm-core/src/clj/backtype/storm/ui/helpers.clj +++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj @@ -23,7 +23,8 @@ (: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]) + (:import [org.eclipse.jetty.server.ssl SslSocketConnector] + [org.eclipse.jetty.servlet ServletHolder FilterMapping]) (:require [ring.util servlet]) (:require [compojure.route :as route] [compojure.handler :as handler])) @@ -129,7 +130,7 @@ $(\"table#%s\").each(function(i) { $(this).tablesorter({ sortList: %s, headers: ))) (defn url-format [fmt & args] - (String/format fmt + (String/format fmt (to-array (map #(url-encode (str %)) args)))) (defn to-tasks [^ExecutorInfo e] @@ -166,18 +167,18 @@ $(\"table#%s\").each(function(i) { $(this).tablesorter({ sortList: %s, headers: (defn config-filter [server handler filters-confs] (if filters-confs - (let [servlet-holder (org.mortbay.jetty.servlet.ServletHolder. + (let [servlet-holder (ServletHolder. (ring.util.servlet/servlet handler)) - context (doto (org.mortbay.jetty.servlet.Context. server "/") + context (doto (org.eclipse.jetty.servlet.ServletContextHandler. 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.) + (let [filter-holder (doto (org.eclipse.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)))) + (.addFilter context filter-holder "/*" FilterMapping/ALL)))) + (.setHandler server context)))) (defn ring-response-from-exception [ex] {:headers {} diff --git a/storm-core/test/clj/backtype/storm/ui_test.clj b/storm-core/test/clj/backtype/storm/ui_test.clj deleted file mode 100644 index 21ddb33355e..00000000000 --- a/storm-core/test/clj/backtype/storm/ui_test.clj +++ /dev/null @@ -1,49 +0,0 @@ -;; Licensed to the Apache Software Foundation (ASF) under one -;; or more contributor license agreements. See the NOTICE file -;; distributed with this work for additional information -;; regarding copyright ownership. The ASF licenses this file -;; to you under the Apache License, Version 2.0 (the -;; "License"); you may not use this file except in compliance -;; with the License. You may obtain a copy of the License at -;; -;; http://www.apache.org/licenses/LICENSE-2.0 -;; -;; Unless required by applicable law or agreed to in writing, software -;; distributed under the License is distributed on an "AS IS" BASIS, -;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -;; See the License for the specific language governing permissions and -;; limitations under the License. -(ns backtype.storm.ui-test - (:use [clojure test]) - (:use [backtype.storm config]) - (:use [backtype.storm testing]) - (:require [backtype.storm.ui [core :as core]]) - ) - -(deftest test-authorized-ui-user - (testing "allow cluster admin" - (let [conf {UI-FILTER "something" NIMBUS-ADMINS ["alice"]}] - (is (core/authorized-ui-user? "alice" conf {})))) - - (testing "ignore any cluster-set topology.users" - (let [conf {UI-FILTER "something" TOPOLOGY-USERS ["alice"]}] - (is (not (core/authorized-ui-user? "alice" conf {}))))) - - (testing "allow cluster ui user" - (let [conf {UI-FILTER "something" UI-USERS ["alice"]}] - (is (core/authorized-ui-user? "alice" conf {})))) - - (testing "allow submitted topology user" - (let [topo-conf {TOPOLOGY-USERS ["alice"]}] - (is (core/authorized-ui-user? "alice" {UI-FILTER "something"} topo-conf)))) - - (testing "allow submitted ui user" - (let [topo-conf {UI-USERS ["alice"]}] - (is (core/authorized-ui-user? "alice" {UI-FILTER "something"} topo-conf)))) - - (testing "disallow user not in nimbus admin, topo user, or ui user" - (is (not (core/authorized-ui-user? "alice" {UI-FILTER "something"} {})))) - - (testing "user cannot override nimbus admin" - (let [topo-conf {NIMBUS-ADMINS ["alice"]}] - (is (not (core/authorized-ui-user? "alice" {UI-FILTER "something"} topo-conf)))))) From b73b051e683f1f27d299b15d79e99ce74bbd6054 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 14 Aug 2014 13:42:40 -0700 Subject: [PATCH 44/61] STORM-349. excluded dependencies from hadoop-auth. removed Config.UI_ACTIONS_ENABLED. --- storm-core/pom.xml | 64 ++++++++++++++++--- storm-core/src/jvm/backtype/storm/Config.java | 33 ++++------ 2 files changed, 68 insertions(+), 29 deletions(-) diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 3ac8f230ae8..f2d4ee3fd50 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -205,14 +205,62 @@ hadoop-auth 2.4.0 - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - + + junit + junit + + + org.mockito + mockito-all + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + jetty + + + javax.servlet + servlet-api + + + org.slf4j + slf4j-api + + + commons-codec + commons-codec + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.apache.hadoop + hadoop-minikdc + + + org.apache.directory.server + apacheds-kerberos-codec + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.apache.httpcomponents + httpclient + diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 94d1cb302fa..5b3da15405e 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -59,39 +59,39 @@ public class Config extends HashMap { /** * Netty based messaging: The buffer size for send/recv buffer */ - public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; + public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = ConfigValidation.IntegerValidator; /** * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible */ - public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; + public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = ConfigValidation.IntegerValidator; /** * Netty based messaging: The min # of milliseconds that a peer will wait. */ - public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; + public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = ConfigValidation.IntegerValidator; /** * Netty based messaging: The max # of milliseconds that a peer will wait. */ - public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; + public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = ConfigValidation.IntegerValidator; /** * Netty based messaging: The # of worker threads for the server. */ - public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads"; + public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads"; public static final Object STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS_SCHEMA = ConfigValidation.IntegerValidator; /** * Netty based messaging: The # of worker threads for the client. */ - public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads"; + public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads"; public static final Object STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS_SCHEMA = ConfigValidation.IntegerValidator; - + /** * If the Netty messaging layer is busy, the Netty client will try to batch message as more as possible up to the size of STORM_NETTY_MESSAGE_BATCH_SIZE bytes */ @@ -107,9 +107,9 @@ public class Config extends HashMap { /** * Netty based messaging: Is authentication required for Netty messaging from client worker process to server worker process. */ - public static final String STORM_MESSAGING_NETTY_AUTHENTICATION = "storm.messaging.netty.authentication"; + public static final String STORM_MESSAGING_NETTY_AUTHENTICATION = "storm.messaging.netty.authentication"; public static final Object STORM_MESSAGING_NETTY_AUTHENTICATION_SCHEMA = Boolean.class; - + /** * A list of hosts of ZooKeeper servers used to manage the cluster. */ @@ -513,15 +513,6 @@ public class Config extends HashMap { 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. */ @@ -782,7 +773,7 @@ public class Config extends HashMap { */ public static final String WORKER_RECEIVER_THREAD_COUNT = "topology.worker.receiver.thread.count"; public static final Object WORKER_RECEIVER_THREAD_COUNT_SCHEMA = ConfigValidation.IntegerValidator; - + /** * How often this worker should heartbeat to the supervisor. */ @@ -947,7 +938,7 @@ public class Config extends HashMap { * Note that this config parameter has no effect for unreliable spouts that don't tag * their tuples with a message id. */ - public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; + public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = ConfigValidation.IntegerValidator; /** @@ -1010,7 +1001,7 @@ public class Config extends HashMap { public static final Object TOPOLOGY_CLASSPATH_SCHEMA = ConfigValidation.StringOrStringListValidator; /** - * Topology-specific environment variables for the worker child process. + * Topology-specific environment variables for the worker child process. * This is added to the existing environment (that of the supervisor) */ public static final String TOPOLOGY_ENVIRONMENT="topology.environment"; From 576726292a88c67aa9e9ba7563de114540487f54 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Sun, 17 Aug 2014 14:56:27 -0700 Subject: [PATCH 45/61] STORM-406: Fix for storm CSRF vulnerability using ring-anti-forgery. --- storm-core/pom.xml | 7 ++++++- storm-core/src/clj/backtype/storm/ui/core.clj | 14 +++++++++++--- storm-core/src/ui/public/css/style.css | 4 ++++ storm-core/src/ui/public/js/script.js | 3 ++- .../templates/anti-forgery-template.html | 19 +++++++++++++++++++ storm-core/src/ui/public/topology.html | 7 +++++++ 6 files changed, 49 insertions(+), 5 deletions(-) create mode 100644 storm-core/src/ui/public/templates/anti-forgery-template.html diff --git a/storm-core/pom.xml b/storm-core/pom.xml index c677b7ce73d..44bca865dea 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -59,6 +59,11 @@ ring ring-jetty-adapter + + ring + ring-anti-forgery + 1.0.0 + org.clojure tools.logging @@ -96,7 +101,7 @@ data.codec test - + commons-io diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj index 8dd301e2a20..f771ee6fa3c 100644 --- a/storm-core/src/clj/backtype/storm/ui/core.clj +++ b/storm-core/src/clj/backtype/storm/ui/core.clj @@ -23,6 +23,7 @@ (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID ACKER-FAIL-STREAM-ID system-id?]]]) (:use [ring.adapter.jetty :only [run-jetty]]) + (:use [ring.middleware.anti-forgery]) (:use [clojure.string :only [blank? lower-case trim]]) (:import [backtype.storm.utils Utils]) (:import [backtype.storm.generated ExecutorSpecificStats @@ -687,7 +688,8 @@ "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys?) "configuration" topology-conf "visualizationTable" (stream-boxes visualizer-data) - "uiActionsEnabled" (ui-actions-enabled?)})))) + "uiActionsEnabled" (ui-actions-enabled?) + "anti-forgery-token" *anti-forgery-token*})))) (defn spout-output-stats [stream-summary window] @@ -956,10 +958,16 @@ (catch Exception ex (json-response (exception->json ex) 500))))) + +(def csrf-error-response + (json-response {"error" "Forbidden action." + "errorMessage" "missing CSRF token."} 403)) + (def app (handler/site (-> main-routes - (wrap-reload '[backtype.storm.ui.core]) - catch-errors))) + (wrap-reload '[backtype.storm.ui.core]) + (wrap-anti-forgery {:error-response csrf-error-response}) + catch-errors))) (defn start-server! [] diff --git a/storm-core/src/ui/public/css/style.css b/storm-core/src/ui/public/css/style.css index 29a45ebd73d..85a8961d590 100644 --- a/storm-core/src/ui/public/css/style.css +++ b/storm-core/src/ui/public/css/style.css @@ -24,6 +24,10 @@ padding: 0.5em; } +.anti-forgery-token { +visibility:hidden; +} + body { color: #808080; padding: 0.2em; diff --git a/storm-core/src/ui/public/js/script.js b/storm-core/src/ui/public/js/script.js index fef3b593a6e..09ecd570709 100644 --- a/storm-core/src/ui/public/js/script.js +++ b/storm-core/src/ui/public/js/script.js @@ -71,7 +71,8 @@ function ensureInt(n) { function confirmAction(id, name, action, wait, defaultWait) { var opts = { type:'POST', - url:'/api/v1/topology/' + id + '/' + action + url:'/api/v1/topology/' + id + '/' + action, + headers: { 'x-csrf-token': $.trim($('#anti-forgery-token').text()) } }; if (wait) { var waitSecs = prompt('Do you really want to ' + action + ' topology "' + name + '"? ' + diff --git a/storm-core/src/ui/public/templates/anti-forgery-template.html b/storm-core/src/ui/public/templates/anti-forgery-template.html new file mode 100644 index 00000000000..1a63ace1211 --- /dev/null +++ b/storm-core/src/ui/public/templates/anti-forgery-template.html @@ -0,0 +1,19 @@ + + diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index eb745228692..805b6b46c37 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -52,6 +52,8 @@

Topology summary

+
+
diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index eb745228692..805b6b46c37 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -52,6 +52,8 @@

Topology summary

+
+
From 6e3c77a69c48c11ce9564b5098953ff7a722e95a Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 28 Aug 2014 09:39:36 -0500 Subject: [PATCH 49/61] Scheduler handles already used slot not reported by supervisor --- .../storm/scheduler/multitenant/Node.java | 20 ++- .../scheduler/multitenant_scheduler_test.clj | 163 ++++++++++++------ 2 files changed, 126 insertions(+), 57 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java index 1c601ca866f..883c65fe656 100644 --- a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java +++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java @@ -127,10 +127,19 @@ private void addOrphanedSlot(WorkerSlot ws) { boolean assignInternal(WorkerSlot ws, String topId, boolean dontThrow) { validateSlot(ws); if (!_freeSlots.remove(ws)) { - if (dontThrow) { - return true; + for (Entry> topologySetEntry : _topIdToUsedSlots.entrySet()) { + if (topologySetEntry.getValue().contains(ws)) { + if (dontThrow) { + LOG.warn("Worker slot [" + ws + "] can't be assigned to " + topId + + ". Its already assigned to " + topologySetEntry.getKey() + "."); + return true; + } + throw new IllegalStateException("Worker slot [" + ws + "] can't be assigned to " + + topId + ". Its already assigned to " + topologySetEntry.getKey() + "."); + } } - throw new IllegalStateException("Assigning a slot that was not free " + ws); + LOG.warn("Adding Worker slot [" + ws + "] that was not reported in the supervisor heartbeats," + + " but the worker is already running for topology " + topId + "."); } Set usedSlots = _topIdToUsedSlots.get(topId); if (usedSlots == null) { @@ -291,7 +300,7 @@ public static Map getAllNodesFrom(Cluster cluster) { //Node ID and supervisor ID are the same. String id = sup.getId(); boolean isAlive = !cluster.isBlackListed(id); - LOG.debug("Found a {} Node {} {}", + LOG.debug("Found a {} Node {} {}", new Object[] {isAlive? "living":"dead", id, sup.getAllPorts()}); nodeIdToNode.put(id, new Node(id, sup.getAllPorts(), isAlive)); } @@ -311,7 +320,8 @@ public static Map getAllNodesFrom(Cluster cluster) { node.addOrphanedSlot(ws); } if (node.assignInternal(ws, topId, true)) { - LOG.warn("Bad scheduling state, "+ws+" assigned multiple workers, unassigning everything..."); + LOG.warn("Bad scheduling state for topology [" + topId+ "], the slot " + + ws + " assigned to multiple workers, un-assigning everything..."); node.free(ws, cluster, true); } } diff --git a/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj index b3cdb131cd2..049a0d6f4d8 100644 --- a/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj +++ b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj @@ -716,57 +716,116 @@ )) (deftest test-multitenant-scheduler-bad-starting-state - (let [supers (gen-supervisors 10) - topology1 (TopologyDetails. "topology1" - {TOPOLOGY-NAME "topology-name-1" - TOPOLOGY-SUBMITTER-USER "userC"} - (StormTopology.) - 4 - (mk-ed-map [["spout1" 0 5] - ["bolt1" 5 10] - ["bolt2" 10 15] - ["bolt3" 15 20]])) - topology2 (TopologyDetails. "topology2" - {TOPOLOGY-NAME "topology-name-2" - TOPOLOGY-ISOLATED-MACHINES 2 - TOPOLOGY-SUBMITTER-USER "userA"} - (StormTopology.) - 4 - (mk-ed-map [["spout11" 0 5] - ["bolt12" 5 6] - ["bolt13" 6 7] - ["bolt14" 7 10]])) - topology3 (TopologyDetails. "topology3" - {TOPOLOGY-NAME "topology-name-3" - TOPOLOGY-ISOLATED-MACHINES 5 - TOPOLOGY-SUBMITTER-USER "userB"} - (StormTopology.) - 10 - (mk-ed-map [["spout21" 0 10] - ["bolt22" 10 20] - ["bolt23" 20 30] - ["bolt24" 30 40]])) - existing-assignments { - "topology2" (SchedulerAssignmentImpl. "topology2" {(ExecutorDetails. 0 5) (WorkerSlot. "super1" 1)}) - "topology3" (SchedulerAssignmentImpl. "topology3" {(ExecutorDetails. 0 10) (WorkerSlot. "super1" 1)}) - } - cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments) - topologies (Topologies. (to-top-map [topology1 topology2 topology3])) - conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}} - scheduler (MultitenantScheduler.)] - (.prepare scheduler conf) - (.schedule scheduler topologies cluster) - (let [assignment (.getAssignmentById cluster "topology1") - assigned-slots (.getSlots assignment) - executors (.getExecutors assignment)] - ;; 4 slots on 1 machine, all executors assigned - (is (= 4 (.size assigned-slots))) - (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) - (is (= 20 (.size executors))) - ) - (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1"))) - (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2"))) - (is (= "Scheduled Isolated on 5 Nodes" (.get (.getStatusMap cluster) "topology3"))) -)) + (testing "Assiging same worker slot to different topologies is bad state" + (let [supers (gen-supervisors 5) + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1" + TOPOLOGY-SUBMITTER-USER "userC"} + (StormTopology.) + 1 + (mk-ed-map [["spout1" 0 1]])) + topology2 (TopologyDetails. "topology2" + {TOPOLOGY-NAME "topology-name-2" + TOPOLOGY-ISOLATED-MACHINES 2 + TOPOLOGY-SUBMITTER-USER "userA"} + (StormTopology.) + 1 + (mk-ed-map [["spout11" 1 2]])) + topology3 (TopologyDetails. "topology3" + {TOPOLOGY-NAME "topology-name-3" + TOPOLOGY-ISOLATED-MACHINES 1 + TOPOLOGY-SUBMITTER-USER "userB"} + (StormTopology.) + 1 + (mk-ed-map [["spout21" 2 3]])) + worker-slot-with-multiple-assignments (WorkerSlot. "super1" 1) + existing-assignments {"topology2" (SchedulerAssignmentImpl. "topology2" {(ExecutorDetails. 1 1) worker-slot-with-multiple-assignments}) + "topology3" (SchedulerAssignmentImpl. "topology3" {(ExecutorDetails. 2 2) worker-slot-with-multiple-assignments})} + cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments) + topologies (Topologies. (to-top-map [topology1 topology2 topology3])) + conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 2 "userB" 1}} + scheduler (MultitenantScheduler.)] + (.prepare scheduler conf) + (.schedule scheduler topologies cluster) + (let [assignment (.getAssignmentById cluster "topology1") + assigned-slots (.getSlots assignment) + executors (.getExecutors assignment)] + (is (= 1 (.size assigned-slots)))) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1"))) + (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2"))) + (is (= "Scheduled Isolated on 1 Nodes" (.get (.getStatusMap cluster) "topology3")))))) +(deftest test-existing-assignment-slot-not-found-in-supervisor + (testing "Scheduler should handle discrepancy when a live supervisor heartbeat does not report slot, + but worker heartbeat says its running on that slot" + (let [supers (gen-supervisors 1) + port-not-reported-by-supervisor 6 + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1" + TOPOLOGY-SUBMITTER-USER "userA"} + (StormTopology.) + 1 + (mk-ed-map [["spout11" 0 1]])) + existing-assignments {"topology1" + (SchedulerAssignmentImpl. "topology1" + {(ExecutorDetails. 0 0) (WorkerSlot. "super0" port-not-reported-by-supervisor)})} + cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments) + topologies (Topologies. (to-top-map [topology1])) + conf {} + scheduler (MultitenantScheduler.)] + (.prepare scheduler conf) + (.schedule scheduler topologies cluster) + (let [assignment (.getAssignmentById cluster "topology1") + assigned-slots (.getSlots assignment) + executors (.getExecutors assignment)] + (is (= 1 (.size assigned-slots)))) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))) +(deftest test-existing-assignment-slot-on-dead-supervisor + (testing "Dead supervisor could have slot with duplicate assignments or slot never reported by supervisor" + (let [supers (gen-supervisors 1) + dead-supervisor "super1" + port-not-reported-by-supervisor 6 + topology1 (TopologyDetails. "topology1" + {TOPOLOGY-NAME "topology-name-1" + TOPOLOGY-SUBMITTER-USER "userA"} + (StormTopology.) + 2 + (mk-ed-map [["spout11" 0 1] + ["bolt12" 1 2]])) + topology2 (TopologyDetails. "topology2" + {TOPOLOGY-NAME "topology-name-2" + TOPOLOGY-SUBMITTER-USER "userA"} + (StormTopology.) + 2 + (mk-ed-map [["spout21" 4 5] + ["bolt22" 5 6]])) + worker-slot-with-multiple-assignments (WorkerSlot. dead-supervisor 1) + existing-assignments {"topology1" + (SchedulerAssignmentImpl. "topology1" + {(ExecutorDetails. 0 0) worker-slot-with-multiple-assignments + (ExecutorDetails. 1 1) (WorkerSlot. dead-supervisor 3)}) + "topology2" + (SchedulerAssignmentImpl. "topology2" + {(ExecutorDetails. 4 4) worker-slot-with-multiple-assignments + (ExecutorDetails. 5 5) (WorkerSlot. dead-supervisor port-not-reported-by-supervisor)})} + cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments) + topologies (Topologies. (to-top-map [topology1 topology2])) + conf {} + scheduler (MultitenantScheduler.)] + (.prepare scheduler conf) + (.schedule scheduler topologies cluster) + (let [assignment (.getAssignmentById cluster "topology1") + assigned-slots (.getSlots assignment) + executors (.getExecutors assignment)] + (is (= 2 (.size assigned-slots))) + (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + (is (= 2 (.size executors)))) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1"))) + (let [assignment (.getAssignmentById cluster "topology2") + assigned-slots (.getSlots assignment) + executors (.getExecutors assignment)] + (is (= 2 (.size assigned-slots))) + (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) + (is (= 2 (.size executors)))) + (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))) From b19fdbb7444e9d3ea38ede9d186055319df4751d Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 28 Aug 2014 15:02:27 -0500 Subject: [PATCH 50/61] Fixed some merge issues. --- pom.xml | 5 +++++ storm-core/pom.xml | 4 ++++ storm-core/src/clj/backtype/storm/daemon/logviewer.clj | 4 +--- storm-core/src/ui/public/templates/index-page-template.html | 2 +- .../src/ui/public/templates/topology-page-template.html | 2 -- 5 files changed, 11 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 7d31916731c..2ee56fc7bea 100644 --- a/pom.xml +++ b/pom.xml @@ -396,6 +396,11 @@ jetty-servlet ${jetty.version}
+ + org.eclipse.jetty + jetty-servlets + ${jetty.version} + org.clojure tools.logging diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 1a676f1f841..37858580dca 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -63,6 +63,10 @@ org.eclipse.jetty jetty-servlet + + org.eclipse.jetty + jetty-servlets + org.clojure tools.logging diff --git a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj index 36a737fdbfe..c3868ebb675 100644 --- a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj +++ b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj @@ -348,15 +348,13 @@ Note that if anything goes wrong, this will throw an Error and exit." :filter-params (or (conf UI-FILTER-PARAMS) {})}] []) filters-confs (concat filters-confs - [{:filter-class "org.mortbay.servlet.GzipFilter" + [{:filter-class "org.eclipse.jetty.servlets.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)))) diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html index 8f1c29b65eb..dfe94460b9f 100644 --- a/storm-core/src/ui/public/templates/index-page-template.html +++ b/storm-core/src/ui/public/templates/index-page-template.html @@ -53,8 +53,8 @@ {{uptime}} {{workersTotal}} {{executorsTotal}} - {{schedulerInfo}} {{tasksTotal}} + {{schedulerInfo}} {{/topologies}} diff --git a/storm-core/src/ui/public/templates/topology-page-template.html b/storm-core/src/ui/public/templates/topology-page-template.html index ddf3c17228e..d442107ca92 100644 --- a/storm-core/src/ui/public/templates/topology-page-template.html +++ b/storm-core/src/ui/public/templates/topology-page-template.html @@ -160,7 +160,6 @@

Bolts ({{windowHint}})

From f5db183648ea2cac5fead49efbc6dfc12df823a4 Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Thu, 28 Aug 2014 16:24:40 -0500 Subject: [PATCH 51/61] change maven-dependency-plugin version 2.6 -> 2.8 --- storm-core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 37858580dca..e5812bcf504 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -404,7 +404,7 @@ maven-dependency-plugin - 2.6 + 2.8 copy-dependencies From b07eff3fd86eed1fd67c1159e03d52e583b655e8 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 29 Aug 2014 19:19:47 +0000 Subject: [PATCH 52/61] Changed plugin to clear out old credentials. --- .../security/auth/DefaultHttpCredentialsPlugin.java | 12 ++++++++---- .../auth/DefaultHttpCredentialsPlugin_test.clj | 10 +++++----- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java index 7d8e0fb5d14..86455580a81 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java @@ -72,12 +72,16 @@ public String getUserName(HttpServletRequest req) { public ReqContext populateContext(ReqContext context, HttpServletRequest req) { String userName = getUserName(req); + Principal p = null; if (userName != null) { - Set principals = new HashSet(1); - principals.add(new SingleUserPrincipal(userName)); - Subject s = new Subject(true, principals, new HashSet(), new HashSet()); - context.setSubject(s); + p = new SingleUserPrincipal(userName); } + Set principals = new HashSet(1); + if (p != null) { + principals.add(p); + } + Subject s = new Subject(true, principals, new HashSet(), new HashSet()); + context.setSubject(s); return context; } } diff --git a/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj b/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj index ab54d822fe7..bf62a6bef7f 100644 --- a/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj +++ b/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj @@ -2,6 +2,7 @@ (:use [clojure test]) (:import [javax.security.auth Subject]) (:import [javax.servlet.http HttpServletRequest]) + (:import [backtype.storm.security.auth SingleUserPrincipal]) (:import [org.mockito Mockito]) (:import [backtype.storm.security.auth DefaultHttpCredentialsPlugin ReqContext SingleUserPrincipal]) @@ -31,10 +32,9 @@ thenReturn princ) (is (.equals exp-name (.getUserName handler req))))))) -(deftest test-populate-req-context-noop-on-null-user +(deftest test-populate-req-context-on-null-user (let [req (Mockito/mock HttpServletRequest) handler (doto (DefaultHttpCredentialsPlugin.) (.prepare {})) - expected-subj (Subject.) - context (ReqContext. expected-subj)] - (is (.equals expected-subj - (-> handler (.populateContext context req) (.subject)))))) + subj (Subject. false (set [(SingleUserPrincipal. "test")]) (set []) (set [])) + context (ReqContext. subj)] + (is (= 0 (-> handler (.populateContext context req) (.subject) (.getPrincipals) (.size)))))) From af61cca75c6215d5a0db2d6fb338deca03e55095 Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Wed, 3 Sep 2014 10:05:25 -0500 Subject: [PATCH 53/61] use separate logback config for workers --- logback/worker.xml | 41 +++++++++++++++++++ .../clj/backtype/storm/daemon/supervisor.clj | 2 +- .../clj/backtype/storm/supervisor_test.clj | 4 +- 3 files changed, 44 insertions(+), 3 deletions(-) create mode 100644 logback/worker.xml diff --git a/logback/worker.xml b/logback/worker.xml new file mode 100644 index 00000000000..3d9009e413b --- /dev/null +++ b/logback/worker.xml @@ -0,0 +1,41 @@ + + + + + + ${storm.log.dir}/${logfile.name} + + ${storm.log.dir}/${logfile.name}.%i + 1 + 9 + + + + 100MB + + + + %d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n + + + + + + + + diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj index 7b242e51383..1fe5b2eed35 100644 --- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj @@ -638,7 +638,7 @@ (str "-Dlogfile.name=" logfilename) (str "-Dstorm.home=" storm-home) (str "-Dstorm.log.dir=" storm-log-dir) - (str "-Dlogback.configurationFile=" storm-home "/logback/cluster.xml") + (str "-Dlogback.configurationFile=" storm-home "/logback/worker.xml") (str "-Dstorm.id=" storm-id) (str "-Dworker.id=" worker-id) (str "-Dworker.port=" port) diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj index c94688f4b53..4261b4d2e81 100644 --- a/storm-core/test/clj/backtype/storm/supervisor_test.clj +++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj @@ -262,7 +262,7 @@ (str "-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log") "-Dstorm.home=" "-Dstorm.log.dir=/logs" - "-Dlogback.configurationFile=/logback/cluster.xml" + "-Dlogback.configurationFile=/logback/worker.xml" (str "-Dstorm.id=" mock-storm-id) (str "-Dworker.id=" mock-worker-id) (str "-Dworker.port=" mock-port) @@ -380,7 +380,7 @@ " '-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log'" " '-Dstorm.home='" " '-Dstorm.log.dir=/logs'" - " '-Dlogback.configurationFile=/logback/cluster.xml'" + " '-Dlogback.configurationFile=/logback/worker.xml'" " '-Dstorm.id=" mock-storm-id "'" " '-Dworker.id=" mock-worker-id "'" " '-Dworker.port=" mock-port "'" From d07da44a76e1365ed8ea5cb44e7efab30f15aafa Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Thu, 4 Sep 2014 12:46:12 -0500 Subject: [PATCH 54/61] add new logback config to binary distribution --- storm-dist/binary/src/main/assembly/binary.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/storm-dist/binary/src/main/assembly/binary.xml b/storm-dist/binary/src/main/assembly/binary.xml index adc1d80b9d7..e612894b7a4 100644 --- a/storm-dist/binary/src/main/assembly/binary.xml +++ b/storm-dist/binary/src/main/assembly/binary.xml @@ -128,6 +128,11 @@ /logback + + ${project.basedir}/../../logback/worker.xml + /logback + + ${project.basedir}/LICENSE From b0cdf65da8542ffffca8b3caffbd7921866a5649 Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Thu, 4 Sep 2014 12:48:19 -0500 Subject: [PATCH 55/61] remove empty line --- storm-dist/binary/src/main/assembly/binary.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/storm-dist/binary/src/main/assembly/binary.xml b/storm-dist/binary/src/main/assembly/binary.xml index e612894b7a4..05720f9a584 100644 --- a/storm-dist/binary/src/main/assembly/binary.xml +++ b/storm-dist/binary/src/main/assembly/binary.xml @@ -133,7 +133,6 @@ /logback - ${project.basedir}/LICENSE / From f8cb7b594aaad68e9298914610f200e970f4eb11 Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Thu, 4 Sep 2014 15:29:31 -0500 Subject: [PATCH 56/61] avoid adding multiple jetty handlers --- .../src/clj/backtype/storm/daemon/drpc.clj | 25 +++++++++-------- .../clj/backtype/storm/daemon/logviewer.clj | 9 +++---- storm-core/src/clj/backtype/storm/ui/core.clj | 12 ++++----- .../src/clj/backtype/storm/ui/helpers.clj | 27 ++++++++++++++++++- 4 files changed, 46 insertions(+), 27 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj index 68128c37bdb..95fa1be31df 100644 --- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj +++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj @@ -30,7 +30,6 @@ (: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)) @@ -190,9 +189,9 @@ (if http-creds-handler (.populateContext http-creds-handler (ReqContext/context) servlet-request)) - (.execute handler func ""))) - (wrap-reload '[backtype.storm.daemon.drpc]) - handle-request)) + (.execute handler func ""))) + (wrap-reload '[backtype.storm.daemon.drpc]) + handle-request)) (defn launch-server! ([] @@ -230,15 +229,15 @@ 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))}))) + (storm-run-jetty + {:port drpc-http-port + :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))))) diff --git a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj index c3868ebb675..0745d5da193 100644 --- a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj +++ b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj @@ -20,7 +20,6 @@ (:use [hiccup core page-helpers]) (: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 [ch.qos.logback.core FileAppender]) @@ -351,11 +350,9 @@ Note that if anything goes wrong, this will throw an Error and exit." [{:filter-class "org.eclipse.jetty.servlets.GzipFilter" :filter-name "Gzipper" :filter-params {}}])] - (run-jetty middle - {:port (int (conf LOGVIEWER-PORT)) - :join? false - :configurator (fn [server] - (config-filter server middle filters-confs))})) + (storm-run-jetty {:port (int (conf LOGVIEWER-PORT)) + :configurator (fn [server] + (config-filter server middle filters-confs))})) (catch Exception ex (log-error ex)))) diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj index d0d71c3aba5..043f1232809 100644 --- a/storm-core/src/clj/backtype/storm/ui/core.clj +++ b/storm-core/src/clj/backtype/storm/ui/core.clj @@ -22,7 +22,6 @@ (:use [backtype.storm.ui helpers]) (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID ACKER-FAIL-STREAM-ID system-id? mk-authorization-handler]]]) - (:use [ring.adapter.jetty :only [run-jetty]]) (:use [clojure.string :only [blank? lower-case trim]]) (:import [backtype.storm.utils Utils]) (:import [backtype.storm.generated ExecutorSpecificStats @@ -967,12 +966,11 @@ 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)] - (.setRequestHeaderSize connector header-buffer-size)) - (config-filter server app filters-confs))})) + (storm-run-jetty {:port (conf UI-PORT) + :configurator (fn [server] + (doseq [connector (.getConnectors server)] + (.setRequestHeaderSize connector header-buffer-size)) + (config-filter server app filters-confs))})) (catch Exception ex (log-error ex)))) diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj index 8033c5022a5..413a68d92c4 100644 --- a/storm-core/src/clj/backtype/storm/ui/helpers.clj +++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj @@ -23,7 +23,9 @@ (:use [backtype.storm.util :only [clojurify-structure uuid defnk url-encode]]) (:use [clj-time coerce format]) (:import [backtype.storm.generated ExecutorInfo ExecutorSummary]) - (:import [org.eclipse.jetty.server.ssl SslSocketConnector] + (:import [org.eclipse.jetty.server Server] + [org.eclipse.jetty.server.nio SelectChannelConnector] + [org.eclipse.jetty.server.ssl SslSocketConnector] [org.eclipse.jetty.servlet ServletHolder FilterMapping]) (:require [ring.util servlet]) (:require [compojure.route :as route] @@ -184,3 +186,26 @@ $(\"table#%s\").each(function(i) { $(this).tablesorter({ sortList: %s, headers: {:headers {} :status 400 :body (.getMessage ex)}) + +;; Modified from ring.adapter.jetty 1.3.0 +(defn- jetty-create-server + "Construct a Jetty Server instance." + [options] + (let [connector (doto (SelectChannelConnector.) + (.setPort (options :port 80)) + (.setHost (options :host)) + (.setMaxIdleTime (options :max-idle-time 200000))) + server (doto (Server.) + (.addConnector connector) + (.setSendDateHeader true))] + server)) + +(defn storm-run-jetty + "Modified version of run-jetty + Assumes configurator sets handler." + [config] + {:pre [(:configurator config)]} + (let [#^Server s (jetty-create-server (dissoc config :configurator)) + configurator (:configurator config)] + (configurator s) + (.start s))) From a7b5d68e65d196c5b6330320b412fbaef6335c1e Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Fri, 5 Sep 2014 10:37:34 -0500 Subject: [PATCH 57/61] fix indentation --- storm-core/src/ui/public/css/style.css | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/ui/public/css/style.css b/storm-core/src/ui/public/css/style.css index 6790a696958..2346f6806a7 100644 --- a/storm-core/src/ui/public/css/style.css +++ b/storm-core/src/ui/public/css/style.css @@ -25,7 +25,7 @@ } .anti-forgery-token { -visibility:hidden; + visibility:hidden; } body { From ce0b18a3915f1020042b90ca3e981e586175732b Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sat, 6 Sep 2014 20:11:24 -0700 Subject: [PATCH 58/61] STORM-489. remove uiActionsEnabled from js and templates. --- storm-core/src/ui/public/js/script.js | 3 +-- storm-core/src/ui/public/topology.html | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/storm-core/src/ui/public/js/script.js b/storm-core/src/ui/public/js/script.js index 09ecd570709..9d75322e6d9 100644 --- a/storm-core/src/ui/public/js/script.js +++ b/storm-core/src/ui/public/js/script.js @@ -141,9 +141,8 @@ function renderToggleSys(div) { } } -function topologyActionJson(id,name,status,msgTimeout, uiActionsEnabled) { +function topologyActionJson(id,name,status,msgTimeout) { var jsonData = {}; - jsonData["uiActionsEnabled"] = uiActionsEnabled; jsonData["id"] = id; jsonData["name"] = name; jsonData["msgTimeout"] = msgTimeout; diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index 805b6b46c37..cadec38bda4 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -91,7 +91,7 @@

Topology summary

var topologyActions = $("#topology-actions"); var topologyVisualization = $("#topology-visualization") var formattedConfig = formatConfigData(response["configuration"]); - var buttonJsonData = topologyActionJson(response["id"],response["name"],response["status"],response["msgTimeout"], response["uiActionsEnabled"]); + var buttonJsonData = topologyActionJson(response["id"],response["name"],response["status"],response["msgTimeout"]); $.get("/templates/topology-page-template.html", function(template) { topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response)); topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData)); From 02f6db5783f75a88ba305430991f6fda0bfc7b1a Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 22 Sep 2014 11:16:45 -0700 Subject: [PATCH 59/61] STORM-502. storm native worker-launcher tests fail. --- .../src/native/worker-launcher/test/test-worker-launcher.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/native/worker-launcher/test/test-worker-launcher.c b/storm-core/src/native/worker-launcher/test/test-worker-launcher.c index 412e9228105..7294511cbf9 100644 --- a/storm-core/src/native/worker-launcher/test/test-worker-launcher.c +++ b/storm-core/src/native/worker-launcher/test/test-worker-launcher.c @@ -97,7 +97,7 @@ int write_config_file(char *file_name) { return EXIT_FAILURE; } fprintf(file, "banned.users=bannedUser\n"); - fprintf(file, "min.user.id=1000\n"); + fprintf(file, "min.user.id=%d\n",getuid()); fclose(file); return 0; } From 11e6fc9821d7d7e676d573836c08272ce18385f2 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 29 Sep 2014 13:57:07 -0700 Subject: [PATCH 60/61] STORM-509. Make groups checking specific for SimpleACLAuthorizer. --- storm-core/src/jvm/backtype/storm/Config.java | 13 +++++++++--- .../auth/authorizer/SimpleACLAuthorizer.java | 21 ++++++++++++------- .../storm/security/auth/auth_test.clj | 4 ++-- 3 files changed, 26 insertions(+), 12 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index a38b51e910e..48e182275da 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -115,7 +115,7 @@ public class Config extends HashMap { */ public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate"; public static final Object STORM_META_SERIALIZATION_DELEGATE_SCHEMA = String.class; - + /** * A list of hosts of ZooKeeper servers used to manage the cluster. */ @@ -769,7 +769,7 @@ public class Config extends HashMap { * The jvm opts provided to workers launched by this supervisor. All "%ID%", "%WORKER-ID%", "%TOPOLOGY-ID%" * and "%WORKER-PORT%" substrings are replaced with: * %ID% -> port (for backward compatibility), - * %WORKER-ID% -> worker-id, + * %WORKER-ID% -> worker-id, * %TOPOLOGY-ID% -> topology-id, * %WORKER-PORT% -> port. */ @@ -828,6 +828,13 @@ public class Config extends HashMap { public static final String TOPOLOGY_USERS = "topology.users"; public static final Object TOPOLOGY_USERS_SCHEMA = ConfigValidation.StringsValidator; + /** + * A list of groups 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_GROUPS = "topology.groups"; + public static final Object TOPOLOGY_GROUPS_SCHEMA = ConfigValidation.StringsValidator; + /** * True if Storm should timeout messages or not. Defaults to true. This is meant to be used * in unit tests to prevent tuples from being accidentally timed out during the test. @@ -1304,7 +1311,7 @@ public static void registerSerialization(Map conf, Class klass, Class serializerClass) { registerSerialization(this, klass, serializerClass); } - + public static void registerMetricsConsumer(Map conf, Class klass, Object argument, long parallelismHint) { HashMap m = new HashMap(); m.put("class", klass.getCanonicalName()); diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java index 0b867e8db05..daef95626c4 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java @@ -49,7 +49,7 @@ public class SimpleACLAuthorizer implements IAuthorizer { protected Set _admins; protected Set _supervisors; protected IPrincipalToLocal _ptol; - protected IGroupMappingServiceProvider _groups; + protected IGroupMappingServiceProvider _groupMappingProvider; /** * Invoked once immediately after construction * @param conf Storm configuration @@ -65,8 +65,9 @@ public void prepare(Map conf) { if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) { _supervisors.addAll((Collection)conf.get(Config.NIMBUS_SUPERVISOR_USERS)); } + _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); - _groups = AuthUtils.GetGroupMappingServiceProviderPlugin(conf); + _groupMappingProvider = AuthUtils.GetGroupMappingServiceProviderPlugin(conf); } /** @@ -107,12 +108,18 @@ public boolean permit(ReqContext context, String operation, Map topology_conf) { if (topoUsers.contains(principal) || topoUsers.contains(user)) { return true; } - if(_groups != null) { + + Set topoGroups = new HashSet(); + if (topology_conf.containsKey(Config.TOPOLOGY_GROUPS)) { + topoGroups.addAll((Collection)topology_conf.get(Config.TOPOLOGY_GROUPS)); + } + + if(_groupMappingProvider != null && topoGroups.size() > 0) { + System.out.println("hello in if"); try { - String topologySubmitterUser = (String) topology_conf.get(Config.TOPOLOGY_SUBMITTER_USER); - Set userGroups = _groups.getGroups(user); - Set topoUserGroups = _groups.getGroups(topologySubmitterUser); - for (String tgroup : topoUserGroups) { + Set userGroups = _groupMappingProvider.getGroups(user); + for (String tgroup : topoGroups) { + System.out.println("tgroup "+tgroup); if(userGroups.contains(tgroup)) return true; } diff --git a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj index 12411e795eb..a01cac34683 100644 --- a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj +++ b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj @@ -284,8 +284,8 @@ user-name (System/getProperty "user.name")] (.prepare groups cluster-conf) (>= 0 (.size (.getGroups groups user-name))) - (>= 0 (.size (.getGroups groups "userDoesNotExist"))) - (>= 0 (.size (.getGroups groups nil))))) + (= 0 (.size (.getGroups groups "userDoesNotExist"))) + (= 0 (.size (.getGroups groups nil))))) (deftest simple-acl-same-user-auth-test (let [cluster-conf (merge (read-storm-config) From 0f6719d4303cae6fc8521b802bf86f077278e160 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 30 Sep 2014 11:43:37 -0700 Subject: [PATCH 61/61] STORM-509. Make groups checking specific for SimpleACLAuthorizer. --- .../storm/security/auth/authorizer/SimpleACLAuthorizer.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java index daef95626c4..e8425bf18e1 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java @@ -115,11 +115,9 @@ public boolean permit(ReqContext context, String operation, Map topology_conf) { } if(_groupMappingProvider != null && topoGroups.size() > 0) { - System.out.println("hello in if"); try { Set userGroups = _groupMappingProvider.getGroups(user); for (String tgroup : topoGroups) { - System.out.println("tgroup "+tgroup); if(userGroups.contains(tgroup)) return true; }