From 00c8658f1d5c70e2886d8f605e2d06d9086d898f Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 20 Jan 2016 17:04:06 -0500 Subject: [PATCH 001/387] add STORM-1406 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index df24acf23b2..64821be52db 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1406: Add MQTT Support * STORM-1466: Move the org.apache.thrift7 namespace to something correct/sensible * STORM-1470: Applies shading to hadoop-auth, cleaner exclusions * STORM-1467: Switch apache-rat plugin off by default, but enable for Travis-CI From bb503d287175b4d536426392d05ff26984768cfb Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 20 Jan 2016 17:16:14 -0500 Subject: [PATCH 002/387] add STORM-1452 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index fdad3da824e..75086d7937b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1452: Fixes profiling/debugging out of the box * STORM-1406: Add MQTT Support * STORM-1481: avoid Math.abs(Integer) get a negative value * STORM-1473: enable log search for daemon logs From e861cfe6d4fa3e8a6e9ad2da0a7beae33f87ceb4 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 21 Jan 2016 07:18:31 +0900 Subject: [PATCH 003/387] add STORM-1482 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index fdad3da824e..2eeafbe0dbb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,7 @@ * STORM-1481: avoid Math.abs(Integer) get a negative value * STORM-1473: enable log search for daemon logs * STORM-1472: Fix the errorTime bug and show the time to be readable + * STORM-1482: add missing 'break' for RedisStoreBolt * STORM-1466: Move the org.apache.thrift7 namespace to something correct/sensible * STORM-1470: Applies shading to hadoop-auth, cleaner exclusions * STORM-1467: Switch apache-rat plugin off by default, but enable for Travis-CI From 86884e7bdefccbfdf18bf2a85eecade3f8a39eed Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 21 Jan 2016 09:54:49 +0900 Subject: [PATCH 004/387] Sync CHANGELOG.md with master except 2.0.0 --- CHANGELOG.md | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index db784318dac..7d07f04f31c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,8 @@ ## 1.0.0 * STORM-1452: Fixes profiling/debugging out of the box * STORM-1406: Add MQTT Support - * STORM-1481: avoid Math.abs(Integer) get a negative value * STORM-1473: enable log search for daemon logs * STORM-1472: Fix the errorTime bug and show the time to be readable - * STORM-1482: add missing 'break' for RedisStoreBolt * STORM-1466: Move the org.apache.thrift7 namespace to something correct/sensible * STORM-1470: Applies shading to hadoop-auth, cleaner exclusions * STORM-1467: Switch apache-rat plugin off by default, but enable for Travis-CI @@ -117,7 +115,6 @@ * STORM-1182: Removing and wrapping some exceptions in ConfigValidation to make code cleaner * STORM-1134. Windows: Fix log4j config. * STORM-1127: allow for boolean arguments (Flux) - * STORM-1180: FLUX logo wasn't appearing quite right * STORM-1138: Storm-hdfs README should be updated with Avro Bolt information * STORM-1154: SequenceFileBolt needs unit tests * STORM-162: Load Aware Shuffle Grouping @@ -241,7 +238,15 @@ * STORM-1144: Display requested and assigned cpu/mem resources for schedulers in UI * STORM-1217: making small fixes in RAS -## 0.10.0-beta2 +## 0.10.1 + + * STORM-1481: avoid Math.abs(Integer) get a negative value + * STORM-1121: Deprecate test only configuraton nimbus.reassign + * STORM-1180: FLUX logo wasn't appearing quite right + * STORM-1482: add missing 'break' for RedisStoreBolt + +## 0.10.0 + * STORM-1096: Fix some issues with impersonation on the UI * STORM-581: Add rebalance params to Storm REST API * STORM-1108: Fix NPE in simulated time * STORM-1099: Fix worker childopts as arraylist of strings @@ -276,8 +281,6 @@ * STORM-742: Let ShellBolt treat all messages to update heartbeat * STORM-992: A bug in the timer.clj might cause unexpected delay to schedule new event -## 0.10.0 - * STORM-1096: Fix some issues with impersonation on the UI ## 0.10.0-beta1 * STORM-873: Flux does not handle diamond topologies From 456c134e40532b6ec40f4e456e82c3c161daf097 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 22 Jan 2016 10:16:11 +0900 Subject: [PATCH 005/387] add STORM-1450 to CHANGELOG.md (for 1.0.0) --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7d07f04f31c..e2db1835174 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1450: Fix minor bugs and refactor code in ResourceAwareScheduler * STORM-1452: Fixes profiling/debugging out of the box * STORM-1406: Add MQTT Support * STORM-1473: enable log search for daemon logs From 9500cc3cefb084ab705460cf93eb41350bfd3ecd Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 22 Jan 2016 14:34:17 -0500 Subject: [PATCH 006/387] reset pom version to 1.0.0-SNAPSHOT --- examples/storm-starter/pom.xml | 2 +- external/flux/flux-core/pom.xml | 2 +- external/flux/flux-examples/pom.xml | 2 +- external/flux/flux-wrappers/pom.xml | 2 +- external/flux/pom.xml | 2 +- external/sql/pom.xml | 2 +- external/sql/storm-sql-core/pom.xml | 2 +- external/sql/storm-sql-kafka/pom.xml | 2 +- external/sql/storm-sql-runtime/pom.xml | 2 +- external/storm-cassandra/pom.xml | 2 +- external/storm-elasticsearch/pom.xml | 2 +- external/storm-eventhubs/pom.xml | 4 ++-- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-hive/pom.xml | 2 +- external/storm-jdbc/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- external/storm-metrics/pom.xml | 2 +- external/storm-mqtt/core/pom.xml | 2 +- external/storm-mqtt/examples/pom.xml | 2 +- external/storm-mqtt/pom.xml | 2 +- external/storm-redis/pom.xml | 2 +- external/storm-solr/pom.xml | 2 +- pom.xml | 2 +- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-buildtools/storm-maven-plugins/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- storm-multilang/javascript/pom.xml | 2 +- storm-multilang/python/pom.xml | 2 +- storm-multilang/ruby/pom.xml | 2 +- storm-rename-hack/pom.xml | 2 +- 33 files changed, 34 insertions(+), 34 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 1a7644af19d..a44e14c10f6 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/flux/flux-core/pom.xml b/external/flux/flux-core/pom.xml index 58dd925354f..538a37c3180 100644 --- a/external/flux/flux-core/pom.xml +++ b/external/flux/flux-core/pom.xml @@ -21,7 +21,7 @@ org.apache.storm flux - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml diff --git a/external/flux/flux-examples/pom.xml b/external/flux/flux-examples/pom.xml index 28d72392489..926f8d059c3 100644 --- a/external/flux/flux-examples/pom.xml +++ b/external/flux/flux-examples/pom.xml @@ -21,7 +21,7 @@ org.apache.storm flux - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml diff --git a/external/flux/flux-wrappers/pom.xml b/external/flux/flux-wrappers/pom.xml index 7d97a9fb688..c34a53aca0c 100644 --- a/external/flux/flux-wrappers/pom.xml +++ b/external/flux/flux-wrappers/pom.xml @@ -21,7 +21,7 @@ org.apache.storm flux - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml diff --git a/external/flux/pom.xml b/external/flux/pom.xml index 1fd1683c6ed..7b38161ee62 100644 --- a/external/flux/pom.xml +++ b/external/flux/pom.xml @@ -26,7 +26,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/sql/pom.xml b/external/sql/pom.xml index 148e6a4d28b..787bc897283 100644 --- a/external/sql/pom.xml +++ b/external/sql/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/sql/storm-sql-core/pom.xml b/external/sql/storm-sql-core/pom.xml index 0ffea80920a..6b62e88b9f0 100644 --- a/external/sql/storm-sql-core/pom.xml +++ b/external/sql/storm-sql-core/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../../pom.xml diff --git a/external/sql/storm-sql-kafka/pom.xml b/external/sql/storm-sql-kafka/pom.xml index 450611ec837..fb754e7dee2 100644 --- a/external/sql/storm-sql-kafka/pom.xml +++ b/external/sql/storm-sql-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../../pom.xml diff --git a/external/sql/storm-sql-runtime/pom.xml b/external/sql/storm-sql-runtime/pom.xml index ebad92bc4b3..464564720ad 100644 --- a/external/sql/storm-sql-runtime/pom.xml +++ b/external/sql/storm-sql-runtime/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../../pom.xml diff --git a/external/storm-cassandra/pom.xml b/external/storm-cassandra/pom.xml index 24b3e2a4e97..c6ce10b6229 100644 --- a/external/storm-cassandra/pom.xml +++ b/external/storm-cassandra/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-elasticsearch/pom.xml b/external/storm-elasticsearch/pom.xml index d369d14c1a8..91583cedf68 100644 --- a/external/storm-elasticsearch/pom.xml +++ b/external/storm-elasticsearch/pom.xml @@ -22,7 +22,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-eventhubs/pom.xml b/external/storm-eventhubs/pom.xml index 195b7fd2cd8..146d4d82218 100755 --- a/external/storm-eventhubs/pom.xml +++ b/external/storm-eventhubs/pom.xml @@ -21,12 +21,12 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml storm-eventhubs - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT jar storm-eventhubs EventHubs Storm Spout diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index cd04460e0fc..13d160da89d 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 65b8eb786c6..da4148afdf9 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-hive/pom.xml b/external/storm-hive/pom.xml index 465fba6569a..547ea28178f 100644 --- a/external/storm-hive/pom.xml +++ b/external/storm-hive/pom.xml @@ -22,7 +22,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-jdbc/pom.xml b/external/storm-jdbc/pom.xml index 2c8c5eb524b..89425ecd79e 100644 --- a/external/storm-jdbc/pom.xml +++ b/external/storm-jdbc/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 43b7796c597..1afd0fafb46 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-metrics/pom.xml b/external/storm-metrics/pom.xml index eaa2b0c1b4c..b460563c93d 100644 --- a/external/storm-metrics/pom.xml +++ b/external/storm-metrics/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-mqtt/core/pom.xml b/external/storm-mqtt/core/pom.xml index 8ef694972c0..ee16dae5e8e 100644 --- a/external/storm-mqtt/core/pom.xml +++ b/external/storm-mqtt/core/pom.xml @@ -26,7 +26,7 @@ org.apache.storm storm-mqtt-parent - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml diff --git a/external/storm-mqtt/examples/pom.xml b/external/storm-mqtt/examples/pom.xml index a261510cf83..ba4f2ec9e80 100644 --- a/external/storm-mqtt/examples/pom.xml +++ b/external/storm-mqtt/examples/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm-mqtt-parent - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml diff --git a/external/storm-mqtt/pom.xml b/external/storm-mqtt/pom.xml index f8fdb233539..5447c175545 100644 --- a/external/storm-mqtt/pom.xml +++ b/external/storm-mqtt/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-redis/pom.xml b/external/storm-redis/pom.xml index 530f010c588..4b7fb435ac8 100644 --- a/external/storm-redis/pom.xml +++ b/external/storm-redis/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/external/storm-solr/pom.xml b/external/storm-solr/pom.xml index d093ae8437f..db56f353aed 100644 --- a/external/storm-solr/pom.xml +++ b/external/storm-solr/pom.xml @@ -22,7 +22,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/pom.xml b/pom.xml index 75345db76e3..691a8cb6589 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index d58c61427b1..7eccb430402 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml diff --git a/storm-buildtools/storm-maven-plugins/pom.xml b/storm-buildtools/storm-maven-plugins/pom.xml index 2330db33c57..c65c4b2021d 100644 --- a/storm-buildtools/storm-maven-plugins/pom.xml +++ b/storm-buildtools/storm-maven-plugins/pom.xml @@ -22,7 +22,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 8de24612119..37da95dc65c 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT .. org.apache.storm diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 7504f43602f..ad23eb8a2b0 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 0907e5ceed9..83d7902ff11 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-multilang/javascript/pom.xml b/storm-multilang/javascript/pom.xml index 85ffbcac4a3..0659ceefd85 100644 --- a/storm-multilang/javascript/pom.xml +++ b/storm-multilang/javascript/pom.xml @@ -22,7 +22,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-multilang/python/pom.xml b/storm-multilang/python/pom.xml index c59aa2d4bdf..84a79234421 100644 --- a/storm-multilang/python/pom.xml +++ b/storm-multilang/python/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-multilang/ruby/pom.xml b/storm-multilang/ruby/pom.xml index 510b44cd06c..2588f09b0f7 100644 --- a/storm-multilang/ruby/pom.xml +++ b/storm-multilang/ruby/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-rename-hack/pom.xml b/storm-rename-hack/pom.xml index def50984c57..72a2acc9946 100644 --- a/storm-rename-hack/pom.xml +++ b/storm-rename-hack/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml From 3374a6c4b570f28b4bea20353a418c811663fbb2 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 22 Jan 2016 15:47:49 -0600 Subject: [PATCH 007/387] Merge branch 'wip-trident-javadoc' of https://github.com/ptgoetz/storm into STORM-1214 STORM-1214: add javadoc for Trident Streams and Operations --- .../jvm/org/apache/storm/trident/Stream.java | 156 ++++++++++++++++-- .../storm/trident/operation/Assembly.java | 20 +++ .../trident/operation/BaseOperation.java | 15 ++ .../storm/trident/operation/Filter.java | 24 +++ .../storm/trident/operation/Function.java | 68 ++++++++ .../storm/trident/operation/Operation.java | 20 +++ .../trident/operation/TridentCollector.java | 26 +++ .../trident/operation/builtin/Debug.java | 7 + .../trident/operation/builtin/FilterNull.java | 4 + .../trident/operation/builtin/FirstN.java | 5 + .../trident/operation/builtin/Negate.java | 28 ++++ 11 files changed, 356 insertions(+), 17 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/trident/Stream.java b/storm-core/src/jvm/org/apache/storm/trident/Stream.java index f9f6210ad05..fb2497a53bf 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/Stream.java +++ b/storm-core/src/jvm/org/apache/storm/trident/Stream.java @@ -57,6 +57,23 @@ import org.apache.storm.trident.state.StateUpdater; import org.apache.storm.trident.util.TridentUtils; +/** + * A Stream represents the core data model in Trident, and can be thought of as a "stream" of tuples that are processed + * as a series of small batches. A stream is partitioned accross the nodes in the cluster, and operations are + * applied to a stream in parallel accross each partition. + * + * There are five types of operations that can be performed on streams in Trident + * + * 1. **Partiton-Local Operations** - Operations that are applied locally to each partition and do not involve network + * transfer + * 2. **Repartitioning Operations** - Operations that change how tuples are partitioned across tasks(thus causing + * network transfer), but do not change the content of the stream. + * 3. **Aggregation Operations** - Operations that *may* repartition a stream (thus causing network transfer) + * 4. **Grouping Operations** - Operations that may repartition a stream on specific fields and group together tuples whose + * fields values are equal. + * 5. **Merge and Join Operations** - Operations that combine different streams together. + * + */ // TODO: need to be able to replace existing fields with the function fields (like Cascading Fields.REPLACE) public class Stream implements IAggregatableStream { Node _node; @@ -68,61 +85,160 @@ protected Stream(TridentTopology topology, String name, Node node) { _node = node; _name = name; } - + + /** + * Applies a label to the stream. Naming a stream will append the label to the name of the bolt(s) created by + * Trident and will be visible in the Storm UI. + * + * @param name - The label to apply to the stream + * @return + */ public Stream name(String name) { return new Stream(_topology, name, _node); } - + + /** + * Applies a parallelism hint to a stream. + * + * @param hint + * @return + */ public Stream parallelismHint(int hint) { _node.parallelismHint = hint; return this; } - + + /** + * Filters out fields from a stream, resulting in a Stream containing only the fields specified by `keepFields`. + * + * For example, if you had a Stream `mystream` containing the fields `["a", "b", "c","d"]`, calling" + * + * ```java + * mystream.project(new Fields("b", "d")) + * ``` + * + * would produce a stream containing only the fields `["b", "d"]`. + * + * + * @param keepFields The fields in the Stream to keep + * @return + */ public Stream project(Fields keepFields) { projectionValidation(keepFields); return _topology.addSourcedNode(this, new ProcessorNode(_topology.getUniqueStreamId(), _name, keepFields, new Fields(), new ProjectedProcessor(keepFields))); } + /** + * ## Grouping Operation + * + * @param fields + * @return + */ public GroupedStream groupBy(Fields fields) { projectionValidation(fields); return new GroupedStream(this, fields); } - + + /** + * ## Repartitioning Operation + * + * @param fields + * @return + */ public Stream partitionBy(Fields fields) { projectionValidation(fields); return partition(Grouping.fields(fields.toList())); } - + + /** + * ## Repartitioning Operation + * + * @param partitioner + * @return + */ public Stream partition(CustomStreamGrouping partitioner) { return partition(Grouping.custom_serialized(Utils.javaSerialize(partitioner))); } - + + /** + * ## Repartitioning Operation + * + * Use random round robin algorithm to evenly redistribute tuples across all target partitions + * + * @return + */ public Stream shuffle() { return partition(Grouping.shuffle(new NullStruct())); } + /** + * ## Repartitioning Operation + * + * Use random round robin algorithm to evenly redistribute tuples across all target partitions, with a preference + * for local tasks. + * + * @return + */ public Stream localOrShuffle() { return partition(Grouping.local_or_shuffle(new NullStruct())); } + + + /** + * ## Repartitioning Operation + * + * All tuples are sent to the same partition. The same partition is chosen for all batches in the stream. + * @return + */ public Stream global() { // use this instead of storm's built in one so that we can specify a singleemitbatchtopartition // without knowledge of storm's internals return partition(new GlobalGrouping()); } - + + /** + * ## Repartitioning Operation + * + * All tuples in the batch are sent to the same partition. Different batches in the stream may go to different + * partitions. + * + * @return + */ public Stream batchGlobal() { // the first field is the batch id return partition(new IndexHashGrouping(0)); } - + + /** + * ## Repartitioning Operation + * + * Every tuple is replicated to all target partitions. This can useful during DRPC – for example, if you need to do + * a stateQuery on every partition of data. + * + * @return + */ public Stream broadcast() { return partition(Grouping.all(new NullStruct())); } - + + /** + * ## Repartitioning Operation + * + * @return + */ public Stream identityPartition() { return partition(new IdentityGrouping()); } - + + /** + * ## Repartitioning Operation + * + * This method takes in a custom partitioning function that implements + * {@link org.apache.storm.grouping.CustomStreamGrouping} + * + * @param grouping + * @return + */ public Stream partition(Grouping grouping) { if(_node instanceof PartitionNode) { return each(new Fields(), new TrueFilter()).partition(grouping); @@ -130,22 +246,28 @@ public Stream partition(Grouping grouping) { return _topology.addSourcedNode(this, new PartitionNode(_node.streamId, _name, getOutputFields(), grouping)); } } - + + /** + * Applies an `Assembly` to this `Stream`. + * + * @see org.apache.storm.trident.operation.Assembly + * @param assembly + * @return + */ public Stream applyAssembly(Assembly assembly) { return assembly.apply(this); } - + @Override public Stream each(Fields inputFields, Function function, Fields functionFields) { projectionValidation(inputFields); return _topology.addSourcedNode(this, new ProcessorNode(_topology.getUniqueStreamId(), - _name, - TridentUtils.fieldsConcat(getOutputFields(), functionFields), - functionFields, - new EachProcessor(inputFields, function))); + _name, + TridentUtils.fieldsConcat(getOutputFields(), functionFields), + functionFields, + new EachProcessor(inputFields, function))); } - //creates brand new tuples with brand new fields @Override public Stream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields) { diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/Assembly.java b/storm-core/src/jvm/org/apache/storm/trident/operation/Assembly.java index 0d55804d24b..c74a9685544 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/Assembly.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/Assembly.java @@ -20,6 +20,26 @@ import org.apache.storm.trident.Stream; +/** + * The `Assembly` interface provides a means to encapsulate logic applied to a {@link org.apache.storm.trident.Stream}. + * + * Usage: + * + * ```java + * Stream mystream = ...; + * Stream assemblyStream = mystream.applyAssembly(myAssembly); + * ``` + * + * @see org.apache.storm.trident.Stream + * @see org.apache.storm.trident.operation.builtin.FirstN + * + */ public interface Assembly { + /** + * Applies the `Assembly` to a given {@link org.apache.storm.trident.Stream} + * + * @param input + * @return + */ Stream apply(Stream input); } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/BaseOperation.java b/storm-core/src/jvm/org/apache/storm/trident/operation/BaseOperation.java index 4098e008cee..f1cf80931ac 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/BaseOperation.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/BaseOperation.java @@ -19,12 +19,27 @@ import java.util.Map; +/** + * Convenience implementation of the {@link org.apache.storm.trident.operation.Operation} interface. + * + * Provides no-op implementations of the `prepare()` and `cleanup()` methods. + */ public class BaseOperation implements Operation { + /** + * No-op implementation. + * @param conf the Storm configuration map + * @param context the operation context which provides information such as the number of partitions in the stream, + * and the current partition index. It also provides methods for registering operation-specific + * metrics. + */ @Override public void prepare(Map conf, TridentOperationContext context) { } + /** + * No-op implemnation. + */ @Override public void cleanup() { } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/Filter.java b/storm-core/src/jvm/org/apache/storm/trident/operation/Filter.java index d8ab95a55c6..7673ca20050 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/Filter.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/Filter.java @@ -19,6 +19,30 @@ import org.apache.storm.trident.tuple.TridentTuple; +import java.util.Map; + +/** + * Filters take in a tuple as input and decide whether or not to keep that tuple or not. + * + * If the `isKeep()` method of a Filter returns `false` for a tuple, that tuple will be filtered out of the Stream + * + * + * ### Configuration + * If your `Filter` implementation has configuration requirements, you will typically want to extend + * {@link org.apache.storm.trident.operation.BaseFilter} and override the + * {@link org.apache.storm.trident.operation.Operation#prepare(Map, TridentOperationContext)} method to perform your custom + * initialization. + + * + * @see org.apache.storm.trident.Stream + */ public interface Filter extends EachOperation { + + /** + * Determines if a tuple should be filtered out of a stream + * + * @param tuple the tuple being evaluated + * @return `false` to drop the tuple, `true` to keep the tuple + */ boolean isKeep(TridentTuple tuple); } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/Function.java b/storm-core/src/jvm/org/apache/storm/trident/operation/Function.java index 08b16801b12..b33a440f4a9 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/Function.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/Function.java @@ -19,6 +19,74 @@ import org.apache.storm.trident.tuple.TridentTuple; +import java.util.Map; + +/** + * A function takes in a set of input fields and emits zero or more tuples as output. The fields of the output tuple + * are appended to the original input tuple in the stream. If a function emits no tuples, the original input tuple is + * filtered out. Otherwise, the input tuple is duplicated for each output tuple. + * + * For example, if you have the following function: + * + * ```java + * public class MyFunction extends BaseFunction { + * public void execute(TridentTuple tuple, TridentCollector collector) { + * for(int i=0; i < tuple.getInteger(0); i++) { + * collector.emit(new Values(i)); + * } + * } + * } + * + * ``` + * + * Now suppose you have a stream in the variable `mystream` with the fields `["a", "b", "c"]` with the following tuples: + * + * ``` + * [1, 2, 3] + * [4, 1, 6] + * [3, 0, 8] + * ``` + * If you had the following code in your topology definition: + * + * ```java + * mystream.each(new Fields("b"), new MyFunction(), new Fields("d"))) + * ``` + * + * The resulting tuples would have the fields `["a", "b", "c", "d"]` and look like this: + * + * ``` + * [1, 2, 3, 0] + * [1, 2, 3, 1] + * [4, 1, 6, 0] + * ``` + * + * In this case, the parameter `new Fields("b")` tells Trident that you would like to select the field "b" as input + * to the function, and that will be the only field in the Tuple passed to the `execute()` method. The value of "b" in + * the first tuple (2) causes the for loop to execute twice, so 2 tuples are emitted. similarly the second tuple causes + * one tuple to be emitted. For the third tuple, the value of 0 causes the `for` loop to be skipped, so nothing is + * emitted and the incoming tuple is filtered out of the stream. + * + * ### Configuration + * If your `Function` implementation has configuration requirements, you will typically want to extend + * {@link org.apache.storm.trident.operation.BaseFunction} and override the + * {@link org.apache.storm.trident.operation.Operation#prepare(Map, TridentOperationContext)} method to perform your custom + * initialization. + * + * ### Performance Considerations + * Because Trident Functions perform logic on individual tuples -- as opposed to batches -- it is advisable + * to avoid expensive operations such as database operations in a Function, if possible. For data store interactions + * it is better to use a {@link org.apache.storm.trident.state.State} or + * {@link org.apache.storm.trident.state.QueryFunction} implementation since Trident states operate on batch partitions + * and can perform bulk updates to a database. + * + * + */ public interface Function extends EachOperation { + /** + * Performs the function logic on an individual tuple and emits 0 or more tuples. + * + * @param tuple The incoming tuple + * @param collector A collector instance that can be used to emit tuples + */ void execute(TridentTuple tuple, TridentCollector collector); } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/Operation.java b/storm-core/src/jvm/org/apache/storm/trident/operation/Operation.java index bf3d4d090d5..878a5c2aee7 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/Operation.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/Operation.java @@ -20,7 +20,27 @@ import java.io.Serializable; import java.util.Map; +/** + * Parent interface for Trident `Filter`s and `Function`s. + * + * `Operation` defines two lifecycle methods for Trident components. The `prepare()` method is called once when the + * `Operation` is first initialized. The `cleanup()` method is called in local mode when the local cluster is + * being shut down. In distributed mode, the `cleanup()` method is not guaranteed to be called in every situation, but + * Storm will make a best effort call `cleanup()` whenever possible. + */ public interface Operation extends Serializable { + /** + * Called when the `Operation` is first initialized. + * @param conf the Storm configuration map + * @param context the operation context which provides information such as the number of partitions in the stream, + * and the current partition index. It also provides methods for registering operation-specific + * metrics. + * @see org.apache.storm.trident.operation.TridentOperationContext + */ void prepare(Map conf, TridentOperationContext context); + + /** + * When running in local mode, called when the local cluster is being shut down. + */ void cleanup(); } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/TridentCollector.java b/storm-core/src/jvm/org/apache/storm/trident/operation/TridentCollector.java index bc023890188..7392c9c584c 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/TridentCollector.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/TridentCollector.java @@ -20,7 +20,33 @@ import java.util.List; +/** + * Interface for publishing tuples to a stream and reporting exceptions (to be displayed in Storm UI). + * + * Trident components that have the ability to emit tuples to a stream are passed an instance of this + * interface. + * + * For example, to emit a new tuple to a stream, you would do something like the following: + * + * ```java + * collector.emit(new Values("a", "b", "c")); + * ``` + * @see org.apache.storm.trident.Stream + * @see org.apache.storm.tuple.Values + */ public interface TridentCollector { + /** + * Emits a tuple to a Stream + * @param values a list of values of which the tuple will be composed + */ void emit(List values); + + /** + * Reports an error. The corresponding stack trace will be visible in the Storm UI. + * + * Note that calling this method does not alter the processing of a batch. To explicitly fail a batch and trigger + * a replay, components should throw {@link org.apache.storm.topology.FailedException}. + * @param t The instance of the error (Throwable) being reported. + */ void reportError(Throwable t); } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Debug.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Debug.java index 91a32933566..07c5ae4592f 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Debug.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Debug.java @@ -20,6 +20,9 @@ import org.apache.storm.trident.operation.BaseFilter; import org.apache.storm.trident.tuple.TridentTuple; +/** + * Filter for debugging purposes. The `isKeep()` method simply prints the tuple to `System.out` and returns `true`. + */ public class Debug extends BaseFilter { private final String name; @@ -27,6 +30,10 @@ public Debug() { name = "DEBUG: "; } + /** + * Creates a `Debug` filter with a string identifier. + * @param name + */ public Debug(String name) { this.name = "DEBUG(" + name + "): "; } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/FilterNull.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/FilterNull.java index 40479366239..5567e4ccbf2 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/FilterNull.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/FilterNull.java @@ -20,6 +20,10 @@ import org.apache.storm.trident.operation.BaseFilter; import org.apache.storm.trident.tuple.TridentTuple; +/** + * Simple `Filter` implementation that filters out any tuples that have fields with a value of `null`. + * + */ public class FilterNull extends BaseFilter { @Override public boolean isKeep(TridentTuple tuple) { diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/FirstN.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/FirstN.java index dd719dc3c2f..5db6f9d201c 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/FirstN.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/FirstN.java @@ -28,6 +28,11 @@ import org.apache.storm.trident.tuple.TridentTuple; +/** + * + * An {@link org.apache.storm.trident.operation.Assembly} implementation + * + */ public class FirstN implements Assembly { Aggregator _agg; diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Negate.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Negate.java index 9269aee90ad..32888adf39b 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Negate.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Negate.java @@ -22,6 +22,34 @@ import org.apache.storm.trident.operation.TridentOperationContext; import org.apache.storm.trident.tuple.TridentTuple; +/** + * A `Filter` implementation that inverts another delegate `Filter`. + * + * The `Negate.isKeep()` method simply returns the opposite of the delegate's `isKeep()` method: + * + * ```java + * public boolean isKeep(TridentTuple tuple) { + * return !this.delegate.isKeep(tuple); + * } + * ``` + * + * The `Negate` filter is useful for dividing a Stream in two based on some boolean condition. + * + * Suppose we had a Stream named `userStream` containing information about users, and a custom `Filter` implementation, + * `RegisteredUserFilter` that filtered out unregistered users. We could divide the `userStream` Stream into two + * separate Streams -- one for registered users, and one for unregistered users -- by doing the following: + * + * ```java + * Stream userStream = ... + * + * Filter registeredFilter = new ResisteredUserFilter(); + * Filter unregisteredFilter = new Negate(registeredFilter); + * + * Stream registeredUserStream = userStream.each(userStream.getOutputFields(), registeredFilter); + * Stream unregisteredUserStream = userStream.each(userStream.getOutputFields(), unregisteredFilter); + * ``` + * + */ public class Negate implements Filter { Filter _delegate; From 1305100949583ee5b627bd7ec1a3706060b6fdd9 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 22 Jan 2016 15:48:17 -0600 Subject: [PATCH 008/387] Added STORM-1214 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e2db1835174..0ef5f44fd9a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1214: add javadoc for Trident Streams and Operations * STORM-1450: Fix minor bugs and refactor code in ResourceAwareScheduler * STORM-1452: Fixes profiling/debugging out of the box * STORM-1406: Add MQTT Support From 819917fde32fc029d32cd736fa95bba686e1563a Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 22 Jan 2016 16:06:16 -0600 Subject: [PATCH 009/387] Merge branch 'STORM-1486' of https://github.com/arunmahadevan/storm into STORM-1486 STORM-1486: Fix storm-kafa documentation --- external/storm-kafka/README.md | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/external/storm-kafka/README.md b/external/storm-kafka/README.md index 3a86cf0b77f..5a34b55baa7 100644 --- a/external/storm-kafka/README.md +++ b/external/storm-kafka/README.md @@ -93,26 +93,33 @@ The KafkaConfig class also has bunch of public variables that controls your appl Most of them are self explanatory except MultiScheme. ###MultiScheme -MultiScheme is an interface that dictates how the byte[] consumed from Kafka gets transformed into a storm tuple. It +MultiScheme is an interface that dictates how the ByteBuffer consumed from Kafka gets transformed into a storm tuple. It also controls the naming of your output field. ```java - public Iterable> deserialize(byte[] ser); + public Iterable> deserialize(ByteBuffer ser); public Fields getOutputFields(); ``` -The default `RawMultiScheme` just takes the `byte[]` and returns a tuple with `byte[]` as is. The name of the outputField is "bytes". There are alternative implementations like `SchemeAsMultiScheme` and `KeyValueSchemeAsMultiScheme` which can convert the `byte[]` to `String`. +The default `RawMultiScheme` just takes the `ByteBuffer` and returns a tuple with the ByteBuffer converted to a `byte[]`. The name of the outputField is "bytes". There are alternative implementations like `SchemeAsMultiScheme` and `KeyValueSchemeAsMultiScheme` which can convert the `ByteBuffer` to `String`. There is also an extension of `SchemeAsMultiScheme`, `MessageMetadataSchemeAsMultiScheme`, -which has an additional deserialize method that accepts the message `byte[]` in addition to the `Partition` and `offset` associated with the message. +which has an additional deserialize method that accepts the message `ByteBuffer` in addition to the `Partition` and `offset` associated with the message. ```java -public Iterable> deserializeMessageWithMetadata(byte[] message, Partition partition, long offset) +public Iterable> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset) ``` This is useful for auditing/replaying messages from arbitrary points on a Kafka topic, saving the partition and offset of each message of a discrete stream instead of persisting the entire message. +#### Version incompatibility +In Storm versions prior to 1.0, the MultiScheme methods accepted a `byte[]` instead of `ByteBuffer`. The `MultScheme` and the related +Scheme apis were changed in version 1.0 to accept a ByteBuffer instead of a byte[]. + +This means that pre 1.0 kafka spouts will not work with Storm versions 1.0 and higher. While running topologies in Storm version 1.0 +and higher, it must be ensured that the storm-kafka version is at least 1.0. Pre 1.0 shaded topology jars that bundles +storm-kafka classes must be rebuilt with storm-kafka version 1.0 for running in clusters with storm 1.0 and higher. ### Examples From ed72acc63b5d4138a5a411b602d235376017c3b6 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 22 Jan 2016 16:06:47 -0600 Subject: [PATCH 010/387] Added STORM-1486 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0ef5f44fd9a..56be926e878 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1486: Fix storm-kafa documentation * STORM-1214: add javadoc for Trident Streams and Operations * STORM-1450: Fix minor bugs and refactor code in ResourceAwareScheduler * STORM-1452: Fixes profiling/debugging out of the box From ae106d54ae05df334d0b76606fe39a1e3a028db8 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 22 Jan 2016 16:13:13 -0600 Subject: [PATCH 011/387] Merge branch 'STORM-1485' of https://github.com/kishorvpatil/incubator-storm into STORM-1485 STORM-1485: DRPC Connectivity Issues --- .../org/apache/storm/drpc/ReturnResults.java | 51 ++- .../netty/KerberosSaslNettyClient.java | 1 - .../netty/KerberosSaslNettyServer.java | 1 - .../apache/storm/messaging/netty/Login.java | 411 ++++++++++++++++++ .../storm/security/auth/TBackoffConnect.java | 10 +- .../storm/security/auth/ThriftClient.java | 4 +- .../kerberos/KerberosSaslTransportPlugin.java | 2 +- .../org/apache/storm/utils/DRPCClient.java | 2 + 8 files changed, 453 insertions(+), 29 deletions(-) create mode 100644 storm-core/src/jvm/org/apache/storm/messaging/netty/Login.java diff --git a/storm-core/src/jvm/org/apache/storm/drpc/ReturnResults.java b/storm-core/src/jvm/org/apache/storm/drpc/ReturnResults.java index 74c45610e0b..a9a5aa17b0a 100644 --- a/storm-core/src/jvm/org/apache/storm/drpc/ReturnResults.java +++ b/storm-core/src/jvm/org/apache/storm/drpc/ReturnResults.java @@ -83,35 +83,40 @@ public void execute(Tuple input) { client = _clients.get(server); } - try { - client.result(id, result); - _collector.ack(input); - } 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); - } - } - } 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); + + int retryCnt = 0; + int maxRetries = 3; + while (retryCnt < maxRetries) { + retryCnt++; + try { + client.result(id, result); + _collector.ack(input); + break; + } catch (AuthorizationException aze) { + LOG.error("Not authorized to return results to DRPC server", aze); + _collector.fail(input); + throw new RuntimeException(aze); + } catch (TException tex) { + if (retryCnt >= maxRetries) { + LOG.error("Failed to return results to DRPC server", tex); + _collector.fail(input); } + reconnectClient((DRPCInvocationsClient) client); } } } } + private void reconnectClient(DRPCInvocationsClient client) { + if (client instanceof DRPCInvocationsClient) { + try { + LOG.info("reconnecting... "); + client.reconnectClient(); //Blocking call + } catch (TException e2) { + LOG.error("Failed to connect to DRPC server", e2); + } + } + } @Override public void cleanup() { for(DRPCInvocationsClient c: _clients.values()) { diff --git a/storm-core/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java b/storm-core/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java index 53f193786ee..d53bb7c1c45 100644 --- a/storm-core/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java +++ b/storm-core/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java @@ -35,7 +35,6 @@ import javax.security.sasl.Sasl; import javax.security.sasl.SaslClient; import javax.security.sasl.SaslException; -import org.apache.zookeeper.Login; import org.apache.zookeeper.server.auth.KerberosName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/storm-core/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java b/storm-core/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java index 63e0dfa070e..72486ef597e 100644 --- a/storm-core/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java +++ b/storm-core/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java @@ -39,7 +39,6 @@ import javax.security.sasl.Sasl; import javax.security.sasl.SaslException; import javax.security.sasl.SaslServer; -import org.apache.zookeeper.Login; import org.apache.zookeeper.server.auth.KerberosName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/storm-core/src/jvm/org/apache/storm/messaging/netty/Login.java b/storm-core/src/jvm/org/apache/storm/messaging/netty/Login.java new file mode 100644 index 00000000000..718c8f37d60 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/messaging/netty/Login.java @@ -0,0 +1,411 @@ +/** + * 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 org.apache.storm.messaging.netty; + +/** + * This class is responsible for refreshing Kerberos credentials for + * logins for both Zookeeper client and server. + * See ZooKeeperSaslServer for server-side usage. + * See ZooKeeperSaslClient for client-side usage. + * This class is a copied from https://github.com/apache/zookeeper/blob/branch-3.4/src/java/main/org/apache/zookeeper/Login.java + * with the difference that refresh thread does not die. + */ + +import javax.security.auth.kerberos.KerberosPrincipal; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginContext; +import javax.security.auth.login.LoginException; +import javax.security.auth.callback.CallbackHandler; + +import org.apache.log4j.Logger; +import org.apache.zookeeper.Shell; +import org.apache.zookeeper.client.ZooKeeperSaslClient; +import javax.security.auth.kerberos.KerberosTicket; +import javax.security.auth.Subject; +import java.util.Date; +import java.util.Random; +import java.util.Set; + +public class Login { + Logger LOG = Logger.getLogger(Login.class); + public CallbackHandler callbackHandler; + + // Login will sleep until 80% of time from last refresh to + // ticket's expiry has been reached, at which time it will wake + // and try to renew the ticket. + private static final float TICKET_RENEW_WINDOW = 0.80f; + + /** + * Percentage of random jitter added to the renewal time + */ + private static final float TICKET_RENEW_JITTER = 0.05f; + + // Regardless of TICKET_RENEW_WINDOW setting above and the ticket expiry time, + // thread will not sleep between refresh attempts any less than 1 minute (60*1000 milliseconds = 1 minute). + // Change the '1' to e.g. 5, to change this to 5 minutes. + private static final long MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L; + + private Subject subject = null; + private Thread t = null; + private boolean isKrbTicket = false; + private boolean isUsingTicketCache = false; + private boolean isUsingKeytab = false; + + /** Random number generator */ + private static Random rng = new Random(); + + private LoginContext login = null; + private String loginContextName = null; + private String keytabFile = null; + private String principal = null; + + private long lastLogin = 0; + + /** + * Login constructor. The constructor starts the thread used + * to periodically re-login to the Kerberos Ticket Granting Server. + * @param loginContextName + * name of section in JAAS file that will be use to login. + * Passed as first param to javax.security.auth.login.LoginContext(). + * + * @param callbackHandler + * Passed as second param to javax.security.auth.login.LoginContext(). + * @throws javax.security.auth.login.LoginException + * Thrown if authentication fails. + */ + public Login(final String loginContextName, CallbackHandler callbackHandler) + throws LoginException { + this.callbackHandler = callbackHandler; + login = login(loginContextName); + this.loginContextName = loginContextName; + subject = login.getSubject(); + isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty(); + AppConfigurationEntry entries[] = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName); + for (AppConfigurationEntry entry: entries) { + // there will only be a single entry, so this for() loop will only be iterated through once. + if (entry.getOptions().get("useTicketCache") != null) { + String val = (String)entry.getOptions().get("useTicketCache"); + if (val.equals("true")) { + isUsingTicketCache = true; + } + } + if (entry.getOptions().get("keyTab") != null) { + keytabFile = (String)entry.getOptions().get("keyTab"); + isUsingKeytab = true; + } + if (entry.getOptions().get("principal") != null) { + principal = (String)entry.getOptions().get("principal"); + } + break; + } + + if (!isKrbTicket) { + // if no TGT, do not bother with ticket management. + return; + } + + // Refresh the Ticket Granting Ticket (TGT) periodically. How often to refresh is determined by the + // TGT's existing expiry date and the configured MIN_TIME_BEFORE_RELOGIN. For testing and development, + // you can decrease the interval of expiration of tickets (for example, to 3 minutes) by running : + // "modprinc -maxlife 3mins " in kadmin. + t = new Thread(new Runnable() { + public void run() { + LOG.info("TGT refresh thread started."); + while (true) { // renewal thread's main loop. if it exits from here, thread will exit. + KerberosTicket tgt = getTGT(); + long now = System.currentTimeMillis(); + long nextRefresh; + Date nextRefreshDate; + if (tgt == null) { + nextRefresh = now + MIN_TIME_BEFORE_RELOGIN; + nextRefreshDate = new Date(nextRefresh); + LOG.warn("No TGT found: will try again at " + nextRefreshDate); + } else { + nextRefresh = getRefreshTime(tgt); + long expiry = tgt.getEndTime().getTime(); + Date expiryDate = new Date(expiry); + if ((isUsingTicketCache) && (tgt.getEndTime().equals(tgt.getRenewTill()))) { + LOG.error("The TGT cannot be renewed beyond the next expiry date: " + expiryDate + "." + + "This process will not be able to authenticate new SASL connections after that " + + "time (for example, it will not be authenticate a new connection with a Zookeeper " + + "Quorum member). Ask your system administrator to either increase the " + + "'renew until' time by doing : 'modprinc -maxrenewlife " + principal + "' within " + + "kadmin, or instead, to generate a keytab for " + principal + ". Because the TGT's " + + "expiry cannot be further extended by refreshing, exiting refresh thread now."); + return; + } + // determine how long to sleep from looking at ticket's expiry. + // We should not allow the ticket to expire, but we should take into consideration + // MIN_TIME_BEFORE_RELOGIN. Will not sleep less than MIN_TIME_BEFORE_RELOGIN, unless doing so + // would cause ticket expiration. + if ((nextRefresh > expiry) || + ((now + MIN_TIME_BEFORE_RELOGIN) > expiry)) { + // expiry is before next scheduled refresh). + nextRefresh = now; + } else { + if (nextRefresh < (now + MIN_TIME_BEFORE_RELOGIN)) { + // next scheduled refresh is sooner than (now + MIN_TIME_BEFORE_LOGIN). + Date until = new Date(nextRefresh); + Date newuntil = new Date(now + MIN_TIME_BEFORE_RELOGIN); + LOG.warn("TGT refresh thread time adjusted from : " + until + " to : " + newuntil + " since " + + "the former is sooner than the minimum refresh interval (" + + MIN_TIME_BEFORE_RELOGIN / 1000 + " seconds) from now."); + } + nextRefresh = Math.max(nextRefresh, now + MIN_TIME_BEFORE_RELOGIN); + } + } + if (tgt != null && now > tgt.getEndTime().getTime()) { + if ((now - tgt.getEndTime().getTime()) < ( 10 * MIN_TIME_BEFORE_RELOGIN)) { + Date until = new Date(now + MIN_TIME_BEFORE_RELOGIN); + LOG.info("TGT already expired but giving additional 10 minutes past TGT expiry, refresh sleeping until: " + until.toString()); + try { + Thread.sleep(MIN_TIME_BEFORE_RELOGIN); + } catch (InterruptedException ie) { + LOG.warn("TGT renewal thread has been interrupted and will exit."); + return; + } + } else { + LOG.error("nextRefresh:" + new Date(nextRefresh) + " is in the past: exiting refresh thread. Check" + + " clock sync between this host and KDC - (KDC's clock is likely ahead of this host)." + + " Manual intervention will be required for this client to successfully authenticate." + + " Exiting worker!."); + Runtime.getRuntime().exit(-3); + } + } else if (now < nextRefresh) { + Date until = new Date(nextRefresh); + LOG.info("TGT refresh sleeping until: " + until.toString()); + try { + Thread.sleep(nextRefresh - now); + } catch (InterruptedException ie) { + LOG.warn("TGT renewal thread has been interrupted and will exit."); + return; + } + } + + if (isUsingTicketCache) { + String cmd = "/usr/bin/kinit"; + if (System.getProperty("zookeeper.kinit") != null) { + cmd = System.getProperty("zookeeper.kinit"); + } + String kinitArgs = "-R"; + int retry = 1; + while (retry >= 0) { + try { + LOG.debug("running ticket cache refresh command: " + cmd + " " + kinitArgs); + Shell.execCommand(cmd, kinitArgs); + break; + } catch (Exception e) { + if (retry > 0) { + --retry; + // sleep for 10 seconds + try { + Thread.sleep(10 * 1000); + } catch (InterruptedException ie) { + LOG.error("Interrupted while renewing TGT, exiting Login thread"); + return; + } + } else { + LOG.warn("Could not renew TGT due to problem running shell command: '" + cmd + + " " + kinitArgs + "'" + "; exception was:" + e + ". Exiting refresh thread.",e); + return; + } + } + } + } + try { + int retry = 1; + while (retry >= 0) { + try { + reLogin(); + break; + } catch (LoginException le) { + if (retry > 0) { + --retry; + // sleep for 10 seconds. + try { + Thread.sleep(10 * 1000); + } catch (InterruptedException e) { + LOG.error("Interrupted during login retry after LoginException:", le); + throw le; + } + } else { + LOG.error("Could not refresh TGT for principal: " + principal + ".", le); + } + } + } + } catch (LoginException le) { + LOG.error("Failed to refresh TGT: refresh thread exiting now.",le); + break; + } + } + } + }); + t.setName("Refresh-TGT"); + t.setDaemon(true); + } + + public void startThreadIfNeeded() { + // thread object 't' will be null if a refresh thread is not needed. + if (t != null) { + t.start(); + } + } + + public void shutdown() { + if ((t != null) && (t.isAlive())) { + t.interrupt(); + try { + t.join(); + } catch (InterruptedException e) { + LOG.warn("error while waiting for Login thread to shutdown: " + e); + } + } + } + + public Subject getSubject() { + return subject; + } + + public String getLoginContextName() { + return loginContextName; + } + + private synchronized LoginContext login(final String loginContextName) throws LoginException { + if (loginContextName == null) { + throw new LoginException("loginContext name (JAAS file section header) was null. " + + "Please check your java.security.login.auth.config (=" + + System.getProperty("java.security.login.auth.config") + + ") and your " + ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY + "(=" + + System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "Client") + ")"); + } + LoginContext loginContext = new LoginContext(loginContextName,callbackHandler); + loginContext.login(); + LOG.info("successfully logged in."); + return loginContext; + } + + // c.f. org.apache.hadoop.security.UserGroupInformation. + private long getRefreshTime(KerberosTicket tgt) { + long start = tgt.getStartTime().getTime(); + long expires = tgt.getEndTime().getTime(); + LOG.info("TGT valid starting at: " + tgt.getStartTime().toString()); + LOG.info("TGT expires: " + tgt.getEndTime().toString()); + long proposedRefresh = start + (long) ((expires - start) * + (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble()))); + if (proposedRefresh > expires) { + // proposedRefresh is too far in the future: it's after ticket expires: simply return now. + return System.currentTimeMillis(); + } + else { + return proposedRefresh; + } + } + + private synchronized KerberosTicket getTGT() { + Set tickets = subject.getPrivateCredentials(KerberosTicket.class); + for(KerberosTicket ticket: tickets) { + KerberosPrincipal server = ticket.getServer(); + if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) { + LOG.debug("Found tgt " + ticket + "."); + return ticket; + } + } + return null; + } + + private void sleepUntilSufficientTimeElapsed() { + long now = System.currentTimeMillis(); + if (now - getLastLogin() < MIN_TIME_BEFORE_RELOGIN ) { + LOG.warn("Not attempting to re-login since the last re-login was " + + "attempted less than " + (MIN_TIME_BEFORE_RELOGIN/1000) + " seconds"+ + " before."); + try { + Thread.sleep(MIN_TIME_BEFORE_RELOGIN - (now - getLastLogin())); + } catch (InterruptedException e) { + LOG.warn("TGT renewal thread has been interrupted and will exit."); + Runtime.getRuntime().exit(-2); + } + } + // register most recent relogin attempt + setLastLogin(System.currentTimeMillis()); + } + + /** + * Returns login object + * @return login + */ + private LoginContext getLogin() { + return login; + } + + /** + * Set the login object + * @param login + */ + private void setLogin(LoginContext login) { + this.login = login; + } + + /** + * Set the last login time. + * @param time the number of milliseconds since the beginning of time + */ + private void setLastLogin(long time) { + lastLogin = time; + } + + /** + * Get the time of the last login. + * @return the number of milliseconds since the beginning of time. + */ + private long getLastLogin() { + return lastLogin; + } + + /** + * Re-login a principal. This method assumes that {@link #login(String)} has happened already. + * @throws javax.security.auth.login.LoginException on a failure + */ + // c.f. HADOOP-6559 + private synchronized void reLogin() + throws LoginException { + if (!isKrbTicket) { + return; + } + LoginContext login = getLogin(); + if (login == null) { + throw new LoginException("login must be done first"); + } + sleepUntilSufficientTimeElapsed(); + LOG.info("Initiating logout for " + principal); + synchronized (Login.class) { + //clear up the kerberos state. But the tokens are not cleared! As per + //the Java kerberos login module code, only the kerberos credentials + //are cleared + login.logout(); + //login and also update the subject field of this instance to + //have the new credentials (pass it to the LoginContext constructor) + login = new LoginContext(loginContextName, getSubject()); + LOG.info("Initiating re-login for " + principal); + login.login(); + setLogin(login); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/TBackoffConnect.java b/storm-core/src/jvm/org/apache/storm/security/auth/TBackoffConnect.java index 688b13089c9..0ca9eadb017 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/TBackoffConnect.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/TBackoffConnect.java @@ -30,15 +30,21 @@ public class TBackoffConnect { private int _completedRetries = 0; private int _retryTimes; private StormBoundedExponentialBackoffRetry waitGrabber; + private boolean _retryForever = false; - public TBackoffConnect(int retryTimes, int retryInterval, int retryIntervalCeiling) { + public TBackoffConnect(int retryTimes, int retryInterval, int retryIntervalCeiling, boolean retryForever) { + _retryForever = retryForever; _retryTimes = retryTimes; waitGrabber = new StormBoundedExponentialBackoffRetry(retryInterval, retryIntervalCeiling, retryTimes); } + public TBackoffConnect(int retryTimes, int retryInterval, int retryIntervalCeiling) { + this(retryTimes, retryInterval, retryIntervalCeiling, false); + } + public TTransport doConnectWithRetry(ITransportPlugin transportPlugin, TTransport underlyingTransport, String host, String asUser) throws IOException { boolean connected = false; TTransport transportResult = null; @@ -71,6 +77,6 @@ private void retryNext(TTransportException ex) { } private boolean canRetry() { - return (_completedRetries < _retryTimes); + return _retryForever || (_completedRetries < _retryTimes); } } diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/ThriftClient.java b/storm-core/src/jvm/org/apache/storm/security/auth/ThriftClient.java index 1ea4a084211..0a8e5158af4 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/ThriftClient.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/ThriftClient.java @@ -36,6 +36,7 @@ public class ThriftClient { private Map _conf; private ThriftConnectionType _type; private String _asUser; + protected boolean _retryForever = false; public ThriftClient(Map storm_conf, ThriftConnectionType type, String host) { this(storm_conf, type, host, null, null, null); @@ -93,7 +94,8 @@ public synchronized void reconnect() { = 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))); + Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL_CEILING)), + _retryForever); _transport = connectionRetry.doConnectWithRetry(transportPlugin, socket, _host, _asUser); } catch (IOException ex) { throw new RuntimeException(ex); diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java index 86bb673717f..6f1c34637f4 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java @@ -33,13 +33,13 @@ import javax.security.auth.login.LoginException; import javax.security.sasl.Sasl; +import org.apache.storm.messaging.netty.Login; import org.apache.commons.lang.StringUtils; 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; diff --git a/storm-core/src/jvm/org/apache/storm/utils/DRPCClient.java b/storm-core/src/jvm/org/apache/storm/utils/DRPCClient.java index ec542ac3b28..7f83789d1ba 100644 --- a/storm-core/src/jvm/org/apache/storm/utils/DRPCClient.java +++ b/storm-core/src/jvm/org/apache/storm/utils/DRPCClient.java @@ -37,6 +37,7 @@ public class DRPCClient extends ThriftClient implements DistributedRPC.Iface { public DRPCClient(Map conf, String host, int port) throws TTransportException { this(conf, host, port, null); + _retryForever = true; } public DRPCClient(Map conf, String host, int port, Integer timeout) throws TTransportException { @@ -44,6 +45,7 @@ public DRPCClient(Map conf, String host, int port, Integer timeout) throws TTran this.host = host; this.port = port; this.client = new DistributedRPC.Client(_protocol); + _retryForever = true; } public String getHost() { From 9dbe2a0148e0ae026ddd1f934209e70a71b050d2 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 22 Jan 2016 16:16:10 -0600 Subject: [PATCH 012/387] Added STORM-1485 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 56be926e878..322730a4679 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1485: DRPC Connectivity Issues * STORM-1486: Fix storm-kafa documentation * STORM-1214: add javadoc for Trident Streams and Operations * STORM-1450: Fix minor bugs and refactor code in ResourceAwareScheduler From 5e6aaae1eace7806a38a74aa845e12c3cd01d332 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 22 Jan 2016 16:45:35 -0600 Subject: [PATCH 013/387] Merge branch 'STORM-1463' of https://github.com/satishd/storm into STORM-1463 STORM-1463: added file scehma to log4j config files for windows env --- bin/storm-config.cmd | 4 ++-- .../src/clj/org/apache/storm/daemon/supervisor.clj | 11 +++++++++-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/bin/storm-config.cmd b/bin/storm-config.cmd index cb1e203a57f..0d851f4cc5d 100644 --- a/bin/storm-config.cmd +++ b/bin/storm-config.cmd @@ -102,7 +102,7 @@ FOR /F "delims=" %%i in (%CMD_TEMP_FILE%) do ( @rem if not %STORM_LOG4J2_CONFIGURATION_DIR% == nil ( - set STORM_LOG4J2_CONFIGURATION_FILE="file://%STORM_LOG4J2_CONFIGURATION_DIR%\cluster.xml" + set STORM_LOG4J2_CONFIGURATION_FILE="file:///%STORM_LOG4J2_CONFIGURATION_DIR%\cluster.xml" ) @rem @@ -110,7 +110,7 @@ if not %STORM_LOG4J2_CONFIGURATION_DIR% == nil ( @rem if not defined STORM_LOG4J2_CONFIGURATION_FILE ( - set STORM_LOG4J2_CONFIGURATION_FILE="file://%STORM_HOME%\log4j2\cluster.xml" + set STORM_LOG4J2_CONFIGURATION_FILE="file:///%STORM_HOME%\log4j2\cluster.xml" ) "%JAVA%" -client -Dstorm.options= -Dstorm.conf.file= -cp "%CLASSPATH%" org.apache.storm.command.config_value java.library.path > %CMD_TEMP_FILE% diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj index 6e6ecc5509d..4bca23e1caf 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj @@ -1077,6 +1077,13 @@ topology-worker-environment (if-let [env (storm-conf TOPOLOGY-ENVIRONMENT)] (merge env {"LD_LIBRARY_PATH" jlp}) {"LD_LIBRARY_PATH" jlp}) + + log4j-configuration-file (str (if (.startsWith (System/getProperty "os.name") "Windows") + (if (.startsWith storm-log4j2-conf-dir "file:") + storm-log4j2-conf-dir + (str "file:///" storm-log4j2-conf-dir)) + storm-log4j2-conf-dir) + file-path-separator "worker.xml") command (concat [(java-cmd) "-cp" classpath topo-worker-logwriter-childopts @@ -1087,7 +1094,7 @@ (str "-Dworker.id=" worker-id) (str "-Dworker.port=" port) (str "-Dstorm.log.dir=" storm-log-dir) - (str "-Dlog4j.configurationFile=" storm-log4j2-conf-dir file-path-separator "worker.xml") + (str "-Dlog4j.configurationFile=" log4j-configuration-file) (str "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector") "org.apache.storm.LogWriter"] [(java-cmd) "-server"] @@ -1103,7 +1110,7 @@ (str "-Dstorm.options=" storm-options) (str "-Dstorm.log.dir=" storm-log-dir) (str "-Dlogging.sensitivity=" logging-sensitivity) - (str "-Dlog4j.configurationFile=" storm-log4j2-conf-dir file-path-separator "worker.xml") + (str "-Dlog4j.configurationFile=" log4j-configuration-file) (str "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector") (str "-Dstorm.id=" storm-id) (str "-Dworker.id=" worker-id) From da71bccc303445a9847921ea0304d0fd83c6f44b Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 22 Jan 2016 16:47:44 -0600 Subject: [PATCH 014/387] Added STORM-1463 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 322730a4679..8c32c9d3626 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1463: added file scehma to log4j config files for windows env * STORM-1485: DRPC Connectivity Issues * STORM-1486: Fix storm-kafa documentation * STORM-1214: add javadoc for Trident Streams and Operations From 974b1f03ccc42c63b8d114f4cc1a5a3fecebfcb8 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 25 Jan 2016 08:25:07 +0900 Subject: [PATCH 015/387] backport STORM-1484 / 1478 to 1.0.0, too --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8c32c9d3626..503a3124566 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,6 @@ ## 1.0.0 + * STORM-1484: ignore subproject .classpath & .project file + * STORM-1478: make bolts getComponentConfiguration method cleaner/simpler * STORM-1463: added file scehma to log4j config files for windows env * STORM-1485: DRPC Connectivity Issues * STORM-1486: Fix storm-kafa documentation From d76510be5cdcf654df21fc000465ef022b8a6d37 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 25 Jan 2016 08:28:53 +0900 Subject: [PATCH 016/387] add STORM-1499 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e7373de8a82..bf9bcec5cc6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,7 @@ ## 1.0.0 * STORM-1484: ignore subproject .classpath & .project file * STORM-1478: make bolts getComponentConfiguration method cleaner/simpler + * STORM-1499: fix wrong package name for storm trident * STORM-1463: added file scehma to log4j config files for windows env * STORM-1485: DRPC Connectivity Issues * STORM-1486: Fix storm-kafa documentation From 5646be4cbe4183550dca458b44b2eca7e6b582db Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 26 Jan 2016 08:13:54 +0900 Subject: [PATCH 017/387] add STORM-1496 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bf9bcec5cc6..4627f04cddb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1496: Nimbus periodically throws blobstore-related exception * STORM-1484: ignore subproject .classpath & .project file * STORM-1478: make bolts getComponentConfiguration method cleaner/simpler * STORM-1499: fix wrong package name for storm trident From 6a37ea5b4d2d10c8e677829ae25e8892db1c588d Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 27 Jan 2016 13:50:38 -0500 Subject: [PATCH 018/387] add STORM-1494 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4627f04cddb..4bb2055e483 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1494: Add link to supervisor log in Storm UI * STORM-1496: Nimbus periodically throws blobstore-related exception * STORM-1484: ignore subproject .classpath & .project file * STORM-1478: make bolts getComponentConfiguration method cleaner/simpler From 70f11942915a20e5703ef8362153a2130b042f03 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 27 Jan 2016 14:00:55 -0500 Subject: [PATCH 019/387] This closes #1028 From 4ff1ef066d683209bf80f9ad446e7650a2a83a0b Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 27 Jan 2016 14:05:57 -0500 Subject: [PATCH 020/387] add STORM-1176 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4bb2055e483..44f872eca2f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1176: Checkpoint window evaluated/expired state * STORM-1494: Add link to supervisor log in Storm UI * STORM-1496: Nimbus periodically throws blobstore-related exception * STORM-1484: ignore subproject .classpath & .project file From be4c77619f70575ab816d077ca2d7c3832fd1141 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 27 Jan 2016 16:04:09 -0500 Subject: [PATCH 021/387] add STORM-1503 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 35087188d6c..8ea316f4151 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1503] Worker should not crash on failure to send heartbeats to Pacemaker/ZK * STORM-1176: Checkpoint window evaluated/expired state * STORM-1494: Add link to supervisor log in Storm UI * STORM-1496: Nimbus periodically throws blobstore-related exception From bb6a3f60995cfe23f6a4d3aabd31026bd19d0fb6 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Sat, 30 Jan 2016 14:13:59 -0600 Subject: [PATCH 022/387] Merge branch 'nimbu-log-link' of https://github.com/redsanket/storm into STORM-1510 STORM-1510: Fix broken nimbus log link --- storm-core/src/clj/org/apache/storm/ui/core.clj | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj index d7269917cbc..cbea15a9e12 100644 --- a/storm-core/src/clj/org/apache/storm/ui/core.clj +++ b/storm-core/src/clj/org/apache/storm/ui/core.clj @@ -386,7 +386,7 @@ { "host" host "port" port - "nimbusLogLink" (nimbus-log-link host port) + "nimbusLogLink" (nimbus-log-link host) "status" "Offline" "version" "Not applicable" "nimbusUpTime" "Not applicable" From 937bc742c097e4e0093f5eb09637f45daa360d21 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Sat, 30 Jan 2016 14:15:53 -0600 Subject: [PATCH 023/387] Added STORM-1510 to Changelog --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8ea316f4151..990aab2a49c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,6 @@ ## 1.0.0 - * STORM-1503] Worker should not crash on failure to send heartbeats to Pacemaker/ZK + * STORM-1510: Fix broken nimbus log link + * STORM-1503: Worker should not crash on failure to send heartbeats to Pacemaker/ZK * STORM-1176: Checkpoint window evaluated/expired state * STORM-1494: Add link to supervisor log in Storm UI * STORM-1496: Nimbus periodically throws blobstore-related exception From ea9abeb2d67402cf82e80adf6498a30a38b3a0a4 Mon Sep 17 00:00:00 2001 From: Aaron Dossett Date: Tue, 2 Feb 2016 15:35:39 -0600 Subject: [PATCH 024/387] STORM-1518: Backport of STORM-1504 --- external/storm-hdfs/README.md | 15 +++- external/storm-hdfs/pom.xml | 22 +++++ .../hdfs/avro/AbstractAvroSerializer.java | 80 ++++++++++++++++++ .../storm/hdfs/avro/AvroSchemaRegistry.java | 28 +++++++ .../org/apache/storm/hdfs/avro/AvroUtils.java | 44 ++++++++++ .../hdfs/avro/ConfluentAvroSerializer.java | 83 +++++++++++++++++++ .../storm/hdfs/avro/FixedAvroSerializer.java | 67 +++++++++++++++ .../hdfs/avro/GenericAvroSerializer.java | 36 ++++++++ .../hdfs/bolt/AvroGenericRecordBolt.java | 4 - .../hdfs/avro/TestFixedAvroSerializer.java | 76 +++++++++++++++++ .../hdfs/avro/TestGenericAvroSerializer.java | 68 +++++++++++++++ .../test/resources/FixedAvroSerializer.config | 2 + pom.xml | 3 + 13 files changed, 522 insertions(+), 6 deletions(-) create mode 100644 external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AbstractAvroSerializer.java create mode 100644 external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AvroSchemaRegistry.java create mode 100644 external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AvroUtils.java create mode 100644 external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java create mode 100644 external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/FixedAvroSerializer.java create mode 100644 external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/GenericAvroSerializer.java create mode 100644 external/storm-hdfs/src/test/java/org/apache/storm/hdfs/avro/TestFixedAvroSerializer.java create mode 100644 external/storm-hdfs/src/test/java/org/apache/storm/hdfs/avro/TestGenericAvroSerializer.java create mode 100644 external/storm-hdfs/src/test/resources/FixedAvroSerializer.config diff --git a/external/storm-hdfs/README.md b/external/storm-hdfs/README.md index bf63ad92664..2fc4c7bda64 100644 --- a/external/storm-hdfs/README.md +++ b/external/storm-hdfs/README.md @@ -307,7 +307,6 @@ The `org.apache.storm.hdfs.bolt.AvroGenericRecordBolt` class allows you to write .withRotationPolicy(rotationPolicy) .withSyncPolicy(syncPolicy); ``` - The setup is very similar to the `SequenceFileBolt` example above. The key difference is that instead of specifying a `SequenceFormat` you must provide a string representation of an Avro schema through the `withSchemaAsString()` method. An `org.apache.avro.Schema` object cannot be directly provided since it does not implement `Serializable`. @@ -315,6 +314,18 @@ An `org.apache.avro.Schema` object cannot be directly provided since it does not The AvroGenericRecordBolt expects to receive tuples containing an Avro GenericRecord that conforms to the provided schema. +To use this bolt you **must** register the appropriate Kryo serializers with your topology configuration. A convenience +method is provided for this: + +`AvroGenericRecordBolt.addAvroKryoSerializations(conf);` + +By default Storm will use the ```GenericAvroSerializer``` to handle serialization. This will work, but there are much +faster options available if you can pre-define the schemas you will be using or utilize an external schema registry. An +implementation using the Confluent Schema Registry is provided, but others can be implemented and provided to Storm. +Please see the javadoc for classes in org.apache.storm.hdfs.avro for information about using the built-in options or +creating your own. + + ## HDFS Bolt support for Trident API storm-hdfs also includes a Trident `state` implementation for writing data to HDFS, with an API that closely mirrors that of the bolts. @@ -548,4 +559,4 @@ under the License. # Committer Sponsors * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org)) - * Bobby Evans ([bobby@apache.org](mailto:bobby@apache.org)) \ No newline at end of file + * Bobby Evans ([bobby@apache.org](mailto:bobby@apache.org)) diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index da4148afdf9..5e8cce4fb40 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -35,6 +35,13 @@ + + + confluent + http://packages.confluent.io/maven + + + org.apache.storm @@ -74,6 +81,10 @@ org.slf4j slf4j-log4j12 + + com.fasterxml.jackson.core + jackson-core + @@ -183,6 +194,17 @@ 4.11 test + + io.confluent + kafka-avro-serializer + 1.0 + + + org.slf4j + slf4j-log4j12 + + + diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AbstractAvroSerializer.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AbstractAvroSerializer.java new file mode 100644 index 00000000000..ddf015d2e24 --- /dev/null +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AbstractAvroSerializer.java @@ -0,0 +1,80 @@ +/** + * 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 org.apache.storm.hdfs.avro; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericContainer; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.EncoderFactory; + +import java.io.IOException; + +//Generously adapted from: +//https://github.com/kijiproject/kiji-express/blob/master/kiji-express/src/main/scala/org/kiji/express/flow/framework/serialization/AvroSerializer.scala +//Which has as an ASL2.0 license + +/** + * This abstract class can be extended to implement concrete classes capable of (de)serializing generic avro objects + * across a Topology. The methods in the AvroSchemaRegistry interface specify how schemas can be mapped to unique + * identifiers and vice versa. Implementations based on pre-defining schemas or utilizing an external schema registry + * are provided. + */ +public abstract class AbstractAvroSerializer extends Serializer implements AvroSchemaRegistry { + + @Override + public void write(Kryo kryo, Output output, GenericContainer record) { + + String fingerPrint = this.getFingerprint(record.getSchema()); + output.writeString(fingerPrint); + GenericDatumWriter writer = new GenericDatumWriter<>(record.getSchema()); + + BinaryEncoder encoder = EncoderFactory + .get() + .directBinaryEncoder(output, null); + try { + writer.write(record, encoder); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public GenericContainer read(Kryo kryo, Input input, Class aClass) { + Schema theSchema = this.getSchema(input.readString()); + GenericDatumReader reader = new GenericDatumReader<>(theSchema); + Decoder decoder = DecoderFactory + .get() + .directBinaryDecoder(input, null); + + GenericContainer foo; + try { + foo = reader.read(null, decoder); + } catch (IOException e) { + throw new RuntimeException(e); + } + return foo; + } +} diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AvroSchemaRegistry.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AvroSchemaRegistry.java new file mode 100644 index 00000000000..0d1dc8bb799 --- /dev/null +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AvroSchemaRegistry.java @@ -0,0 +1,28 @@ +/** + * 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 org.apache.storm.hdfs.avro; + +import org.apache.avro.Schema; + +import java.io.Serializable; + +public interface AvroSchemaRegistry extends Serializable { + String getFingerprint(Schema schema); + + Schema getSchema(String fingerPrint); +} diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AvroUtils.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AvroUtils.java new file mode 100644 index 00000000000..5549291c0b0 --- /dev/null +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AvroUtils.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 org.apache.storm.hdfs.avro; + +import org.apache.avro.generic.GenericData; +import org.apache.storm.Config; + +public class AvroUtils { + /** + * A helper method to extract avro serialization configurations from the topology configuration and register + * specific kryo serializers as necessary. A default serializer will be provided if none is specified in the + * configuration. "avro.serializer" should specify the complete class name of the serializer, e.g. + * "org.apache.stgorm.hdfs.avro.GenericAvroSerializer" + * + * @param conf The topology configuration + * @throws ClassNotFoundException If the specified serializer cannot be located. + */ + public static void addAvroKryoSerializations(Config conf) throws ClassNotFoundException { + final Class serializerClass; + if (conf.containsKey("avro.serializer")) { + serializerClass = Class.forName((String)conf.get("avro.serializer")); + } + else { + serializerClass = GenericAvroSerializer.class; + } + conf.registerSerialization(GenericData.Record.class, serializerClass); + conf.setSkipMissingKryoRegistrations(false); + } +} diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java new file mode 100644 index 00000000000..2008a3e8729 --- /dev/null +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.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 org.apache.storm.hdfs.avro; + +import com.esotericsoftware.kryo.Kryo; +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.apache.avro.Schema; + +import java.io.IOException; +import java.util.Map; + +/** + * This class provides a mechanism to utilize the Confluent Schema Registry (https://github.com/confluentinc/schema-registry) + * for Storm to (de)serialize Avro generic records across a topology. It assumes the schema registry is up and running + * completely independent of Storm. + */ +public class ConfluentAvroSerializer extends AbstractAvroSerializer { + + private SchemaRegistryClient theClient; + final private String url; + + /** + * A constructor for use by test cases ONLY, thus the default scope. + * @param url The complete URL reference of a confluent schema registry, e.g. "http://HOST:PORT" + */ + ConfluentAvroSerializer(String url) { + this.url = url; + this.theClient = new CachedSchemaRegistryClient(this.url, 10000); + } + + /** + * A constructor with a signature that Storm can locate and use with kryo registration. + * See Storm's SerializationFactory class for details + * + * @param k Unused but needs to be present for Serialization Factory to find this constructor + * @param stormConf The global storm configuration. Must define "avro.schemaregistry.confluent" to locate the + * confluent schema registry. Should in the form of "http://HOST:PORT" + */ + public ConfluentAvroSerializer(Kryo k, Map stormConf) { + url = (String) stormConf.get("avro.schemaregistry.confluent"); + this.theClient = new CachedSchemaRegistryClient(this.url, 10000); + } + + @Override + public String getFingerprint(Schema schema) { + final String subject = schema.getName(); + final int guid; + try { + guid = theClient.register(subject, schema); + } catch (IOException | RestClientException e) { + throw new RuntimeException(e); + } + return Integer.toString(guid); + } + + @Override + public Schema getSchema(String fingerPrint) { + final Schema theSchema; + try { + theSchema = theClient.getByID(Integer.parseInt(fingerPrint)); + } catch (IOException | RestClientException e) { + throw new RuntimeException(e); + } + return theSchema; + } +} diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/FixedAvroSerializer.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/FixedAvroSerializer.java new file mode 100644 index 00000000000..4dd5fdcfce9 --- /dev/null +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/FixedAvroSerializer.java @@ -0,0 +1,67 @@ +/** + * 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 org.apache.storm.hdfs.avro; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaNormalization; +import org.apache.commons.codec.binary.Base64; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.security.NoSuchAlgorithmException; +import java.util.HashMap; +import java.util.Map; + +/** + * A class to help (de)serialize a pre-defined set of Avro schemas. Schemas should be listed, one per line, in a file + * called "FixedAvroSerializer.config", which must be part of the Storm topology jar file. Any schemas intended to be + * used with this class **MUST** be defined in that file. + */ +public class FixedAvroSerializer extends AbstractAvroSerializer { + + private final static String FP_ALGO = "CRC-64-AVRO"; + final Map fingerprint2schemaMap = new HashMap<>(); + final Map schema2fingerprintMap = new HashMap<>(); + + public FixedAvroSerializer() throws IOException, NoSuchAlgorithmException { + InputStream in = this.getClass().getClassLoader().getResourceAsStream("FixedAvroSerializer.config"); + BufferedReader reader = new BufferedReader(new InputStreamReader(in)); + + String line; + while((line = reader.readLine()) != null) { + Schema schema = new Schema.Parser().parse(line); + byte [] fp = SchemaNormalization.parsingFingerprint(FP_ALGO, schema); + String fingerPrint = new String(Base64.decodeBase64(fp)); + + fingerprint2schemaMap.put(fingerPrint, schema); + schema2fingerprintMap.put(schema, fingerPrint); + } + } + + @Override + public String getFingerprint(Schema schema) { + return schema2fingerprintMap.get(schema); + } + + @Override + public Schema getSchema(String fingerPrint) { + return fingerprint2schemaMap.get(fingerPrint); + } +} diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/GenericAvroSerializer.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/GenericAvroSerializer.java new file mode 100644 index 00000000000..ecf8c491761 --- /dev/null +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/GenericAvroSerializer.java @@ -0,0 +1,36 @@ +/** + * 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 org.apache.storm.hdfs.avro; + +import org.apache.avro.Schema; + +/** + * A default implementation of the AvroSerializer that will just pass literal schemas back and forth. This should + * only be used if no other serializer will fit a use case. + */ +public class GenericAvroSerializer extends AbstractAvroSerializer { + @Override + public String getFingerprint(Schema schema) { + return schema.toString(); + } + + @Override + public Schema getSchema(String fingerPrint) { + return new Schema.Parser().parse(fingerPrint); + } +} \ No newline at end of file diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java index c817c988969..1fd2e2ff543 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java @@ -17,8 +17,6 @@ */ package org.apache.storm.hdfs.bolt; -import org.apache.storm.Config; -import org.apache.storm.Constants; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Tuple; @@ -41,8 +39,6 @@ import java.io.IOException; import java.net.URI; import java.util.EnumSet; -import java.util.List; -import java.util.LinkedList; import java.util.Map; public class AvroGenericRecordBolt extends AbstractHdfsBolt{ diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/avro/TestFixedAvroSerializer.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/avro/TestFixedAvroSerializer.java new file mode 100644 index 00000000000..a584f911284 --- /dev/null +++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/avro/TestFixedAvroSerializer.java @@ -0,0 +1,76 @@ +/** + * 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 org.apache.storm.hdfs.avro; + +import org.apache.avro.Schema; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class TestFixedAvroSerializer { + //These should match FixedAvroSerializer.config in the test resources + private static final String schemaString1 = "{\"type\":\"record\"," + + "\"name\":\"stormtest1\"," + + "\"fields\":[{\"name\":\"foo1\",\"type\":\"string\"}," + + "{ \"name\":\"int1\", \"type\":\"int\" }]}"; + private static final String schemaString2 = "{\"type\":\"record\"," + + "\"name\":\"stormtest2\"," + + "\"fields\":[{\"name\":\"foobar1\",\"type\":\"string\"}," + + "{ \"name\":\"intint1\", \"type\":\"int\" }]}"; + private static final Schema schema1; + private static final Schema schema2; + + final AvroSchemaRegistry reg; + + static { + + Schema.Parser parser = new Schema.Parser(); + schema1 = parser.parse(schemaString1); + + parser = new Schema.Parser(); + schema2 = parser.parse(schemaString2); + } + + public TestFixedAvroSerializer() throws Exception{ + reg = new FixedAvroSerializer(); + } + + @Test + public void testSchemas() { + testTheSchema(schema1); + testTheSchema(schema2); + } + + @Test public void testDifferentFPs() { + String fp1 = reg.getFingerprint(schema1); + String fp2 = reg.getFingerprint(schema2); + + Assert.assertNotEquals(fp1, fp2); + } + + private void testTheSchema(Schema schema) { + String fp1 = reg.getFingerprint(schema); + Schema found = reg.getSchema(fp1); + String fp2 = reg.getFingerprint(found); + + Assert.assertEquals(found, schema); + Assert.assertEquals(fp1, fp2); + } +} diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/avro/TestGenericAvroSerializer.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/avro/TestGenericAvroSerializer.java new file mode 100644 index 00000000000..ddfdcf5c33d --- /dev/null +++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/avro/TestGenericAvroSerializer.java @@ -0,0 +1,68 @@ +/** + * 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 org.apache.storm.hdfs.avro; + +import org.apache.avro.Schema; +import org.junit.Assert; +import org.junit.Test; + +public class TestGenericAvroSerializer { + private static final String schemaString1 = "{\"type\":\"record\"," + + "\"name\":\"stormtest1\"," + + "\"fields\":[{\"name\":\"foo1\",\"type\":\"string\"}," + + "{ \"name\":\"int1\", \"type\":\"int\" }]}"; + private static final String schemaString2 = "{\"type\":\"record\"," + + "\"name\":\"stormtest2\"," + + "\"fields\":[{\"name\":\"foobar1\",\"type\":\"string\"}," + + "{ \"name\":\"intint1\", \"type\":\"int\" }]}"; + private static final Schema schema1; + private static final Schema schema2; + + AvroSchemaRegistry reg = new GenericAvroSerializer(); + + static { + + Schema.Parser parser = new Schema.Parser(); + schema1 = parser.parse(schemaString1); + + parser = new Schema.Parser(); + schema2 = parser.parse(schemaString2); + } + + @Test + public void testSchemas() { + testTheSchema(schema1); + testTheSchema(schema2); + } + + @Test public void testDifferentFPs() { + String fp1 = reg.getFingerprint(schema1); + String fp2 = reg.getFingerprint(schema2); + + Assert.assertNotEquals(fp1, fp2); + } + + private void testTheSchema(Schema schema) { + String fp1 = reg.getFingerprint(schema); + Schema found = reg.getSchema(fp1); + String fp2 = reg.getFingerprint(found); + + Assert.assertEquals(found, schema); + Assert.assertEquals(fp1, fp2); + } +} diff --git a/external/storm-hdfs/src/test/resources/FixedAvroSerializer.config b/external/storm-hdfs/src/test/resources/FixedAvroSerializer.config new file mode 100644 index 00000000000..971d4115d5d --- /dev/null +++ b/external/storm-hdfs/src/test/resources/FixedAvroSerializer.config @@ -0,0 +1,2 @@ +{"type":"record", "name":"stormtest1", "fields":[{"name":"foo1","type":"string"}, {"name":"int1", "type":"int" }]} +{"type":"record", "name":"stormtest2", "fields":[{"name":"foobar1","type":"string"}, {"name":"intint1", "type":"int" }]} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 691a8cb6589..025fb683d33 100644 --- a/pom.xml +++ b/pom.xml @@ -357,6 +357,9 @@ **/src/codegen/config.fmpp **/src/codegen/data/Parser.tdd + + + **/src/test/resources/FixedAvroSerializer.config From d9c04093a9a6eb114751640c8e2473b7f6479b0b Mon Sep 17 00:00:00 2001 From: Aaron Dossett Date: Wed, 3 Feb 2016 07:54:57 -0600 Subject: [PATCH 025/387] Added STORM-1514 to ChangeLog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 990aab2a49c..4fdf3539218 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1518: Backport of STORM-1504 * STORM-1510: Fix broken nimbus log link * STORM-1503: Worker should not crash on failure to send heartbeats to Pacemaker/ZK * STORM-1176: Checkpoint window evaluated/expired state From f18dead5c9ec807f57da1498383a5f30fe2906fa Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 3 Feb 2016 16:55:58 -0500 Subject: [PATCH 026/387] add STORM-1505 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 990aab2a49c..7e75f5b65f8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1505: Add map, flatMap and filter functions in trident stream * STORM-1510: Fix broken nimbus log link * STORM-1503: Worker should not crash on failure to send heartbeats to Pacemaker/ZK * STORM-1176: Checkpoint window evaluated/expired state From 0f5abf43dfd02eb878b5ec09b098c224ba8c5bc0 Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal Date: Tue, 19 Jan 2016 21:43:22 +0530 Subject: [PATCH 027/387] STORM-1455: Do not reset the emittedOffset for offsetOutOfRangeExceptions --- .../org/apache/storm/kafka/KafkaUtils.java | 44 +++++++++++-------- .../apache/storm/kafka/PartitionManager.java | 42 ++++++++++++------ 2 files changed, 54 insertions(+), 32 deletions(-) diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java index 8cd0fd0e7c8..a2be825a326 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaUtils.java @@ -17,24 +17,15 @@ */ package org.apache.storm.kafka; -import org.apache.storm.metric.api.IMetric; -import org.apache.storm.utils.Utils; import com.google.common.base.Preconditions; -import kafka.api.FetchRequest; -import kafka.api.FetchRequestBuilder; -import kafka.api.PartitionOffsetRequestInfo; -import kafka.common.TopicAndPartition; -import kafka.javaapi.FetchResponse; -import kafka.javaapi.OffsetRequest; -import kafka.javaapi.consumer.SimpleConsumer; -import kafka.javaapi.message.ByteBufferMessageSet; -import kafka.message.Message; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + import org.apache.storm.kafka.trident.GlobalPartitionInformation; import org.apache.storm.kafka.trident.IBrokerReader; import org.apache.storm.kafka.trident.StaticBrokerReader; import org.apache.storm.kafka.trident.ZkBrokerReader; +import org.apache.storm.metric.api.IMetric; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.net.ConnectException; @@ -43,6 +34,16 @@ import java.nio.channels.UnresolvedAddressException; import java.util.*; +import kafka.api.FetchRequest; +import kafka.api.FetchRequestBuilder; +import kafka.api.PartitionOffsetRequestInfo; +import kafka.common.TopicAndPartition; +import kafka.javaapi.FetchResponse; +import kafka.javaapi.OffsetRequest; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; +import kafka.message.Message; + public class KafkaUtils { @@ -80,7 +81,7 @@ public static long getOffset(SimpleConsumer consumer, String topic, int partitio } public static class KafkaOffsetMetric implements IMetric { - Map _partitionToOffset = new HashMap(); + Map _partitionToOffset = new HashMap(); Set _partitions; DynamicPartitionConnections _connections; @@ -88,8 +89,8 @@ public KafkaOffsetMetric(DynamicPartitionConnections connections) { _connections = connections; } - public void setLatestEmittedOffset(Partition partition, long offset) { - _partitionToOffset.put(partition, offset); + public void setOffsetData(Partition partition, PartitionManager.OffsetData offsetData) { + _partitionToOffset.put(partition, offsetData); } private class TopicMetrics { @@ -97,6 +98,7 @@ private class TopicMetrics { long totalEarliestTimeOffset = 0; long totalLatestTimeOffset = 0; long totalLatestEmittedOffset = 0; + long totalLatestCompletedOffset = 0; } @Override @@ -105,7 +107,7 @@ public Object getValueAndReset() { HashMap ret = new HashMap(); if (_partitions != null && _partitions.size() == _partitionToOffset.size()) { Map topicMetricsMap = new TreeMap(); - for (Map.Entry e : _partitionToOffset.entrySet()) { + for (Map.Entry e : _partitionToOffset.entrySet()) { Partition partition = e.getKey(); SimpleConsumer consumer = _connections.getConnection(partition); if (consumer == null) { @@ -118,8 +120,9 @@ public Object getValueAndReset() { LOG.warn("No data found in Kafka Partition " + partition.getId()); return null; } - long latestEmittedOffset = e.getValue(); - long spoutLag = latestTimeOffset - latestEmittedOffset; + long latestEmittedOffset = e.getValue().latestEmittedOffset; + long latestCompletedOffset = e.getValue().latestCompletedOffset; + long spoutLag = latestTimeOffset - latestCompletedOffset; String topic = partition.topic; String metricPath = partition.getId(); //Handle the case where Partition Path Id does not contain topic name Partition.getId() == "partition_" + partition @@ -130,6 +133,7 @@ public Object getValueAndReset() { ret.put(metricPath + "/" + "earliestTimeOffset", earliestTimeOffset); ret.put(metricPath + "/" + "latestTimeOffset", latestTimeOffset); ret.put(metricPath + "/" + "latestEmittedOffset", latestEmittedOffset); + ret.put(metricPath + "/" + "latestCompletedOffset", latestCompletedOffset); if (!topicMetricsMap.containsKey(partition.topic)) { topicMetricsMap.put(partition.topic,new TopicMetrics()); @@ -140,6 +144,7 @@ public Object getValueAndReset() { topicMetrics.totalEarliestTimeOffset += earliestTimeOffset; topicMetrics.totalLatestTimeOffset += latestTimeOffset; topicMetrics.totalLatestEmittedOffset += latestEmittedOffset; + topicMetrics.totalLatestCompletedOffset += latestCompletedOffset; } for(Map.Entry e : topicMetricsMap.entrySet()) { @@ -149,6 +154,7 @@ public Object getValueAndReset() { ret.put(topic + "/" + "totalEarliestTimeOffset", topicMetrics.totalEarliestTimeOffset); ret.put(topic + "/" + "totalLatestTimeOffset", topicMetrics.totalLatestTimeOffset); ret.put(topic + "/" + "totalLatestEmittedOffset", topicMetrics.totalLatestEmittedOffset); + ret.put(topic + "/" + "totalLatestCompletedOffset", topicMetrics.totalLatestCompletedOffset); } return ret; diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java index 0f5aaa0425f..bcc40014997 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java @@ -17,28 +17,26 @@ */ package org.apache.storm.kafka; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; + import org.apache.storm.Config; +import org.apache.storm.kafka.KafkaSpout.EmitState; +import org.apache.storm.kafka.trident.MaxMetric; import org.apache.storm.metric.api.CombinedMetric; import org.apache.storm.metric.api.CountMetric; import org.apache.storm.metric.api.MeanReducer; import org.apache.storm.metric.api.ReducedMetric; import org.apache.storm.spout.SpoutOutputCollector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableMap; +import java.util.*; import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.message.ByteBufferMessageSet; import kafka.message.MessageAndOffset; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.storm.kafka.KafkaSpout.EmitState; -import org.apache.storm.kafka.trident.MaxMetric; - -import java.util.*; - public class PartitionManager { public static final Logger LOG = LoggerFactory.getLogger(PartitionManager.class); @@ -186,8 +184,7 @@ private void fill() { try { msgs = KafkaUtils.fetchMessages(_spoutConfig, _consumer, _partition, offset); } catch (TopicOffsetOutOfRangeException e) { - _emittedToOffset = KafkaUtils.getOffset(_consumer, _partition.topic, _partition.partition, kafka.api.OffsetRequest.EarliestTime()); - LOG.warn("{} Using new offset: {}", _partition.partition, _emittedToOffset); + offset = KafkaUtils.getOffset(_consumer, _partition.topic, _partition.partition, kafka.api.OffsetRequest.EarliestTime()); // fetch failed, so don't update the metrics //fix bug [STORM-643] : remove outdated failed offsets @@ -196,10 +193,15 @@ private void fill() { // all the failed offsets, that are earlier than actual EarliestTime // offset, since they are anyway not there. // These calls to broker API will be then saved. - Set omitted = this._failedMsgRetryManager.clearInvalidMessages(_emittedToOffset); + Set omitted = this._failedMsgRetryManager.clearInvalidMessages(offset); LOG.warn("Removing the failed offsets that are out of range: {}", omitted); } + + if (offset > _emittedToOffset) { + _emittedToOffset = offset; + LOG.warn("{} Using new offset: {}", _partition.partition, _emittedToOffset); + } return; } @@ -294,6 +296,10 @@ public long lastCompletedOffset() { } } + public OffsetData getOffsetData() { + return new OffsetData(_emittedToOffset, lastCompletedOffset()); + } + public Partition getPartition() { return _partition; } @@ -313,4 +319,14 @@ public KafkaMessageId(Partition partition, long offset) { this.offset = offset; } } + + static class OffsetData { + public long latestEmittedOffset; + public long latestCompletedOffset; + + public OffsetData(long latestEmittedOffset, long latestCompletedOffset) { + this.latestEmittedOffset = latestEmittedOffset; + this.latestCompletedOffset = latestCompletedOffset; + } + } } From 5eac6855642df52823af9ba13ad12ab5505f9753 Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal Date: Tue, 26 Jan 2016 23:29:50 +0530 Subject: [PATCH 028/387] STORM-1455: Fix build failure --- .../org/apache/storm/kafka/KafkaSpout.java | 8 +++---- .../apache/storm/kafka/PartitionManager.java | 2 +- .../kafka/trident/TridentKafkaEmitter.java | 23 +++++++++++-------- 3 files changed, 18 insertions(+), 15 deletions(-) diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java index 7a83ae0d0f4..d1da446347b 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/KafkaSpout.java @@ -17,17 +17,17 @@ */ package org.apache.storm.kafka; +import com.google.common.base.Strings; + import org.apache.storm.Config; +import org.apache.storm.kafka.PartitionManager.KafkaMessageId; import org.apache.storm.metric.api.IMetric; import org.apache.storm.spout.SpoutOutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.topology.base.BaseRichSpout; -import com.google.common.base.Strings; -import kafka.message.Message; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.storm.kafka.PartitionManager.KafkaMessageId; import java.util.*; @@ -100,7 +100,7 @@ public Object getValueAndReset() { } _kafkaOffsetMetric.refreshPartitions(latestPartitions); for (PartitionManager pm : pms) { - _kafkaOffsetMetric.setLatestEmittedOffset(pm.getPartition(), pm.lastCompletedOffset()); + _kafkaOffsetMetric.setOffsetData(pm.getPartition(), pm.getOffsetData()); } return _kafkaOffsetMetric.getValueAndReset(); } diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java index bcc40014997..dbf70a0a9a2 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java @@ -320,7 +320,7 @@ public KafkaMessageId(Partition partition, long offset) { } } - static class OffsetData { + public static class OffsetData { public long latestEmittedOffset; public long latestCompletedOffset; diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaEmitter.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaEmitter.java index 6eddaf5de66..9732c8c57b4 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaEmitter.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaEmitter.java @@ -17,27 +17,28 @@ */ package org.apache.storm.kafka.trident; +import com.google.common.collect.ImmutableMap; + import org.apache.storm.Config; +import org.apache.storm.kafka.*; import org.apache.storm.metric.api.CombinedMetric; import org.apache.storm.metric.api.MeanReducer; import org.apache.storm.metric.api.ReducedMetric; import org.apache.storm.task.TopologyContext; -import com.google.common.collect.ImmutableMap; -import kafka.javaapi.consumer.SimpleConsumer; -import kafka.javaapi.message.ByteBufferMessageSet; -import kafka.message.Message; -import kafka.message.MessageAndOffset; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.storm.kafka.*; -import org.apache.storm.kafka.TopicOffsetOutOfRangeException; import org.apache.storm.trident.operation.TridentCollector; import org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout; import org.apache.storm.trident.spout.IPartitionedTridentSpout; import org.apache.storm.trident.topology.TransactionAttempt; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.*; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; +import kafka.message.Message; +import kafka.message.MessageAndOffset; + public class TridentKafkaEmitter { public static final Logger LOG = LoggerFactory.getLogger(TridentKafkaEmitter.class); @@ -65,7 +66,9 @@ public TridentKafkaEmitter(Map conf, TopologyContext context, TridentKafkaConfig private Map failFastEmitNewPartitionBatch(TransactionAttempt attempt, TridentCollector collector, Partition partition, Map lastMeta) { SimpleConsumer consumer = _connections.register(partition); Map ret = doEmitNewPartitionBatch(consumer, partition, collector, lastMeta); - _kafkaOffsetMetric.setLatestEmittedOffset(partition, (Long) ret.get("offset")); + Long offset = (Long) ret.get("offset"); + Long endOffset = (Long) ret.get("nextOffset"); + _kafkaOffsetMetric.setOffsetData(partition, new PartitionManager.OffsetData(endOffset, offset)); return ret; } From 7090be454dcbe24fba819a92b58af8b468bbe728 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 5 Feb 2016 14:48:16 +0900 Subject: [PATCH 029/387] add STORM-1455 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 57458596d70..df1ddb60d23 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1455: kafka spout should not reset to the beginning of partition when offsetoutofrange exception occurs * STORM-1518: Backport of STORM-1504 * STORM-1505: Add map, flatMap and filter functions in trident stream * STORM-1510: Fix broken nimbus log link From 96bde69a0e0790546427fd1c4a29ae5e4e4af7ae Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Tue, 2 Feb 2016 16:42:11 +0530 Subject: [PATCH 030/387] [STORM-1517] add peek api in trident stream Similar to the Java 8 peek, the peek api can be used to examine trident tuples at some point in the stream pipeline or execute some custom actions. --- .../starter/trident/TridentMapExample.java | 7 ++++ .../jvm/org/apache/storm/trident/Stream.java | 21 ++++++++++ .../storm/trident/operation/Consumer.java | 35 +++++++++++++++++ .../operation/impl/ConsumerExecutor.java | 38 +++++++++++++++++++ 4 files changed, 101 insertions(+) create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/Consumer.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/impl/ConsumerExecutor.java diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java index 95b52ccc97d..fbb91277e8a 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java @@ -25,6 +25,7 @@ import org.apache.storm.trident.TridentState; import org.apache.storm.trident.TridentTopology; import org.apache.storm.trident.operation.BaseFilter; +import org.apache.storm.trident.operation.Consumer; import org.apache.storm.trident.operation.Filter; import org.apache.storm.trident.operation.FlatMapFunction; import org.apache.storm.trident.operation.MapFunction; @@ -84,6 +85,12 @@ public static StormTopology buildTopology(LocalDRPC drpc) { .flatMap(split) .map(toUpper) .filter(theFilter) + .peek(new Consumer() { + @Override + public void accept(TridentTuple input) { + System.out.println(input.getString(0)); + } + }) .groupBy(new Fields("word")) .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count")) .parallelismHint(16); diff --git a/storm-core/src/jvm/org/apache/storm/trident/Stream.java b/storm-core/src/jvm/org/apache/storm/trident/Stream.java index dffc984a95b..7c6d93f7669 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/Stream.java +++ b/storm-core/src/jvm/org/apache/storm/trident/Stream.java @@ -21,8 +21,10 @@ import org.apache.storm.generated.NullStruct; import org.apache.storm.trident.fluent.ChainedAggregatorDeclarer; import org.apache.storm.grouping.CustomStreamGrouping; +import org.apache.storm.trident.operation.Consumer; import org.apache.storm.trident.operation.FlatMapFunction; import org.apache.storm.trident.operation.MapFunction; +import org.apache.storm.trident.operation.impl.ConsumerExecutor; import org.apache.storm.trident.operation.impl.FlatMapFunctionExecutor; import org.apache.storm.trident.operation.impl.MapFunctionExecutor; import org.apache.storm.trident.planner.processor.MapProcessor; @@ -387,6 +389,25 @@ public Stream flatMap(FlatMapFunction function) { new MapProcessor(getOutputFields(), new FlatMapFunctionExecutor(function)))); } + /** + * Returns a stream consisting of the trident tuples of this stream, additionally performing the provided action on + * each trident tuple as they are consumed from the resulting stream. This is mostly useful for debugging + * to see the tuples as they flow past a certain point in a pipeline. + * + * @param action the action to perform on the trident tuple as they are consumed from the stream + * @return the new stream + */ + public Stream peek(Consumer action) { + projectionValidation(getOutputFields()); + return _topology.addSourcedNode(this, + new ProcessorNode( + _topology.getUniqueStreamId(), + _name, + getOutputFields(), + getOutputFields(), + new MapProcessor(getOutputFields(), new ConsumerExecutor(action)))); + } + public ChainedAggregatorDeclarer chainedAgg() { return new ChainedAggregatorDeclarer(this, new BatchGlobalAggScheme()); } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/Consumer.java b/storm-core/src/jvm/org/apache/storm/trident/operation/Consumer.java new file mode 100644 index 00000000000..dd13b48a6b4 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/Consumer.java @@ -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. + */ +package org.apache.storm.trident.operation; + +import org.apache.storm.trident.tuple.TridentTuple; + +import java.io.Serializable; + +/** + * Represents an operation that accepts a single input argument and returns no result. + * This is similar to the Consumer interface in Java 8. + */ +public interface Consumer extends Serializable { + /** + * Performs the operation on the input trident tuple. + * + * @param input the input trident tuple + */ + void accept(TridentTuple input); +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/impl/ConsumerExecutor.java b/storm-core/src/jvm/org/apache/storm/trident/operation/impl/ConsumerExecutor.java new file mode 100644 index 00000000000..c08a8f7bcda --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/impl/ConsumerExecutor.java @@ -0,0 +1,38 @@ +/** + * 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 org.apache.storm.trident.operation.impl; + +import org.apache.storm.trident.operation.BaseOperation; +import org.apache.storm.trident.operation.Consumer; +import org.apache.storm.trident.operation.Function; +import org.apache.storm.trident.operation.TridentCollector; +import org.apache.storm.trident.tuple.TridentTuple; + +public class ConsumerExecutor extends BaseOperation implements Function { + private final Consumer consumer; + + public ConsumerExecutor(Consumer consumer) { + this.consumer = consumer; + } + + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + consumer.accept(tuple); + collector.emit(tuple); + } +} From ef7581b19ff7fd94f8183d65c94830d49281d6fa Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 5 Feb 2016 21:55:02 +0900 Subject: [PATCH 031/387] add STORM-1517 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index df1ddb60d23..56529ce57cc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1517: Add peek api in trident stream * STORM-1455: kafka spout should not reset to the beginning of partition when offsetoutofrange exception occurs * STORM-1518: Backport of STORM-1504 * STORM-1505: Add map, flatMap and filter functions in trident stream From 68682331a219c6d2ad706729389d6eba2585c32c Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 3 Feb 2016 10:49:56 -0600 Subject: [PATCH 032/387] Create stats plugin for JMX Add config for stats reporter plugin and use it Use Regular Map instead of Config in interface Adding log entries for statiscs plugin actions. Conflicts: storm-core/src/clj/org/apache/storm/daemon/common.clj --- .../clj/org/apache/storm/daemon/common.clj | 12 +++-- .../src/clj/org/apache/storm/daemon/drpc.clj | 2 +- .../clj/org/apache/storm/daemon/logviewer.clj | 2 +- .../clj/org/apache/storm/daemon/nimbus.clj | 2 +- .../org/apache/storm/daemon/supervisor.clj | 2 +- .../src/clj/org/apache/storm/ui/core.clj | 2 +- .../src/jvm/org/apache/storm/Config.java | 7 +++ .../storm/statistics/StatisticsUtils.java | 26 ++++++++++ .../reporters/JMXPreparableReporter.java | 49 +++++++++++++++++++ .../reporters/PreparableReporter.java | 15 ++++++ 10 files changed, 111 insertions(+), 8 deletions(-) create mode 100644 storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java create mode 100644 storm-core/src/jvm/org/apache/storm/statistics/reporters/JMXPreparableReporter.java create mode 100644 storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj index dd761a56212..44a1d43cb85 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/common.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj @@ -19,6 +19,9 @@ InvalidTopologyException GlobalStreamId] [org.apache.storm.utils ThriftTopologyUtils]) (:import [org.apache.storm.utils Utils]) + (:import [org.apache.storm.statistics.reporters PreparableReporter] + [com.codahale.metrics MetricRegistry] + [org.apache.storm.statistics StatisticsUtils]) (:import [org.apache.storm.task WorkerTopologyContext]) (:import [org.apache.storm Constants]) (:import [org.apache.storm.metric SystemBolt]) @@ -28,10 +31,13 @@ (:require [clojure.set :as set]) (:require [org.apache.storm.daemon.acker :as acker]) (:require [org.apache.storm.thrift :as thrift]) - (:require [metrics.reporters.jmx :as jmx])) + (:require [metrics.core :refer [default-registry]])) -(defn start-metrics-reporters [] - (jmx/start (jmx/reporter {}))) +(defn start-metrics-reporters [conf] + (doto (StatisticsUtils/getPreparableReporter conf) + (.prepare default-registry conf) + (.start)) + (log-message "Started statistics report plugin...")) (def ACKER-COMPONENT-ID acker/ACKER-COMPONENT-ID) (def ACKER-INIT-STREAM-ID acker/ACKER-INIT-STREAM-ID) diff --git a/storm-core/src/clj/org/apache/storm/daemon/drpc.clj b/storm-core/src/clj/org/apache/storm/daemon/drpc.clj index d6f77c3f693..c2fadc665f6 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/drpc.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/drpc.clj @@ -265,7 +265,7 @@ https-need-client-auth https-want-client-auth) (config-filter server app filters-confs))}))) - (start-metrics-reporters) + (start-metrics-reporters conf) (when handler-server (.serve handler-server))))) diff --git a/storm-core/src/clj/org/apache/storm/daemon/logviewer.clj b/storm-core/src/clj/org/apache/storm/daemon/logviewer.clj index 66a189914df..1fcb5d5b2c2 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/logviewer.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/logviewer.clj @@ -1198,4 +1198,4 @@ STORM-VERSION "'") (start-logviewer! conf log-root daemonlog-root) - (start-metrics-reporters))) + (start-metrics-reporters conf))) diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj index 587da65e1de..9376d6e2bfb 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj @@ -1450,7 +1450,7 @@ (defgauge nimbus:num-supervisors (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil)))) - (start-metrics-reporters) + (start-metrics-reporters conf) (reify Nimbus$Iface (^void submitTopologyWithOpts diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj index 4bca23e1caf..d27a6090c68 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj @@ -1196,7 +1196,7 @@ (let [supervisor (mk-supervisor conf nil supervisor)] (add-shutdown-hook-with-force-kill-in-1-sec #(.shutdown supervisor))) (defgauge supervisor:num-slots-used-gauge #(count (my-worker-ids conf))) - (start-metrics-reporters))) + (start-metrics-reporters conf))) (defn standalone-supervisor [] (let [conf-atom (atom nil) diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj index cbea15a9e12..bd1dba5541a 100644 --- a/storm-core/src/clj/org/apache/storm/ui/core.clj +++ b/storm-core/src/clj/org/apache/storm/ui/core.clj @@ -1260,7 +1260,7 @@ https-ts-type (conf UI-HTTPS-TRUSTSTORE-TYPE) https-want-client-auth (conf UI-HTTPS-WANT-CLIENT-AUTH) https-need-client-auth (conf UI-HTTPS-NEED-CLIENT-AUTH)] - (start-metrics-reporters) + (start-metrics-reporters conf) (storm-run-jetty {:port (conf UI-PORT) :host (conf UI-HOST) :https-port https-port diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index f7f516985c0..bf502232afd 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -139,6 +139,13 @@ public class Config extends HashMap { @isString public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate"; + /** + * A list of statistics preparable reporter class names. + */ + @NotNull + @isImplementationOfClass(implementsClass = org.apache.storm.statistics.reporters.PreparableReporter.class) + public static final String STORM_STATISTICS_PREPARABLE_REPORTER_PLUGIN = "storm.statistics.preparable.reporter.plugin"; + /** * A list of hosts of ZooKeeper servers used to manage the cluster. */ diff --git a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java b/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java new file mode 100644 index 00000000000..19f7690880e --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java @@ -0,0 +1,26 @@ +package org.apache.storm.statistics; + +import org.apache.storm.Config; +import org.apache.storm.statistics.reporters.JMXPreparableReporter; +import org.apache.storm.statistics.reporters.PreparableReporter; +import org.apache.storm.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +public class StatisticsUtils { + private final static Logger LOG = LoggerFactory.getLogger(StatisticsUtils.class); + + public static PreparableReporter getPreparableReporter(Map stormConf) { + PreparableReporter reporter = new JMXPreparableReporter(); + String clazz = (String) stormConf.get(Config.STORM_STATISTICS_PREPARABLE_REPORTER_PLUGIN); + LOG.info("Using statistics reporter plugin:" + clazz); + if(clazz != null) { + reporter = (PreparableReporter) Utils.newInstance(clazz); + } else { + reporter = new JMXPreparableReporter(); + } + return reporter; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/JMXPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/JMXPreparableReporter.java new file mode 100644 index 00000000000..5d94ffcbf23 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/JMXPreparableReporter.java @@ -0,0 +1,49 @@ +package org.apache.storm.statistics.reporters; + +import com.codahale.metrics.JmxReporter; +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; +import org.apache.storm.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class JMXPreparableReporter implements PreparableReporter { + private final static Logger LOG = LoggerFactory.getLogger(JMXPreparableReporter.class); + + JmxReporter reporter = null; + + @Override + public void prepare(MetricRegistry metricsRegistry, Map stormConf) { + LOG.info("Preparing..."); + JmxReporter.Builder builder = JmxReporter.forRegistry(metricsRegistry); + String domain = Utils.getString(stormConf.get(":domain"), null); + if (domain != null) { + builder.inDomain(domain); + } + String rateUnit = Utils.getString(stormConf.get(":rate-unit"), null); + if (rateUnit != null) { + builder.convertRatesTo(TimeUnit.valueOf(rateUnit)); + } + MetricFilter filter = (MetricFilter) stormConf.get(":filter"); + if (filter != null) { + builder.filter(filter); + } + reporter = builder.build(); + + } + + @Override + public void start() { + LOG.info("Starting..."); + reporter.start(); + } + + @Override + public void stop() { + LOG.info("Stopping..."); + reporter.stop(); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java new file mode 100644 index 00000000000..f6e8b2bcd26 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java @@ -0,0 +1,15 @@ +package org.apache.storm.statistics.reporters; + +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.Reporter; + +import java.io.Closeable; +import java.util.Map; + + +public interface PreparableReporter { + public abstract void prepare(MetricRegistry metricsRegistry, Map stormConf); + public abstract void start(); + public abstract void stop(); + +} From ea6cccfcab257fe6854867c2b54029378953080a Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 3 Feb 2016 13:47:09 -0600 Subject: [PATCH 033/387] Adding Cvs and Console statistics reporter plugins Make statistics reporter plugins a list. --- conf/defaults.yaml | 4 + .../clj/org/apache/storm/daemon/common.clj | 9 ++- .../src/jvm/org/apache/storm/Config.java | 2 +- .../storm/statistics/StatisticsUtils.java | 27 +++++-- .../reporters/ConsolePreparableReporter.java | 65 +++++++++++++++ .../reporters/CsvPreparableReporter.java | 80 +++++++++++++++++++ ...porter.java => JmxPreparableReporter.java} | 21 +++-- 7 files changed, 192 insertions(+), 16 deletions(-) create mode 100644 storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java create mode 100644 storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java rename storm-core/src/jvm/org/apache/storm/statistics/reporters/{JMXPreparableReporter.java => JmxPreparableReporter.java} (65%) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 8873d123925..b468290b9f3 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -281,3 +281,7 @@ pacemaker.thread.timeout: 10 pacemaker.childopts: "-Xmx1024m" pacemaker.auth.method: "NONE" pacemaker.kerberos.users: [] + +#default plugin for daemon statistics reporter +storm.statistics.preparable.reporter.plugin: + - "org.apache.storm.statistics.reporters.JmxPreparableReporter" diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj index 44a1d43cb85..6b7d539658c 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/common.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj @@ -33,12 +33,17 @@ (:require [org.apache.storm.thrift :as thrift]) (:require [metrics.core :refer [default-registry]])) -(defn start-metrics-reporters [conf] - (doto (StatisticsUtils/getPreparableReporter conf) +(defn start-metrics-reporter [reporter conf] + (doto reporter (.prepare default-registry conf) (.start)) (log-message "Started statistics report plugin...")) +(defn start-metrics-reporters [conf] + (doseq [reporter (StatisticsUtils/getPreparableReporters conf)] + (start-metrics-reporter reporter conf))) + + (def ACKER-COMPONENT-ID acker/ACKER-COMPONENT-ID) (def ACKER-INIT-STREAM-ID acker/ACKER-INIT-STREAM-ID) (def ACKER-ACK-STREAM-ID acker/ACKER-ACK-STREAM-ID) diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index bf502232afd..9d18667ca08 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -143,7 +143,7 @@ public class Config extends HashMap { * A list of statistics preparable reporter class names. */ @NotNull - @isImplementationOfClass(implementsClass = org.apache.storm.statistics.reporters.PreparableReporter.class) + @isStringList public static final String STORM_STATISTICS_PREPARABLE_REPORTER_PLUGIN = "storm.statistics.preparable.reporter.plugin"; /** diff --git a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java b/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java index 19f7690880e..666e44db5df 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java @@ -1,25 +1,40 @@ package org.apache.storm.statistics; import org.apache.storm.Config; -import org.apache.storm.statistics.reporters.JMXPreparableReporter; +import org.apache.storm.statistics.reporters.JmxPreparableReporter; import org.apache.storm.statistics.reporters.PreparableReporter; import org.apache.storm.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.List; import java.util.Map; public class StatisticsUtils { private final static Logger LOG = LoggerFactory.getLogger(StatisticsUtils.class); - public static PreparableReporter getPreparableReporter(Map stormConf) { - PreparableReporter reporter = new JMXPreparableReporter(); - String clazz = (String) stormConf.get(Config.STORM_STATISTICS_PREPARABLE_REPORTER_PLUGIN); + public static List getPreparableReporters(Map stormConf) { + PreparableReporter reporter = new JmxPreparableReporter(); + List clazzes = (List) stormConf.get(Config.STORM_STATISTICS_PREPARABLE_REPORTER_PLUGIN); + List reporterList = new ArrayList<>(); + + if (clazzes != null) { + for(String clazz: clazzes ) { + reporterList.add(getPreparableReporter(clazz)); + } + } + if(reporterList.isEmpty()) { + reporterList.add(new JmxPreparableReporter()); + } + return reporterList; + } + + private static PreparableReporter getPreparableReporter(String clazz) { + PreparableReporter reporter = null; LOG.info("Using statistics reporter plugin:" + clazz); if(clazz != null) { reporter = (PreparableReporter) Utils.newInstance(clazz); - } else { - reporter = new JMXPreparableReporter(); } return reporter; } diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java new file mode 100644 index 00000000000..f545b5b0126 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java @@ -0,0 +1,65 @@ +package org.apache.storm.statistics.reporters; + +import com.codahale.metrics.ConsoleReporter; +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; +import org.apache.storm.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.PrintStream; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class ConsolePreparableReporter implements PreparableReporter { + private final static Logger LOG = LoggerFactory.getLogger(ConsolePreparableReporter.class); + ConsoleReporter reporter = null; + + @Override + public void prepare(MetricRegistry metricsRegistry, Map stormConf) { + LOG.info("Preparing..."); + ConsoleReporter.Builder builder = ConsoleReporter.forRegistry(metricsRegistry); + PrintStream stream = (PrintStream)stormConf.get(":stream"); + if (stream != null) { + builder.outputTo(stream); + } + Locale locale = (Locale)stormConf.get(":locale"); + if (locale != null) { + builder.formattedFor(locale); + } + String rateUnit = Utils.getString(stormConf.get(":rate-unit"), null); + if (rateUnit != null) { + builder.convertRatesTo(TimeUnit.valueOf(rateUnit)); + } + String durationUnit = Utils.getString(stormConf.get(":duration-unit"), null); + if (durationUnit != null) { + builder.convertDurationsTo(TimeUnit.valueOf(durationUnit)); + } + MetricFilter filter = (MetricFilter) stormConf.get(":filter"); + if (filter != null) { + builder.filter(filter); + } + reporter = builder.build(); + } + + @Override + public void start() { + if (reporter != null ) { + LOG.info("Starting..."); + reporter.start(10, TimeUnit.SECONDS); + } else { + throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName()); + } + } + + @Override + public void stop() { + if (reporter !=null) { + LOG.info("Stopping..."); + reporter.stop(); + } else { + throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName()); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java new file mode 100644 index 00000000000..610df33fc28 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java @@ -0,0 +1,80 @@ +package org.apache.storm.statistics.reporters; + +import com.codahale.metrics.CsvReporter; +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; +import org.apache.storm.Config; +import org.apache.storm.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class CsvPreparableReporter implements PreparableReporter { + private final static Logger LOG = LoggerFactory.getLogger(CsvPreparableReporter.class); + CsvReporter reporter = null; + + @Override + public void prepare(MetricRegistry metricsRegistry, Map stormConf) { + LOG.info("Preparing..."); + CsvReporter.Builder builder = CsvReporter.forRegistry(metricsRegistry); + + Locale locale = (Locale) stormConf.get(":locale"); + if (locale != null) { + builder.formatFor(locale); + } + String rateUnit = Utils.getString(stormConf.get(":rate-unit"), null); + if (rateUnit != null) { + builder.convertRatesTo(TimeUnit.valueOf(rateUnit)); + } + String durationUnit = Utils.getString(stormConf.get(":duration-unit"), null); + if (durationUnit != null) { + builder.convertDurationsTo(TimeUnit.valueOf(durationUnit)); + } + MetricFilter filter = (MetricFilter) stormConf.get(":filter"); + if (filter != null) { + builder.filter(filter); + } + String localStormDirLocation = Utils.getString(stormConf.get(Config.STORM_LOCAL_DIR), "."); + File logDir = new File(localStormDirLocation + "csvmetrics" ); + validateCreateOutputDir(logDir); + reporter = builder.build(logDir); + } + + @Override + public void start() { + if (reporter != null) { + LOG.info("Starting..."); + reporter.start(10, TimeUnit.SECONDS); + } else { + throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName()); + } + } + + @Override + public void stop() { + if (reporter != null) { + LOG.info("Stopping..."); + reporter.stop(); + } else { + throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName()); + } + } + + + private void validateCreateOutputDir(File dir) { + if (!dir.exists()) { + dir.mkdirs(); + } + if (!dir.canWrite()) { + throw new IllegalStateException(dir.getName() + " does not have write permissions."); + } + if (!dir.isDirectory()) { + throw new IllegalStateException(dir.getName() + " is not a directory."); + } + } +} + diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/JMXPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java similarity index 65% rename from storm-core/src/jvm/org/apache/storm/statistics/reporters/JMXPreparableReporter.java rename to storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java index 5d94ffcbf23..ba596114128 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/JMXPreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java @@ -10,9 +10,8 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -public class JMXPreparableReporter implements PreparableReporter { - private final static Logger LOG = LoggerFactory.getLogger(JMXPreparableReporter.class); - +public class JmxPreparableReporter implements PreparableReporter { + private final static Logger LOG = LoggerFactory.getLogger(JmxPreparableReporter.class); JmxReporter reporter = null; @Override @@ -37,13 +36,21 @@ public void prepare(MetricRegistry metricsRegistry, Map stormConf) { @Override public void start() { - LOG.info("Starting..."); - reporter.start(); + if (reporter != null ) { + LOG.info("Starting..."); + reporter.start(); + } else { + throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName()); + } } @Override public void stop() { - LOG.info("Stopping..."); - reporter.stop(); + if (reporter !=null) { + LOG.info("Stopping..."); + reporter.stop(); + } else { + throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName()); + } } } From e84528bb10196e6441d8784d5b8125b690027129 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 3 Feb 2016 16:21:06 -0600 Subject: [PATCH 034/387] Adding Apache license header to new files. --- .../storm/statistics/StatisticsUtils.java | 17 +++++++++++++++++ .../reporters/ConsolePreparableReporter.java | 17 +++++++++++++++++ .../reporters/CsvPreparableReporter.java | 17 +++++++++++++++++ .../reporters/JmxPreparableReporter.java | 17 +++++++++++++++++ .../reporters/PreparableReporter.java | 17 +++++++++++++++++ 5 files changed, 85 insertions(+) diff --git a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java b/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java index 666e44db5df..ba7edc4e5b9 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java @@ -1,3 +1,20 @@ +/** + * 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 org.apache.storm.statistics; import org.apache.storm.Config; diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java index f545b5b0126..35ae83f6bf0 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java @@ -1,3 +1,20 @@ +/** + * 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 org.apache.storm.statistics.reporters; import com.codahale.metrics.ConsoleReporter; diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java index 610df33fc28..8ed0b3e7bc5 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java @@ -1,3 +1,20 @@ +/** + * 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 org.apache.storm.statistics.reporters; import com.codahale.metrics.CsvReporter; diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java index ba596114128..6b0cbdacd1e 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java @@ -1,3 +1,20 @@ +/** + * 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 org.apache.storm.statistics.reporters; import com.codahale.metrics.JmxReporter; diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java index f6e8b2bcd26..ce3e8fedba0 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java @@ -1,3 +1,20 @@ +/** + * 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 org.apache.storm.statistics.reporters; import com.codahale.metrics.MetricRegistry; From 9368619fb745da2e370881264f4c8278cb3b5bbf Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 3 Feb 2016 16:35:36 -0600 Subject: [PATCH 035/387] Addressing Code review comments --- conf/defaults.yaml | 2 +- storm-core/src/jvm/org/apache/storm/Config.java | 5 ++--- .../jvm/org/apache/storm/statistics/StatisticsUtils.java | 2 +- .../storm/statistics/reporters/PreparableReporter.java | 6 +++--- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index b468290b9f3..5df4a6351ce 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -283,5 +283,5 @@ pacemaker.auth.method: "NONE" pacemaker.kerberos.users: [] #default plugin for daemon statistics reporter -storm.statistics.preparable.reporter.plugin: +storm.statistics.preparable.reporter.plugins: - "org.apache.storm.statistics.reporters.JmxPreparableReporter" diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index 9d18667ca08..adeb4d66256 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -140,11 +140,10 @@ public class Config extends HashMap { public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate"; /** - * A list of statistics preparable reporter class names. + * A list of daemon statistics reporter plugin class names. */ - @NotNull @isStringList - public static final String STORM_STATISTICS_PREPARABLE_REPORTER_PLUGIN = "storm.statistics.preparable.reporter.plugin"; + public static final String STORM_STATISTICS_PREPARABLE_REPORTER_PLUGINS = "storm.statistics.preparable.reporter.plugins"; /** * A list of hosts of ZooKeeper servers used to manage the cluster. diff --git a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java b/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java index ba7edc4e5b9..12d33c4fc19 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java @@ -33,7 +33,7 @@ public class StatisticsUtils { public static List getPreparableReporters(Map stormConf) { PreparableReporter reporter = new JmxPreparableReporter(); - List clazzes = (List) stormConf.get(Config.STORM_STATISTICS_PREPARABLE_REPORTER_PLUGIN); + List clazzes = (List) stormConf.get(Config.STORM_STATISTICS_PREPARABLE_REPORTER_PLUGINS); List reporterList = new ArrayList<>(); if (clazzes != null) { diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java b/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java index ce3e8fedba0..dc29a4a8ca5 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java @@ -25,8 +25,8 @@ public interface PreparableReporter { - public abstract void prepare(MetricRegistry metricsRegistry, Map stormConf); - public abstract void start(); - public abstract void stop(); + public void prepare(MetricRegistry metricsRegistry, Map stormConf); + public void start(); + public void stop(); } From 2207d662b0c45fd84edf0f9a9c64d5651b182aab Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 4 Feb 2016 10:25:29 -0600 Subject: [PATCH 036/387] Renaming the package and config --- conf/defaults.yaml | 6 +++--- storm-core/src/jvm/org/apache/storm/Config.java | 4 ++-- .../{statistics => daemon/metrics}/StatisticsUtils.java | 8 ++++---- .../metrics}/reporters/ConsolePreparableReporter.java | 2 +- .../metrics}/reporters/CsvPreparableReporter.java | 2 +- .../metrics}/reporters/JmxPreparableReporter.java | 2 +- .../metrics}/reporters/PreparableReporter.java | 2 +- 7 files changed, 13 insertions(+), 13 deletions(-) rename storm-core/src/jvm/org/apache/storm/{statistics => daemon/metrics}/StatisticsUtils.java (90%) rename storm-core/src/jvm/org/apache/storm/{statistics => daemon/metrics}/reporters/ConsolePreparableReporter.java (98%) rename storm-core/src/jvm/org/apache/storm/{statistics => daemon/metrics}/reporters/CsvPreparableReporter.java (98%) rename storm-core/src/jvm/org/apache/storm/{statistics => daemon/metrics}/reporters/JmxPreparableReporter.java (98%) rename storm-core/src/jvm/org/apache/storm/{statistics => daemon/metrics}/reporters/PreparableReporter.java (95%) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 5df4a6351ce..d381f0d72b6 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -282,6 +282,6 @@ pacemaker.childopts: "-Xmx1024m" pacemaker.auth.method: "NONE" pacemaker.kerberos.users: [] -#default plugin for daemon statistics reporter -storm.statistics.preparable.reporter.plugins: - - "org.apache.storm.statistics.reporters.JmxPreparableReporter" +#default storm daemon metrics reporter plugins +storm.daemon.metrics.reporter.plugins: + - "org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter" diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index adeb4d66256..100a824a980 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -140,10 +140,10 @@ public class Config extends HashMap { public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate"; /** - * A list of daemon statistics reporter plugin class names. + * A list of daemon metrics reporter plugin class names. */ @isStringList - public static final String STORM_STATISTICS_PREPARABLE_REPORTER_PLUGINS = "storm.statistics.preparable.reporter.plugins"; + public static final String STORM_DAEMON_METRICS_REPORTER_PLUGINS = "storm.daemon.metrics.reporter.plugins"; /** * A list of hosts of ZooKeeper servers used to manage the cluster. diff --git a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/StatisticsUtils.java similarity index 90% rename from storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java rename to storm-core/src/jvm/org/apache/storm/daemon/metrics/StatisticsUtils.java index 12d33c4fc19..d28e66737cb 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/StatisticsUtils.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/StatisticsUtils.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.statistics; +package org.apache.storm.daemon.metrics; import org.apache.storm.Config; -import org.apache.storm.statistics.reporters.JmxPreparableReporter; -import org.apache.storm.statistics.reporters.PreparableReporter; +import org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter; +import org.apache.storm.daemon.metrics.reporters.PreparableReporter; import org.apache.storm.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,7 +33,7 @@ public class StatisticsUtils { public static List getPreparableReporters(Map stormConf) { PreparableReporter reporter = new JmxPreparableReporter(); - List clazzes = (List) stormConf.get(Config.STORM_STATISTICS_PREPARABLE_REPORTER_PLUGINS); + List clazzes = (List) stormConf.get(Config.STORM_DAEMON_METRICS_REPORTER_PLUGINS); List reporterList = new ArrayList<>(); if (clazzes != null) { diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java similarity index 98% rename from storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java rename to storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java index 35ae83f6bf0..1b987a8684e 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/ConsolePreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.statistics.reporters; +package org.apache.storm.daemon.metrics.reporters; import com.codahale.metrics.ConsoleReporter; import com.codahale.metrics.MetricFilter; diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java similarity index 98% rename from storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java rename to storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java index 8ed0b3e7bc5..77d5393ef7a 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/CsvPreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.statistics.reporters; +package org.apache.storm.daemon.metrics.reporters; import com.codahale.metrics.CsvReporter; import com.codahale.metrics.MetricFilter; diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java similarity index 98% rename from storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java rename to storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java index 6b0cbdacd1e..988bb47d52e 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/JmxPreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.statistics.reporters; +package org.apache.storm.daemon.metrics.reporters; import com.codahale.metrics.JmxReporter; import com.codahale.metrics.MetricFilter; diff --git a/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java similarity index 95% rename from storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java rename to storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java index dc29a4a8ca5..f19f8b1d296 100644 --- a/storm-core/src/jvm/org/apache/storm/statistics/reporters/PreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.statistics.reporters; +package org.apache.storm.daemon.metrics.reporters; import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.Reporter; From d51981559e015c5b3b26e05eb07988c3a34bb737 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 4 Feb 2016 13:16:26 -0600 Subject: [PATCH 037/387] Addressing comments about reporter configs Conflicts: storm-core/src/clj/org/apache/storm/daemon/common.clj --- .../clj/org/apache/storm/daemon/common.clj | 2 +- .../src/jvm/org/apache/storm/Config.java | 24 ++++++++++++++ ...StatisticsUtils.java => MetricsUtils.java} | 31 ++++++++++++++++-- .../reporters/ConsolePreparableReporter.java | 29 ++++++++--------- .../reporters/CsvPreparableReporter.java | 32 +++++++++---------- .../reporters/JmxPreparableReporter.java | 17 ++++------ .../metrics/reporters/PreparableReporter.java | 6 ++-- 7 files changed, 93 insertions(+), 48 deletions(-) rename storm-core/src/jvm/org/apache/storm/daemon/metrics/{StatisticsUtils.java => MetricsUtils.java} (64%) diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj index 6b7d539658c..3c9eaca12d5 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/common.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj @@ -40,7 +40,7 @@ (log-message "Started statistics report plugin...")) (defn start-metrics-reporters [conf] - (doseq [reporter (StatisticsUtils/getPreparableReporters conf)] + (doseq [reporter (MetricsUtils/getPreparableReporters conf)] (start-metrics-reporter reporter conf))) diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index 100a824a980..49306eb32d2 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -145,6 +145,30 @@ public class Config extends HashMap { @isStringList public static final String STORM_DAEMON_METRICS_REPORTER_PLUGINS = "storm.daemon.metrics.reporter.plugins"; + /** + * A specify Locale for daemon metrics reporter plugin. + * Use the specified IETF BCP 47 language tag string for a Locale. + */ + @isString + public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_LOCALE = "storm.daemon.metrics.reporter.plugin.local"; + + /** + * A specify domain for daemon metrics reporter plugin to limit reporting to specific domain. + */ + @isString + public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_DOMAIN = "storm.daemon.metrics.reporter.plugin.domain"; + + /** + * A specify rate-unit in TimeUnit to specify reporting frequency for daemon metrics reporter plugin. + */ + @isString + public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_RATE_UNIT = "storm.daemon.metrics.reporter.plugin.rate.unit"; + + /** + * A specify duration-unit in TimeUnit to specify reporting window for daemon metrics reporter plugin. + */ + @isString + public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_DURATION_UNIT = "storm.daemon.metrics.reporter.plugin.duration.unit"; /** * A list of hosts of ZooKeeper servers used to manage the cluster. */ diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/StatisticsUtils.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java similarity index 64% rename from storm-core/src/jvm/org/apache/storm/daemon/metrics/StatisticsUtils.java rename to storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java index d28e66737cb..4425f598787 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/StatisticsUtils.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java @@ -26,13 +26,14 @@ import java.util.ArrayList; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.concurrent.TimeUnit; -public class StatisticsUtils { - private final static Logger LOG = LoggerFactory.getLogger(StatisticsUtils.class); +public class MetricsUtils { + private final static Logger LOG = LoggerFactory.getLogger(MetricsUtils.class); public static List getPreparableReporters(Map stormConf) { - PreparableReporter reporter = new JmxPreparableReporter(); List clazzes = (List) stormConf.get(Config.STORM_DAEMON_METRICS_REPORTER_PLUGINS); List reporterList = new ArrayList<>(); @@ -55,4 +56,28 @@ private static PreparableReporter getPreparableReporter(String clazz) { } return reporter; } + + public static Locale getMetricsReporterLocale(Map stormConf) { + String languageTag = Utils.getString(stormConf.get(Config.STORM_DAEMON_METRICS_REPORTER_PLUGIN_LOCALE), null); + if(languageTag != null) { + return Locale.forLanguageTag(languageTag); + } + return null; + } + + public static TimeUnit getMetricsRateUnit(Map stormConf) { + return getTimeUnitForCofig(stormConf, Config.STORM_DAEMON_METRICS_REPORTER_PLUGIN_RATE_UNIT); + } + + public static TimeUnit getMetricsDurationUnit(Map stormConf) { + return getTimeUnitForCofig(stormConf, Config.STORM_DAEMON_METRICS_REPORTER_PLUGIN_DURATION_UNIT); + } + + private static TimeUnit getTimeUnitForCofig(Map stormConf, String configName) { + String rateUnitString = Utils.getString(stormConf.get(configName), null); + if(rateUnitString != null) { + return TimeUnit.valueOf(rateUnitString); + } + return null; + } } diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java index 1b987a8684e..2f466ce0adc 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java @@ -18,9 +18,8 @@ package org.apache.storm.daemon.metrics.reporters; import com.codahale.metrics.ConsoleReporter; -import com.codahale.metrics.MetricFilter; import com.codahale.metrics.MetricRegistry; -import org.apache.storm.utils.Utils; +import org.apache.storm.daemon.metrics.MetricsUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,27 +34,27 @@ public class ConsolePreparableReporter implements PreparableReporter { @Override public void prepare(MetricRegistry metricsRegistry, Map stormConf) { - LOG.info("Preparing..."); + LOG.debug("Preparing..."); CsvReporter.Builder builder = CsvReporter.forRegistry(metricsRegistry); - Locale locale = (Locale) stormConf.get(":locale"); + Locale locale = MetricsUtils.getMetricsReporterLocale(stormConf); if (locale != null) { builder.formatFor(locale); } - String rateUnit = Utils.getString(stormConf.get(":rate-unit"), null); + + TimeUnit rateUnit = MetricsUtils.getMetricsRateUnit(stormConf); if (rateUnit != null) { - builder.convertRatesTo(TimeUnit.valueOf(rateUnit)); + builder.convertRatesTo(rateUnit); } - String durationUnit = Utils.getString(stormConf.get(":duration-unit"), null); + + TimeUnit durationUnit = MetricsUtils.getMetricsDurationUnit(stormConf); if (durationUnit != null) { - builder.convertDurationsTo(TimeUnit.valueOf(durationUnit)); - } - MetricFilter filter = (MetricFilter) stormConf.get(":filter"); - if (filter != null) { - builder.filter(filter); + builder.convertDurationsTo(durationUnit); } + String localStormDirLocation = Utils.getString(stormConf.get(Config.STORM_LOCAL_DIR), "."); - File logDir = new File(localStormDirLocation + "csvmetrics" ); - validateCreateOutputDir(logDir); - reporter = builder.build(logDir); + File csvMetricsDir = new File(localStormDirLocation + System.getProperty("file.separator") + "csvmetrics" ); + validateCreateOutputDir(csvMetricsDir); + + reporter = builder.build(csvMetricsDir); } @Override public void start() { if (reporter != null) { - LOG.info("Starting..."); + LOG.debug("Starting..."); reporter.start(10, TimeUnit.SECONDS); } else { throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName()); @@ -74,7 +74,7 @@ public void start() { @Override public void stop() { if (reporter != null) { - LOG.info("Stopping..."); + LOG.debug("Stopping..."); reporter.stop(); } else { throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName()); diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java index 988bb47d52e..eff6e5a38a6 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java @@ -18,8 +18,9 @@ package org.apache.storm.daemon.metrics.reporters; import com.codahale.metrics.JmxReporter; -import com.codahale.metrics.MetricFilter; import com.codahale.metrics.MetricRegistry; +import org.apache.storm.Config; +import org.apache.storm.daemon.metrics.MetricsUtils; import org.apache.storm.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,17 +36,13 @@ public class JmxPreparableReporter implements PreparableReporter { public void prepare(MetricRegistry metricsRegistry, Map stormConf) { LOG.info("Preparing..."); JmxReporter.Builder builder = JmxReporter.forRegistry(metricsRegistry); - String domain = Utils.getString(stormConf.get(":domain"), null); + String domain = Utils.getString(stormConf.get(Config.STORM_DAEMON_METRICS_REPORTER_PLUGIN_DOMAIN), null); if (domain != null) { builder.inDomain(domain); } - String rateUnit = Utils.getString(stormConf.get(":rate-unit"), null); + TimeUnit rateUnit = MetricsUtils.getMetricsRateUnit(stormConf); if (rateUnit != null) { - builder.convertRatesTo(TimeUnit.valueOf(rateUnit)); - } - MetricFilter filter = (MetricFilter) stormConf.get(":filter"); - if (filter != null) { - builder.filter(filter); + builder.convertRatesTo(rateUnit); } reporter = builder.build(); @@ -54,7 +51,7 @@ public void prepare(MetricRegistry metricsRegistry, Map stormConf) { @Override public void start() { if (reporter != null ) { - LOG.info("Starting..."); + LOG.debug("Starting..."); reporter.start(); } else { throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName()); @@ -64,7 +61,7 @@ public void start() { @Override public void stop() { if (reporter !=null) { - LOG.info("Stopping..."); + LOG.debug("Stopping..."); reporter.stop(); } else { throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName()); diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java index f19f8b1d296..2968bfb0251 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java @@ -25,8 +25,8 @@ public interface PreparableReporter { - public void prepare(MetricRegistry metricsRegistry, Map stormConf); - public void start(); - public void stop(); + void prepare(MetricRegistry metricsRegistry, Map stormConf); + void start(); + void stop(); } From 6811c9b82873ca48efbc337e5493c7aba85d5731 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 4 Feb 2016 14:19:02 -0600 Subject: [PATCH 038/387] Addressing comments about reporter configs --- .../src/jvm/org/apache/storm/Config.java | 10 ++++- .../storm/daemon/metrics/MetricsUtils.java | 39 +++++++++++++++---- .../reporters/ConsolePreparableReporter.java | 8 ++-- .../reporters/CsvPreparableReporter.java | 23 ++--------- .../reporters/JmxPreparableReporter.java | 8 ++-- 5 files changed, 52 insertions(+), 36 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index 49306eb32d2..a456bb2d4f4 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -150,7 +150,7 @@ public class Config extends HashMap { * Use the specified IETF BCP 47 language tag string for a Locale. */ @isString - public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_LOCALE = "storm.daemon.metrics.reporter.plugin.local"; + public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_LOCALE = "storm.daemon.metrics.reporter.plugin.locale"; /** * A specify domain for daemon metrics reporter plugin to limit reporting to specific domain. @@ -169,6 +169,14 @@ public class Config extends HashMap { */ @isString public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_DURATION_UNIT = "storm.daemon.metrics.reporter.plugin.duration.unit"; + + + /** + * A specify csv reporter directory for CvsPreparableReporter daemon metrics reporter. + */ + @isString + public static final String STORM_DAEMON_METRICS_REPORTER_CSV_LOG_DIR = "storm.daemon.metrics.reporter.csv.log.dir"; + /** * A list of hosts of ZooKeeper servers used to manage the cluster. */ diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java index 4425f598787..aa5ce2857fe 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java @@ -6,9 +6,9 @@ * 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. @@ -20,10 +20,12 @@ import org.apache.storm.Config; import org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter; import org.apache.storm.daemon.metrics.reporters.PreparableReporter; +import org.apache.storm.utils.ConfigUtils; import org.apache.storm.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.util.ArrayList; import java.util.List; import java.util.Locale; @@ -38,11 +40,11 @@ public static List getPreparableReporters(Map stormConf) { List reporterList = new ArrayList<>(); if (clazzes != null) { - for(String clazz: clazzes ) { + for (String clazz : clazzes) { reporterList.add(getPreparableReporter(clazz)); } } - if(reporterList.isEmpty()) { + if (reporterList.isEmpty()) { reporterList.add(new JmxPreparableReporter()); } return reporterList; @@ -51,7 +53,7 @@ public static List getPreparableReporters(Map stormConf) { private static PreparableReporter getPreparableReporter(String clazz) { PreparableReporter reporter = null; LOG.info("Using statistics reporter plugin:" + clazz); - if(clazz != null) { + if (clazz != null) { reporter = (PreparableReporter) Utils.newInstance(clazz); } return reporter; @@ -59,7 +61,7 @@ private static PreparableReporter getPreparableReporter(String clazz) { public static Locale getMetricsReporterLocale(Map stormConf) { String languageTag = Utils.getString(stormConf.get(Config.STORM_DAEMON_METRICS_REPORTER_PLUGIN_LOCALE), null); - if(languageTag != null) { + if (languageTag != null) { return Locale.forLanguageTag(languageTag); } return null; @@ -75,9 +77,32 @@ public static TimeUnit getMetricsDurationUnit(Map stormConf) { private static TimeUnit getTimeUnitForCofig(Map stormConf, String configName) { String rateUnitString = Utils.getString(stormConf.get(configName), null); - if(rateUnitString != null) { + if (rateUnitString != null) { return TimeUnit.valueOf(rateUnitString); } return null; } + + public static File getCsvLogDir(Map stormConf) { + String csvMetricsLogDirectory = Utils.getString(stormConf.get(Config.STORM_DAEMON_METRICS_REPORTER_CSV_LOG_DIR), null); + if (csvMetricsLogDirectory == null) { + csvMetricsLogDirectory = ConfigUtils.absoluteHealthCheckDir(stormConf); + csvMetricsLogDirectory = csvMetricsLogDirectory + ConfigUtils.FILE_SEPARATOR + "csvmetrics"; + } + File csvMetricsDir = new File(csvMetricsLogDirectory); + validateCreateOutputDir(csvMetricsDir); + return csvMetricsDir; + } + + private static void validateCreateOutputDir(File dir) { + if (!dir.exists()) { + dir.mkdirs(); + } + if (!dir.canWrite()) { + throw new IllegalStateException(dir.getName() + " does not have write permissions."); + } + if (!dir.isDirectory()) { + throw new IllegalStateException(dir.getName() + " is not a directory."); + } + } } diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java index 2f466ce0adc..3ef42372395 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java @@ -6,9 +6,9 @@ * 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. @@ -61,7 +61,7 @@ public void prepare(MetricRegistry metricsRegistry, Map stormConf) { @Override public void start() { - if (reporter != null ) { + if (reporter != null) { LOG.debug("Starting..."); reporter.start(10, TimeUnit.SECONDS); } else { @@ -71,7 +71,7 @@ public void start() { @Override public void stop() { - if (reporter !=null) { + if (reporter != null) { LOG.debug("Stopping..."); reporter.stop(); } else { diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java index 28fd6053dc4..605f389a6ae 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java @@ -6,9 +6,9 @@ * 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. @@ -19,9 +19,7 @@ import com.codahale.metrics.CsvReporter; import com.codahale.metrics.MetricRegistry; -import org.apache.storm.Config; import org.apache.storm.daemon.metrics.MetricsUtils; -import org.apache.storm.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,10 +52,7 @@ public void prepare(MetricRegistry metricsRegistry, Map stormConf) { builder.convertDurationsTo(durationUnit); } - String localStormDirLocation = Utils.getString(stormConf.get(Config.STORM_LOCAL_DIR), "."); - File csvMetricsDir = new File(localStormDirLocation + System.getProperty("file.separator") + "csvmetrics" ); - validateCreateOutputDir(csvMetricsDir); - + File csvMetricsDir = MetricsUtils.getCsvLogDir(stormConf); reporter = builder.build(csvMetricsDir); } @@ -81,17 +76,5 @@ public void stop() { } } - - private void validateCreateOutputDir(File dir) { - if (!dir.exists()) { - dir.mkdirs(); - } - if (!dir.canWrite()) { - throw new IllegalStateException(dir.getName() + " does not have write permissions."); - } - if (!dir.isDirectory()) { - throw new IllegalStateException(dir.getName() + " is not a directory."); - } - } } diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java index eff6e5a38a6..cf4aa1c9443 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java @@ -6,9 +6,9 @@ * 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. @@ -50,7 +50,7 @@ public void prepare(MetricRegistry metricsRegistry, Map stormConf) { @Override public void start() { - if (reporter != null ) { + if (reporter != null) { LOG.debug("Starting..."); reporter.start(); } else { @@ -60,7 +60,7 @@ public void start() { @Override public void stop() { - if (reporter !=null) { + if (reporter != null) { LOG.debug("Stopping..."); reporter.stop(); } else { From 257f1d355367a3ea24e099f65a3b11c3907e037a Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 4 Feb 2016 14:47:32 -0600 Subject: [PATCH 039/387] Modify config variable documentation --- storm-core/src/jvm/org/apache/storm/Config.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index a456bb2d4f4..df0e64cb94f 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -140,7 +140,8 @@ public class Config extends HashMap { public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate"; /** - * A list of daemon metrics reporter plugin class names. + * A list of daemon metrics reporter plugin class names. The classes should implement + * These plugins must implement {@link org.apache.storm.daemon.metrics.reporters.PreparableReporter} interface. */ @isStringList public static final String STORM_DAEMON_METRICS_REPORTER_PLUGINS = "storm.daemon.metrics.reporter.plugins"; From 137efb108c2084d1cf371c899d6cd454b9166882 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 4 Feb 2016 15:29:17 -0600 Subject: [PATCH 040/387] Removing unnecessary null check on STDOUT stream --- .../metrics/reporters/ConsolePreparableReporter.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java index 3ef42372395..1eacb63f5c7 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java @@ -23,7 +23,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.PrintStream; import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -37,11 +36,7 @@ public void prepare(MetricRegistry metricsRegistry, Map stormConf) { LOG.debug("Preparing..."); ConsoleReporter.Builder builder = ConsoleReporter.forRegistry(metricsRegistry); - PrintStream stream = System.out; - if (stream != null) { - builder.outputTo(stream); - } - + builder.outputTo(System.out); Locale locale = MetricsUtils.getMetricsReporterLocale(stormConf); if (locale != null) { builder.formattedFor(locale); From 2b8c8424ec80b75c37bea46f75b8c44a8c095198 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 4 Feb 2016 22:12:59 -0600 Subject: [PATCH 041/387] Fixing java docs --- .../clj/org/apache/storm/daemon/common.clj | 4 ++-- .../src/jvm/org/apache/storm/Config.java | 2 +- .../storm/daemon/metrics/MetricsUtils.java | 19 ++++++++++++++++--- 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj index 3c9eaca12d5..c1e261fabf3 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/common.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj @@ -19,9 +19,9 @@ InvalidTopologyException GlobalStreamId] [org.apache.storm.utils ThriftTopologyUtils]) (:import [org.apache.storm.utils Utils]) - (:import [org.apache.storm.statistics.reporters PreparableReporter] + (:import [org.apache.storm.daemon.metrics.reporters PreparableReporter] [com.codahale.metrics MetricRegistry] - [org.apache.storm.statistics StatisticsUtils]) + [org.apache.storm.daemon.metrics MetricsUtils]) (:import [org.apache.storm.task WorkerTopologyContext]) (:import [org.apache.storm Constants]) (:import [org.apache.storm.metric SystemBolt]) diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index df0e64cb94f..74231a06f0d 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -140,7 +140,7 @@ public class Config extends HashMap { public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate"; /** - * A list of daemon metrics reporter plugin class names. The classes should implement + * A list of daemon metrics reporter plugin class names. * These plugins must implement {@link org.apache.storm.daemon.metrics.reporters.PreparableReporter} interface. */ @isStringList diff --git a/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java b/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java index aa5ce2857fe..56b920b2a5d 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/metrics/MetricsUtils.java @@ -20,7 +20,6 @@ import org.apache.storm.Config; import org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter; import org.apache.storm.daemon.metrics.reporters.PreparableReporter; -import org.apache.storm.utils.ConfigUtils; import org.apache.storm.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,8 +85,8 @@ private static TimeUnit getTimeUnitForCofig(Map stormConf, String configName) { public static File getCsvLogDir(Map stormConf) { String csvMetricsLogDirectory = Utils.getString(stormConf.get(Config.STORM_DAEMON_METRICS_REPORTER_CSV_LOG_DIR), null); if (csvMetricsLogDirectory == null) { - csvMetricsLogDirectory = ConfigUtils.absoluteHealthCheckDir(stormConf); - csvMetricsLogDirectory = csvMetricsLogDirectory + ConfigUtils.FILE_SEPARATOR + "csvmetrics"; + csvMetricsLogDirectory = absoluteStormLocalDir(stormConf); + csvMetricsLogDirectory = csvMetricsLogDirectory + File.separator + "csvmetrics"; } File csvMetricsDir = new File(csvMetricsLogDirectory); validateCreateOutputDir(csvMetricsDir); @@ -105,4 +104,18 @@ private static void validateCreateOutputDir(File dir) { throw new IllegalStateException(dir.getName() + " is not a directory."); } } + + public static String absoluteStormLocalDir(Map conf) { + String stormHome = System.getProperty("storm.home"); + String localDir = (String) conf.get(Config.STORM_LOCAL_DIR); + if (localDir == null) { + return (stormHome + File.separator + "storm-local"); + } else { + if (new File(localDir).isAbsolute()) { + return localDir; + } else { + return (stormHome + File.separator + localDir); + } + } + } } From 1582c220a92170590a58450457ac5007c757a396 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 5 Feb 2016 19:33:05 +0000 Subject: [PATCH 042/387] add STORM-1524 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 56529ce57cc..2aa68829869 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1524: Add Pluggable daemon metrics Reporters * STORM-1517: Add peek api in trident stream * STORM-1455: kafka spout should not reset to the beginning of partition when offsetoutofrange exception occurs * STORM-1518: Backport of STORM-1504 From fa2cdff139fe9ca9403a94687403ab9c2f98aa4c Mon Sep 17 00:00:00 2001 From: Shoeb Mohammed Date: Mon, 8 Feb 2016 16:24:20 -0600 Subject: [PATCH 043/387] Added scope to dependencies in storm-elasticsearch pom.xml --- external/storm-elasticsearch/pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/external/storm-elasticsearch/pom.xml b/external/storm-elasticsearch/pom.xml index 91583cedf68..efd36ffbc85 100644 --- a/external/storm-elasticsearch/pom.xml +++ b/external/storm-elasticsearch/pom.xml @@ -61,6 +61,7 @@ junit junit + test org.hamcrest @@ -77,6 +78,7 @@ org.mockito mockito-all + test From 8250a47336607e06c7c73c5ab1aee7ba7bcfe4ad Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Thu, 4 Feb 2016 18:06:07 -0800 Subject: [PATCH 044/387] STORM-1526 fix perf issue related to clojure dynamic method lookup in the spout.nextTuple() call tree --- storm-core/src/clj/org/apache/storm/daemon/executor.clj | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj index c742352ab53..251387b485b 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj @@ -58,7 +58,7 @@ (.prepare grouping context (GlobalStreamId. component-id stream-id) target-tasks) (if (instance? LoadAwareCustomStreamGrouping grouping) (fn [task-id ^List values load] - (.chooseTasks grouping task-id values load)) + (.chooseTasks ^LoadAwareCustomStreamGrouping grouping task-id values load)) (fn [task-id ^List values load] (.chooseTasks grouping task-id values)))) From c96c4eef5348806e2c396df7a58001997e7dad54 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 9 Feb 2016 11:11:02 +0900 Subject: [PATCH 045/387] add STORM-1526 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2aa68829869..4f0c7f985e6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1526 Improve Storm core performance * STORM-1524: Add Pluggable daemon metrics Reporters * STORM-1517: Add peek api in trident stream * STORM-1455: kafka spout should not reset to the beginning of partition when offsetoutofrange exception occurs From 0164e522ed59f41798eb2b75aef0bde41b17b927 Mon Sep 17 00:00:00 2001 From: Aaron Dossett Date: Mon, 8 Feb 2016 20:18:06 -0600 Subject: [PATCH 046/387] Add STORM-1531 to CHANGELOG.md --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4f0c7f985e6..def5816587f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,6 @@ ## 1.0.0 - * STORM-1526 Improve Storm core performance + * STORM-1531: Junit and mockito dependencies need to have correct scope defined in storm-elasticsearch pom.xml + * STORM-1526: Improve Storm core performance * STORM-1524: Add Pluggable daemon metrics Reporters * STORM-1517: Add peek api in trident stream * STORM-1455: kafka spout should not reset to the beginning of partition when offsetoutofrange exception occurs From ed4a55f50f5e73589e2a59c99236651edf0f18b1 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 9 Feb 2016 11:26:06 +0900 Subject: [PATCH 047/387] STORM-1520 Nimbus Clojure/Zookeeper issue ("stateChanged" method not found) * fix a bug which passes wrong type of parameter to ClusterStateListener.stateChanged() * we passed ConnectionState from Curator which method needs storm's ConnectionState --- .../cluster_state/zookeeper_state_factory.clj | 3 +- .../utils/StormConnectionStateConverter.java | 44 +++++++++++++++++++ 2 files changed, 46 insertions(+), 1 deletion(-) create mode 100644 storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.java diff --git a/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj b/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj index 3104c528092..7eca32887f4 100644 --- a/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj +++ b/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj @@ -18,6 +18,7 @@ (:import [org.apache.curator.framework.state ConnectionStateListener]) (:import [org.apache.zookeeper KeeperException$NoNodeException] [org.apache.storm.cluster ClusterState DaemonType]) + (:import [org.apache.storm.utils StormConnectionStateConverter]) (:use [org.apache.storm cluster config log util]) (:require [org.apache.storm [zookeeper :as zk]]) (:gen-class @@ -142,7 +143,7 @@ (let [curator-listener (reify ConnectionStateListener (stateChanged [this client newState] - (.stateChanged listener client newState)))] + (.stateChanged listener (StormConnectionStateConverter/convert newState))))] (zk/add-listener zk-reader curator-listener))) (sync-path diff --git a/storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.java b/storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.java new file mode 100644 index 00000000000..587b5ebeffd --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.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 org.apache.storm.utils; + +import org.apache.storm.cluster.ConnectionState; + +import java.util.HashMap; +import java.util.Map; + +public class StormConnectionStateConverter { + + private static final Map mapCuratorToStorm = new HashMap<>(); + static { + mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.CONNECTED, ConnectionState.CONNECTED); + mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.LOST, ConnectionState.LOST); + mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.RECONNECTED, ConnectionState.RECONNECTED); + mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.READ_ONLY, ConnectionState.LOST); + mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.SUSPENDED, ConnectionState.LOST); + } + + public static ConnectionState convert(org.apache.curator.framework.state.ConnectionState state) { + ConnectionState stormState = mapCuratorToStorm.get(state); + if (stormState != null) { + return stormState; + } + throw new IllegalStateException("Unknown ConnectionState from Curator: " + state); + } +} \ No newline at end of file From ec25fad1bc18cfde4736c00016543658e610eeb7 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 9 Feb 2016 11:38:31 +0900 Subject: [PATCH 048/387] add STORM-1520 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index def5816587f..15a0fd42426 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1520: Nimbus Clojure/Zookeeper issue ("stateChanged" method not found) * STORM-1531: Junit and mockito dependencies need to have correct scope defined in storm-elasticsearch pom.xml * STORM-1526: Improve Storm core performance * STORM-1524: Add Pluggable daemon metrics Reporters From 1a2c66a7e34fd3971f85a20f5d46a80028566e7d Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Wed, 10 Feb 2016 11:49:21 +0530 Subject: [PATCH 049/387] Added STORM-1476 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 15a0fd42426..e3c19d63717 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1476: Filter -c options from args and add them as part of storm.options * STORM-1520: Nimbus Clojure/Zookeeper issue ("stateChanged" method not found) * STORM-1531: Junit and mockito dependencies need to have correct scope defined in storm-elasticsearch pom.xml * STORM-1526: Improve Storm core performance From 6a1e54f62bb82971dacb6097ceec686dfaca3e3c Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 11 Feb 2016 08:44:22 -0600 Subject: [PATCH 050/387] Merge branch 'release1' of https://github.com/abhishekagarwal87/storm into STORM-1533 STORM-1533: IntegerValidator for metric consumer parallelism hint STORM-1534: Pick correct version of jackson-annotations jar --- external/sql/storm-sql-core/pom.xml | 9 +++++++++ pom.xml | 1 - .../org/apache/storm/validation/ConfigValidation.java | 2 +- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/external/sql/storm-sql-core/pom.xml b/external/sql/storm-sql-core/pom.xml index 6b62e88b9f0..0700515766b 100644 --- a/external/sql/storm-sql-core/pom.xml +++ b/external/sql/storm-sql-core/pom.xml @@ -80,8 +80,17 @@ org.pentaho pentaho-aggdesigner-algorithm + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + commons-lang commons-lang diff --git a/pom.xml b/pom.xml index 025fb683d33..e85e4c155c7 100644 --- a/pom.xml +++ b/pom.xml @@ -231,7 +231,6 @@ 2.21 2.5 2.3 - 2.3.1 0.9.3 4.11 2.5.1 diff --git a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java index 4ec5ffda10f..edff5cfd953 100644 --- a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java +++ b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java @@ -473,7 +473,7 @@ public void validateField(String name, Object o) { } SimpleTypeValidator.validateField(name, String.class, ((Map) o).get("class")); - SimpleTypeValidator.validateField(name, Long.class, ((Map) o).get("parallelism.hint")); + new IntegerValidator().validateField(name, ((Map) o).get("parallelism.hint")); } } From afbbf235947ceddf7456e578b7185665d54a5baf Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 11 Feb 2016 09:32:08 -0600 Subject: [PATCH 051/387] Added STORM-1544 and STORM-1534 to Changelog Conflicts: CHANGELOG.md --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e3c19d63717..5777ab112a1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,6 @@ ## 1.0.0 + * STORM-1533: IntegerValidator for metric consumer parallelism hint + * STORM-1534: Pick correct version of jackson-annotations jar * STORM-1476: Filter -c options from args and add them as part of storm.options * STORM-1520: Nimbus Clojure/Zookeeper issue ("stateChanged" method not found) * STORM-1531: Junit and mockito dependencies need to have correct scope defined in storm-elasticsearch pom.xml From 63a2adb5fb81d4d4739000d227973d9702b5046b Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 11 Feb 2016 13:29:14 -0600 Subject: [PATCH 052/387] Merge branch 'STORM-1519' of https://github.com/jerrypeng/storm into STORM-1519 STORM-1519: Storm syslog logging not confirming to RFC5426 3.1 --- log4j2/cluster.xml | 2 +- log4j2/worker.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/log4j2/cluster.xml b/log4j2/cluster.xml index ca333b2e33e..baf5d446098 100644 --- a/log4j2/cluster.xml +++ b/log4j2/cluster.xml @@ -69,7 +69,7 @@ + messageId="[${sys:user.name}:S0]" id="storm" immediateFlush="true" immediateFail="true"/> diff --git a/log4j2/worker.xml b/log4j2/worker.xml index 967585b4937..f4988d46539 100644 --- a/log4j2/worker.xml +++ b/log4j2/worker.xml @@ -58,7 +58,7 @@ + messageId="[${sys:user.name}:${sys:logging.sensitivity}]" id="storm" immediateFail="true" immediateFlush="true"/> From 5fd312153e8c41014b229f72d52e2a8080988f1b Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 11 Feb 2016 13:40:40 -0600 Subject: [PATCH 053/387] Added STORM-1519 to Changelog Conflicts: CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5777ab112a1..ca286bb4b23 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1519: Storm syslog logging not confirming to RFC5426 3.1 * STORM-1533: IntegerValidator for metric consumer parallelism hint * STORM-1534: Pick correct version of jackson-annotations jar * STORM-1476: Filter -c options from args and add them as part of storm.options From 731410622cabdf6a5413e637f44cd7c0c6c657fa Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Thu, 11 Feb 2016 15:40:39 -0800 Subject: [PATCH 054/387] STORM-1539 - Improve Storm ACK-ing performance --- storm-core/src/clj/org/apache/storm/daemon/executor.clj | 2 +- storm-core/src/jvm/org/apache/storm/utils/Utils.java | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj index 251387b485b..be18c3b424f 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj @@ -581,7 +581,7 @@ (if (sampler) (System/currentTimeMillis))]) (task/send-unanchored task-data ACKER-INIT-STREAM-ID - [root-id (bit-xor-vals out-ids) task-id])) + [root-id (Utils/bitXorVals out-ids) task-id])) (when message-id (ack-spout-msg executor-data task-data message-id {:stream out-stream-id :values values} diff --git a/storm-core/src/jvm/org/apache/storm/utils/Utils.java b/storm-core/src/jvm/org/apache/storm/utils/Utils.java index 9d0c7c610a7..cbf34a2658f 100644 --- a/storm-core/src/jvm/org/apache/storm/utils/Utils.java +++ b/storm-core/src/jvm/org/apache/storm/utils/Utils.java @@ -250,6 +250,15 @@ public static String join(Iterable coll, String sep) { return ret.toString(); } + + public static long bitXorVals(List coll) { + long result = 0; + for (Long val : coll) { + result ^= val; + } + return result; + } + public static void sleep(long millis) { try { Time.sleep(millis); From ba1f953e960818ece09b6de5d461007d90854a96 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Fri, 12 Feb 2016 13:48:36 -0800 Subject: [PATCH 055/387] Removing unused clojure function o.a.s.util.bit-xor-vals --- storm-core/src/clj/org/apache/storm/util.clj | 4 ---- 1 file changed, 4 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/util.clj b/storm-core/src/clj/org/apache/storm/util.clj index 23d39f672c0..7241bfffb7f 100644 --- a/storm-core/src/clj/org/apache/storm/util.clj +++ b/storm-core/src/clj/org/apache/storm/util.clj @@ -844,10 +844,6 @@ [v] (or v 0)) -(defn bit-xor-vals - [vals] - (reduce bit-xor 0 vals)) - (defmacro with-error-reaction [afn & body] `(try ~@body From 406052cdc7138046a79104fc5c6f72212415f7f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Sun, 14 Feb 2016 02:39:42 +0100 Subject: [PATCH 056/387] STORM-1549: Add support for resetting tuple timeout from bolts via the OutputCollector --- .../src/clj/org/apache/storm/clojure.clj | 3 ++ .../src/clj/org/apache/storm/daemon/acker.clj | 10 +++- .../clj/org/apache/storm/daemon/common.clj | 9 +++- .../clj/org/apache/storm/daemon/executor.clj | 10 ++++ .../storm/coordination/CoordinatedBolt.java | 4 ++ .../apache/storm/task/IOutputCollector.java | 1 + .../apache/storm/task/OutputCollector.java | 10 ++++ .../storm/topology/BasicOutputCollector.java | 4 ++ .../storm/topology/IBasicOutputCollector.java | 2 + .../trident/topology/TridentBoltExecutor.java | 4 ++ .../org/apache/storm/integration_test.clj | 51 +++++++++++++++++-- 11 files changed, 103 insertions(+), 5 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/clojure.clj b/storm-core/src/clj/org/apache/storm/clojure.clj index ff338295cd5..6e88cb66fd5 100644 --- a/storm-core/src/clj/org/apache/storm/clojure.clj +++ b/storm-core/src/clj/org/apache/storm/clojure.clj @@ -173,6 +173,9 @@ (defn fail! [collector ^Tuple tuple] (.fail ^OutputCollector (:output-collector collector) tuple)) +(defn reset-timeout! [collector ^Tuple tuple] + (.resetTimeout ^OutputCollector (:output-collector collector) tuple)) + (defn report-error! [collector ^Tuple tuple] (.reportError ^OutputCollector (:output-collector collector) tuple)) diff --git a/storm-core/src/clj/org/apache/storm/daemon/acker.clj b/storm-core/src/clj/org/apache/storm/daemon/acker.clj index 7c4d6147147..7c29f467d07 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/acker.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/acker.clj @@ -30,6 +30,7 @@ (def ACKER-INIT-STREAM-ID "__ack_init") (def ACKER-ACK-STREAM-ID "__ack_ack") (def ACKER-FAIL-STREAM-ID "__ack_fail") +(def ACKER-RESET-TIMEOUT-STREAM-ID "__ack_reset_timeout") (defn- update-ack [curr-entry val] (let [old (get curr-entry :val 0)] @@ -61,7 +62,8 @@ (update-ack (.getValue tuple 1)) (assoc :spout-task (.getValue tuple 2))) ACKER-ACK-STREAM-ID (update-ack curr (.getValue tuple 1)) - ACKER-FAIL-STREAM-ID (assoc curr :failed true))] + ACKER-FAIL-STREAM-ID (assoc curr :failed true) + ACKER-RESET-TIMEOUT-STREAM-ID curr)] (.put pending id curr) (when (and curr (:spout-task curr)) (cond (= 0 (:val curr)) @@ -80,6 +82,12 @@ ACKER-FAIL-STREAM-ID [id] )) + (= stream-id ACKER-RESET-TIMEOUT-STREAM-ID) + (acker-emit-direct output-collector + (:spout-task curr) + ACKER-RESET-TIMEOUT-STREAM-ID + [id] + ) )) (.ack output-collector tuple) )))) diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj index c1e261fabf3..55bc030a63e 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/common.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj @@ -48,6 +48,7 @@ (def ACKER-INIT-STREAM-ID acker/ACKER-INIT-STREAM-ID) (def ACKER-ACK-STREAM-ID acker/ACKER-ACK-STREAM-ID) (def ACKER-FAIL-STREAM-ID acker/ACKER-FAIL-STREAM-ID) +(def ACKER-RESET-TIMEOUT-STREAM-ID acker/ACKER-RESET-TIMEOUT-STREAM-ID) (def SYSTEM-STREAM-ID "__system") @@ -195,7 +196,8 @@ bolt-inputs (apply merge (for [id bolt-ids] {[id ACKER-ACK-STREAM-ID] ["id"] - [id ACKER-FAIL-STREAM-ID] ["id"]} + [id ACKER-FAIL-STREAM-ID] ["id"] + [id ACKER-RESET-TIMEOUT-STREAM-ID] ["id"]} ))] (merge spout-inputs bolt-inputs))) @@ -221,6 +223,7 @@ (new org.apache.storm.daemon.acker) {ACKER-ACK-STREAM-ID (thrift/direct-output-fields ["id"]) ACKER-FAIL-STREAM-ID (thrift/direct-output-fields ["id"]) + ACKER-RESET-TIMEOUT-STREAM-ID (thrift/direct-output-fields ["id"]) } :p num-executors :conf {TOPOLOGY-TASKS num-executors @@ -230,6 +233,7 @@ (do (.put_to_streams common ACKER-ACK-STREAM-ID (thrift/output-fields ["id" "ack-val"])) (.put_to_streams common ACKER-FAIL-STREAM-ID (thrift/output-fields ["id"])) + (.put_to_streams common ACKER-RESET-TIMEOUT-STREAM-ID (thrift/output-fields ["id"])) )) (dofor [[_ spout] (.get_spouts ret) :let [common (.get_common spout) @@ -246,6 +250,9 @@ (.put_to_inputs common (GlobalStreamId. ACKER-COMPONENT-ID ACKER-FAIL-STREAM-ID) (thrift/mk-direct-grouping)) + (.put_to_inputs common + (GlobalStreamId. ACKER-COMPONENT-ID ACKER-RESET-TIMEOUT-STREAM-ID) + (thrift/mk-direct-grouping)) )) (.put_to_bolts ret "__acker" acker-bolt) )) diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj index 251387b485b..0d485484a70 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj @@ -519,6 +519,11 @@ spout-obj (:object task-data)] (when (instance? ICredentialsListener spout-obj) (.setCredentials spout-obj (.getValue tuple 0)))) + ACKER-RESET-TIMEOUT-STREAM-ID + (let [id (.getValue tuple 0) + pending-for-id (.get pending id)] + (when pending-for-id + (.put pending id pending-for-id))) (let [id (.getValue tuple 0) [stored-task-id spout-id tuple-finished-info start-time-ms] (.remove pending id)] (when spout-id @@ -828,6 +833,11 @@ (.getSourceComponent tuple) (.getSourceStreamId tuple) delta)))) + (^void resetTimeout [this ^Tuple tuple] + (fast-list-iter [root (.. tuple getMessageId getAnchors)] + (task/send-unanchored task-data + ACKER-RESET-TIMEOUT-STREAM-ID + [root]))) (reportError [this error] (report-error error) ))))) diff --git a/storm-core/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java b/storm-core/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java index ee66b094290..15ac5e2ca3d 100644 --- a/storm-core/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java +++ b/storm-core/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java @@ -124,6 +124,10 @@ public void fail(Tuple tuple) { checkFinishId(tuple, TupleType.REGULAR); _delegate.fail(tuple); } + + public void resetTimeout(Tuple tuple) { + _delegate.resetTimeout(tuple); + } public void reportError(Throwable error) { _delegate.reportError(error); diff --git a/storm-core/src/jvm/org/apache/storm/task/IOutputCollector.java b/storm-core/src/jvm/org/apache/storm/task/IOutputCollector.java index cbbe1083c0b..cda4d9f75af 100644 --- a/storm-core/src/jvm/org/apache/storm/task/IOutputCollector.java +++ b/storm-core/src/jvm/org/apache/storm/task/IOutputCollector.java @@ -29,4 +29,5 @@ public interface IOutputCollector extends IErrorReporter { void emitDirect(int taskId, String streamId, Collection anchors, List tuple); void ack(Tuple input); void fail(Tuple input); + void resetTimeout(Tuple input); } diff --git a/storm-core/src/jvm/org/apache/storm/task/OutputCollector.java b/storm-core/src/jvm/org/apache/storm/task/OutputCollector.java index e6e54acd966..071d8aaa9a0 100644 --- a/storm-core/src/jvm/org/apache/storm/task/OutputCollector.java +++ b/storm-core/src/jvm/org/apache/storm/task/OutputCollector.java @@ -218,6 +218,16 @@ public void fail(Tuple input) { _delegate.fail(input); } + /** + * Resets the message timeout for any tuple trees to which the given tuple belongs. + * The timeout is reset to Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS. + * @param input the tuple to reset timeout for + */ + @Override + public void resetTimeout(Tuple input) { + _delegate.resetTimeout(input); + } + @Override public void reportError(Throwable error) { _delegate.reportError(error); diff --git a/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java b/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java index cedc7c9dc4e..343c349ec06 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java +++ b/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java @@ -52,6 +52,10 @@ public void emitDirect(int taskId, List tuple) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); } + public void resetTimeout(Tuple tuple){ + out.resetTimeout(tuple); + } + protected IOutputCollector getOutputter() { return out; } diff --git a/storm-core/src/jvm/org/apache/storm/topology/IBasicOutputCollector.java b/storm-core/src/jvm/org/apache/storm/topology/IBasicOutputCollector.java index 60da48a9f89..7b7c9fc1589 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/IBasicOutputCollector.java +++ b/storm-core/src/jvm/org/apache/storm/topology/IBasicOutputCollector.java @@ -18,10 +18,12 @@ package org.apache.storm.topology; import org.apache.storm.task.IErrorReporter; +import org.apache.storm.tuple.Tuple; import java.util.List; public interface IBasicOutputCollector extends IErrorReporter{ List emit(String streamId, List tuple); void emitDirect(int taskId, String streamId, List tuple); + void resetTimeout(Tuple tuple); } diff --git a/storm-core/src/jvm/org/apache/storm/trident/topology/TridentBoltExecutor.java b/storm-core/src/jvm/org/apache/storm/trident/topology/TridentBoltExecutor.java index d85d217f0ad..41feb12e277 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/topology/TridentBoltExecutor.java +++ b/storm-core/src/jvm/org/apache/storm/trident/topology/TridentBoltExecutor.java @@ -180,6 +180,10 @@ public void ack(Tuple tuple) { public void fail(Tuple tuple) { throw new IllegalStateException("Method should never be called"); } + + public void resetTimeout(Tuple tuple) { + throw new IllegalStateException("Method should never be called"); + } public void reportError(Throwable error) { _delegate.reportError(error); diff --git a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj index cd2bc266866..238e0dbce2b 100644 --- a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj +++ b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj @@ -20,6 +20,7 @@ (:import [org.apache.storm.generated InvalidTopologyException SubmitOptions TopologyInitialStatus RebalanceOptions]) (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout]) + (:import [org.apache.storm.utils Time]) (:import [org.apache.storm.tuple Fields]) (:use [org.apache.storm testing config clojure util]) (:use [org.apache.storm.daemon common]) @@ -83,9 +84,18 @@ (ack! collector tuple) )))))) -(defn assert-loop [afn ids] - (while (not (every? afn ids)) - (Thread/sleep 1))) +(defn assert-loop +([afn ids] (assert-loop afn ids 10)) +([afn ids timeout-secs] + (loop [remaining-time (* timeout-secs 1000)] + (let [start-time (System/currentTimeMillis) + assertion-is-true (every? afn ids)] + (if (or assertion-is-true (neg? remaining-time)) + (is assertion-is-true) + (do + (Thread/sleep 1) + (recur (- remaining-time (- (System/currentTimeMillis) start-time))) + )))))) (defn assert-acked [tracker & ids] (assert-loop #(.isAcked tracker %) ids)) @@ -116,6 +126,41 @@ (assert-failed tracker 2) ))) +(defbolt extend-timeout-twice {} {:prepare true} + [conf context collector] + (let [state (atom -1)] + (bolt + (execute [tuple] + (do + (Time/sleep (* 8 1000)) + (reset-timeout! collector tuple) + (Time/sleep (* 8 1000)) + (reset-timeout! collector tuple) + (Time/sleep (* 8 1000)) + (ack! collector tuple) + ))))) + +(deftest test-reset-timeout + (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS true}] + (let [feeder (feeder-spout ["field1"]) + tracker (AckFailMapTracker.) + _ (.setAckFailDelegate feeder tracker) + topology (thrift/mk-topology + {"1" (thrift/mk-spout-spec feeder)} + {"2" (thrift/mk-bolt-spec {"1" :global} extend-timeout-twice)})] + (submit-local-topology (:nimbus cluster) + "timeout-tester" + {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10} + topology) + (advance-cluster-time cluster 11) + (.feed feeder ["a"] 1) + (advance-cluster-time cluster 21) + (is (not (.isFailed tracker 1))) + (is (not (.isAcked tracker 1))) + (advance-cluster-time cluster 5) + (assert-acked tracker 1) + ))) + (defn mk-validate-topology-1 [] (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 3)} From 6861ad332cb96e89f5b1c3f5651068fb4a7bfeb8 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 16 Feb 2016 17:52:07 -0800 Subject: [PATCH 057/387] Added STORM-1539 to CHANGELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index ca286bb4b23..3b91c20464d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1539: Improve Storm ACK-ing performance * STORM-1519: Storm syslog logging not confirming to RFC5426 3.1 * STORM-1533: IntegerValidator for metric consumer parallelism hint * STORM-1534: Pick correct version of jackson-annotations jar From b2f8c1b71cd24854c8be064fbd8c0ff8cc18dfaa Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Wed, 10 Feb 2016 11:23:34 +0530 Subject: [PATCH 058/387] [STORM-1532]: Fix readCommandLineOpts to parse JSON correctly In Windows env, the storm.options are not url-encoded. The parsing logic needs to be fixed to not split in the middle of raw JSON objects. --- bin/storm-config.cmd | 4 ++++ bin/storm.cmd | 2 +- storm-core/src/jvm/org/apache/storm/utils/Utils.java | 12 +++++++++++- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/bin/storm-config.cmd b/bin/storm-config.cmd index 2a91234e6d1..0a9ae5ec6e6 100644 --- a/bin/storm-config.cmd +++ b/bin/storm-config.cmd @@ -86,6 +86,10 @@ if not defined STORM_LOG_DIR ( @rem retrieve storm.log4j2.conf.dir from conf file @rem +if not defined CMD_TEMP_FILE ( + set CMD_TEMP_FILE=tmpfile +) + "%JAVA%" -client -Dstorm.options= -Dstorm.conf.file= -cp "%CLASSPATH%" org.apache.storm.command.config_value storm.log4j2.conf.dir > %CMD_TEMP_FILE% FOR /F "delims=" %%i in (%CMD_TEMP_FILE%) do ( diff --git a/bin/storm.cmd b/bin/storm.cmd index 6f4e934425c..20b7a85db34 100644 --- a/bin/storm.cmd +++ b/bin/storm.cmd @@ -90,7 +90,7 @@ ) if "%c-opt%"=="second" ( - set config-options=%config-options%=%1 + set config-options=%config-options%=%~1 set c-opt= goto start ) diff --git a/storm-core/src/jvm/org/apache/storm/utils/Utils.java b/storm-core/src/jvm/org/apache/storm/utils/Utils.java index cbf34a2658f..1a76b8a287a 100644 --- a/storm-core/src/jvm/org/apache/storm/utils/Utils.java +++ b/storm-core/src/jvm/org/apache/storm/utils/Utils.java @@ -355,7 +355,17 @@ public static Map readCommandLineOpts() { Map ret = new HashMap(); String commandOptions = System.getProperty("storm.options"); if (commandOptions != null) { - String[] configs = commandOptions.split(","); + /* + Below regex uses negative lookahead to not split in the middle of json objects '{}' + or json arrays '[]'. This is needed to parse valid json object/arrays passed as options + via 'storm.cmd' in windows. This is not an issue while using 'storm.py' since it url-encodes + the options and the below regex just does a split on the commas that separates each option. + + Note:- This regex handles only valid json strings and could produce invalid results + if the options contain un-encoded invalid json or strings with unmatched '[, ], { or }'. We can + replace below code with split(",") once 'storm.cmd' is fixed to send url-encoded options. + */ + String[] configs = commandOptions.split(",(?![^\\[\\]{}]*(]|}))"); for (String config : configs) { config = URLDecoder.decode(config); String[] options = config.split("=", 2); From b5f58d064a3bd451ebd068637f06c921daafe399 Mon Sep 17 00:00:00 2001 From: Satish Duggana Date: Wed, 17 Feb 2016 13:03:51 +0530 Subject: [PATCH 059/387] STORM-1522 should create error worker log location only when error-host and error-port are available --- storm-core/src/clj/org/apache/storm/ui/core.clj | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj index bd1dba5541a..9bd7a26b7dc 100644 --- a/storm-core/src/clj/org/apache/storm/ui/core.clj +++ b/storm-core/src/clj/org/apache/storm/ui/core.clj @@ -146,8 +146,10 @@ (logviewer-link host (event-logs-filename topology-id port) secure?)) (defn worker-log-link [host port topology-id secure?] - (let [fname (logs-filename topology-id port)] - (logviewer-link host fname secure?))) + (if (or (empty? host) (let [port_str (str port "")] (or (empty? port_str) (= "0" port_str)))) + "" + (let [fname (logs-filename topology-id port)] + (logviewer-link host fname secure?)))) (defn nimbus-log-link [host] (url-format "http://%s:%s/daemonlog?file=nimbus.log" host (*STORM-CONF* LOGVIEWER-PORT))) From 83ea6cb7ea7818b1b834cf7e448e7844db1e818e Mon Sep 17 00:00:00 2001 From: Satish Duggana Date: Fri, 29 Jan 2016 12:39:26 +0530 Subject: [PATCH 060/387] min/max operators implementation in Trident streams API. --- .../TridentMinMaxOperationsTopology.java | 208 ++++++++++++++++++ .../jvm/org/apache/storm/trident/Stream.java | 117 ++++++++-- .../builtin/ComparisonAggregator.java | 72 ++++++ .../storm/trident/operation/builtin/Max.java | 43 ++++ .../operation/builtin/MaxWithComparator.java | 44 ++++ .../storm/trident/operation/builtin/Min.java | 44 ++++ .../operation/builtin/MinWithComparator.java | 44 ++++ .../trident/testing/NumberGeneratorSpout.java | 92 ++++++++ 8 files changed, 651 insertions(+), 13 deletions(-) create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOperationsTopology.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/builtin/ComparisonAggregator.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Max.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Min.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/testing/NumberGeneratorSpout.java diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOperationsTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOperationsTopology.java new file mode 100644 index 00000000000..dedaaffbbcc --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOperationsTopology.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 org.apache.storm.starter.trident; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.trident.Stream; +import org.apache.storm.trident.TridentTopology; +import org.apache.storm.trident.operation.BaseFunction; +import org.apache.storm.trident.operation.TridentCollector; +import org.apache.storm.trident.operation.builtin.Debug; +import org.apache.storm.trident.testing.FixedBatchSpout; +import org.apache.storm.trident.testing.NumberGeneratorSpout; +import org.apache.storm.trident.tuple.TridentTuple; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; + +/** + * This class contains different usages of minBy, maxBy, min and max operations on trident streams. + * + */ +public class TridentMinMaxOperationsTopology { + public static class Split extends BaseFunction { + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + String sentence = tuple.getString(0); + for (String word : sentence.split(" ")) { + collector.emit(new Values(word)); + } + } + } + + public static StormTopology buildIdsTopology() { + NumberGeneratorSpout spout = new NumberGeneratorSpout(new Fields("id"), 10, 1000); + + TridentTopology topology = new TridentTopology(); + Stream wordsStream = topology.newStream("numgen-spout", spout). + each(new Fields("id"), new Debug("##### ids")); + + wordsStream.minBy("id"). + each(new Fields("id"), new Debug("#### min-id")); + + wordsStream.maxBy("id"). + each(new Fields("id"), new Debug("#### max-id")); + + return topology.build(); + } + + public static StormTopology buildWordsTopology() { + FixedBatchSpout spout = new FixedBatchSpout(new Fields("sentence"), 3, new Values("the cow jumped over the moon"), + new Values("the man went to the store and bought some candy"), new Values("four score and seven years ago"), + new Values("how many apples can you eat"), new Values("to be or not to be the person")); + spout.setCycle(true); + + TridentTopology topology = new TridentTopology(); + Stream wordsStream = topology.newStream("spout1", spout).parallelismHint(16). + each(new Fields("sentence"), new Split(), new Fields("word")). + each(new Fields("word"), new Debug("##### words")); + + wordsStream.minBy("word"). + each(new Fields("word"), new Debug("#### lowest word")); + + wordsStream.maxBy("word"). + each(new Fields("word"), new Debug("#### highest word")); + + return topology.build(); + } + + public static StormTopology buildVehiclesTopology() { + + FixedBatchSpout spout = new FixedBatchSpout(new Fields("vehicle", "driver"), 10, Vehicle.generateVehicles(20)); + spout.setCycle(true); + + TridentTopology topology = new TridentTopology(); + Stream vehiclesStream = topology.newStream("spout1", spout). + each(new Fields("vehicle"), new Debug("##### vehicles")); + + vehiclesStream.min(new SpeedComparator()) + .each(new Fields("vehicle"), new Debug("#### slowest vehicle")) + .project(new Fields("driver")). each(new Fields("driver"), new Debug("##### slowest driver")); + + vehiclesStream.max(new SpeedComparator()) + .each(new Fields("vehicle"), new Debug("#### fastest vehicle")) + .project(new Fields("driver")). each(new Fields("driver"), new Debug("##### fastest driver")); + + vehiclesStream.max(new EfficiencyComparator()). + each(new Fields("vehicle"), new Debug("#### efficient vehicle")); + + return topology.build(); + } + + public static void main(String[] args) throws Exception { + Config conf = new Config(); + conf.setMaxSpoutPending(20); + StormTopology[] topologies = {buildWordsTopology(), buildIdsTopology(), buildVehiclesTopology()}; + if (args.length == 0) { + for (StormTopology topology : topologies) { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("min-max-topology", conf, topology); + Utils.sleep(60*1000); + cluster.shutdown(); + } + System.exit(0); + } else { + conf.setNumWorkers(3); + int ct=1; + for (StormTopology topology : topologies) { + StormSubmitter.submitTopologyWithProgressBar(args[0]+"-"+ct++, conf, topology); + } + } + } + + static class SpeedComparator implements Comparator, Serializable { + + @Override + public int compare(TridentTuple tuple1, TridentTuple tuple2) { + Vehicle vehicle1 = (Vehicle) tuple1.getValueByField("vehicle"); + Vehicle vehicle2 = (Vehicle) tuple2.getValueByField("vehicle"); + return Integer.compare(vehicle1.maxSpeed, vehicle2.maxSpeed); + } + } + + static class EfficiencyComparator implements Comparator, Serializable { + + @Override + public int compare(TridentTuple tuple1, TridentTuple tuple2) { + Vehicle vehicle1 = (Vehicle) tuple1.getValueByField("vehicle"); + Vehicle vehicle2 = (Vehicle) tuple2.getValueByField("vehicle"); + return Double.compare(vehicle1.efficiency, vehicle2.efficiency); + } + + } + + static class Driver implements Serializable { + final String name; + final int id; + + Driver(String name, int id) { + this.name = name; + this.id = id; + } + + @Override + public String toString() { + return "Driver{" + + "name='" + name + '\'' + + ", id=" + id + + '}'; + } + } + + static class Vehicle implements Serializable { + final String name; + final int maxSpeed; + final double efficiency; + + public Vehicle(String name, int maxSpeed, double efficiency) { + this.name = name; + this.maxSpeed = maxSpeed; + this.efficiency = efficiency; + } + + @Override + public String toString() { + return "Vehicle{" + + "name='" + name + '\'' + + ", maxSpeed=" + maxSpeed + + ", efficiency=" + efficiency + + '}'; + } + + public static List[] generateVehicles(int count) { + List[] vehicles = new List[count]; + for(int i=0; i min = new Min(inputFieldName); + return comparableAggregateStream(inputFieldName, min); + } + + /** + * This aggregator operation computes the minimum of tuples by the given {@code inputFieldName} in a stream by + * using the given {@code comparator}. + * + * @param inputFieldName input field name + * @param comparator comparator used in for finding minimum of two tuple values of {@code inputFieldName}. + * @param type of tuple's given input field value. + * @return + */ + public Stream minBy(String inputFieldName, Comparator comparator) { + Aggregator min = new MinWithComparator<>(inputFieldName, comparator); + return comparableAggregateStream(inputFieldName, min); + } + + /** + * This aggregator operation computes the minimum of tuples in a stream by using the given {@code comparator} with + * {@code TridentTuple}s. + * + * @param comparator comparator used in for finding minimum of two tuple values. + * @return + */ + public Stream min(Comparator comparator) { + Aggregator min = new MinWithComparator<>(comparator); + return comparableAggregateStream(null, min); + } + + /** + * This aggregator operation computes the maximum of tuples by the given {@code inputFieldName} and it is + * assumed that its value is an instance of {@code Comparable}. + * + * @param inputFieldName input field name + * @return + */ + public Stream maxBy(String inputFieldName) { + Aggregator max = new Max(inputFieldName); + return comparableAggregateStream(inputFieldName, max); + } + + /** + * This aggregator operation computes the maximum of tuples by the given {@code inputFieldName} in a stream by + * using the given {@code comparator}. + * + * @param inputFieldName input field name + * @param comparator comparator used in for finding maximum of two tuple values of {@code inputFieldName}. + * @param type of tuple's given input field value. + * @return + */ + public Stream maxBy(String inputFieldName, Comparator comparator) { + Aggregator max = new MaxWithComparator<>(inputFieldName, comparator); + return comparableAggregateStream(inputFieldName, max); + } + + /** + * This aggregator operation computes the maximum of tuples in a stream by using the given {@code comparator} with + * {@code TridentTuple}s. + * + * @param comparator comparator used in for finding maximum of two tuple values. + * @return + */ + public Stream max(Comparator comparator) { + Aggregator max = new MaxWithComparator<>(comparator); + return comparableAggregateStream(null, max); + } + + private Stream comparableAggregateStream(String inputFieldName, Aggregator aggregator) { + if(inputFieldName != null) { + projectionValidation(new Fields(inputFieldName)); + } + return partitionAggregate(getOutputFields(), aggregator, getOutputFields()); + } + public Stream aggregate(Aggregator agg, Fields functionFields) { return aggregate(null, agg, functionFields); } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/ComparisonAggregator.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/ComparisonAggregator.java new file mode 100644 index 00000000000..0109bb59acf --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/ComparisonAggregator.java @@ -0,0 +1,72 @@ +/** + * 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 org.apache.storm.trident.operation.builtin; + +import org.apache.storm.trident.operation.BaseAggregator; +import org.apache.storm.trident.operation.TridentCollector; +import org.apache.storm.trident.tuple.TridentTuple; + +/** + * Abstract {@code Aggregator} for comparing two values in a stream. + * + */ +public abstract class ComparisonAggregator extends BaseAggregator { + + public static class State { + TridentTuple previousTuple; + } + + private final String inputFieldName; + + public ComparisonAggregator(String inputFieldName) { + this.inputFieldName = inputFieldName; + } + + protected abstract T compare(T value1, T value2); + + @Override + public State init(Object batchId, TridentCollector collector) { + return new State(); + } + + @Override + public void aggregate(State state, TridentTuple tuple, TridentCollector collector) { + T value1 = valueFromTuple(state.previousTuple); + T value2 = valueFromTuple(tuple); + + if(value2 == null) { + return; + } + + if(value1 == null || compare(value1, value2) == value2) { + state.previousTuple = tuple; + } + + } + + protected T valueFromTuple(TridentTuple tuple) { + // when there is no input field then the whole tuple is considered for comparison. + return (T) (inputFieldName != null && tuple != null ? tuple.getValueByField(inputFieldName) : tuple); + } + + @Override + public void complete(State state, TridentCollector collector) { + collector.emit(state.previousTuple.getValues()); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Max.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Max.java new file mode 100644 index 00000000000..5385dfb6aaa --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Max.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 org.apache.storm.trident.operation.builtin; + +/** + * This aggregator computes the maximum of aggregated tuples in a stream. It assumes that the tuple has one value and + * it is an instance of {@code Comparable}. + * + */ +public class Max extends ComparisonAggregator> { + + public Max(String inputFieldName) { + super(inputFieldName); + } + + @Override + protected Comparable compare(Comparable value1, Comparable value2) { + return value1.compareTo(value2) > 0 ? value1 : value2; + } + + /** + * Returns an aggregator computes the maximum of aggregated tuples in a stream. It assumes that the tuple has one value and + * it is an instance of {@code Comparable}. + * + * @return + */ +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java new file mode 100644 index 00000000000..172aa58cf59 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.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 org.apache.storm.trident.operation.builtin; + +import java.util.Comparator; + +/** + * This aggregator computes the maximum of aggregated tuples in a stream. It uses given {@code comparator} for comparing + * two values in a stream. + * + */ +public class MaxWithComparator extends ComparisonAggregator { + private final Comparator comparator; + + public MaxWithComparator(Comparator comparator) { + this(null, comparator); + } + + public MaxWithComparator(String inputFieldName, Comparator comparator) { + super(inputFieldName); + this.comparator = comparator; + } + + @Override + protected T compare(T value1, T value2) { + return comparator.compare(value1, value2) > 0 ? value1 : value2; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Min.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Min.java new file mode 100644 index 00000000000..0757d7ce984 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Min.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 org.apache.storm.trident.operation.builtin; + +/** + * This aggregator computes the minimum of aggregated tuples in a stream. It assumes that the tuple has one value and + * it is an instance of {@code Comparable}. + * + */ +public class Min extends ComparisonAggregator> { + + public Min(String inputFieldName) { + super(inputFieldName); + } + + @Override + protected Comparable compare(Comparable value1, Comparable value2) { + return value1.compareTo(value2) < 0 ? value1 : value2; + } + + /** + * Returns an aggregator computes the maximum of aggregated tuples in a stream. It assumes that the tuple has one value and + * it is an instance of {@code Comparable}. + * + * @return + * @param inputFieldName + */ +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java new file mode 100644 index 00000000000..d33e0001d52 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.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 org.apache.storm.trident.operation.builtin; + +import java.util.Comparator; + +/** + * This aggregator computes the minimum of aggregated tuples in a stream. It uses given @{code comparator} for comparing + * two values in a stream. + * + */ +public class MinWithComparator extends ComparisonAggregator { + private final Comparator comparator; + + public MinWithComparator(String inputFieldName, Comparator comparator) { + super(inputFieldName); + this.comparator = comparator; + } + + public MinWithComparator(Comparator comparator) { + this(null, comparator); + } + + @Override + protected T compare(T value1, T value2) { + return comparator.compare(value1, value2) < 0 ? value1 : value2; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/testing/NumberGeneratorSpout.java b/storm-core/src/jvm/org/apache/storm/trident/testing/NumberGeneratorSpout.java new file mode 100644 index 00000000000..a4a9a7998b7 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/testing/NumberGeneratorSpout.java @@ -0,0 +1,92 @@ +/** + * 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 org.apache.storm.trident.testing; + +import org.apache.storm.Config; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.trident.operation.TridentCollector; +import org.apache.storm.trident.spout.IBatchSpout; +import org.apache.storm.tuple.Fields; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + +/** + * + */ +public class NumberGeneratorSpout implements IBatchSpout { + private final Fields fields; + private final int batchSize; + private final int maxNumber; + private final Map>> batches = new HashMap<>(); + + public NumberGeneratorSpout(Fields fields, int batchSize, int maxNumber) { + this.fields = fields; + this.batchSize = batchSize; + this.maxNumber = maxNumber; + } + + @Override + public void open(Map conf, TopologyContext context) { + } + + @Override + public void emitBatch(long batchId, TridentCollector collector) { + List> values = null; + if(batches.containsKey(batchId)) { + values = batches.get(batchId); + } else { + values = new ArrayList<>(); + for (int i = 0; i < batchSize; i++) { + values.add(Collections.singletonList((Object) ThreadLocalRandom.current().nextInt(0, maxNumber + 1))); + } + batches.put(batchId, values); + } + for (List value : values) { + collector.emit(value); + } + } + + @Override + public void ack(long batchId) { + batches.remove(batchId); + } + + @Override + public void close() { + + } + + @Override + public Map getComponentConfiguration() { + Config conf = new Config(); + conf.setMaxTaskParallelism(1); + return conf; + } + + @Override + public Fields getOutputFields() { + return fields; + } +} From d2089c4047982b22cf4583bb63018a285be4d42c Mon Sep 17 00:00:00 2001 From: Satish Duggana Date: Mon, 8 Feb 2016 16:49:46 +0530 Subject: [PATCH 061/387] Addressed review comments --- .../spout/RandomNumberGeneratorSpout.java | 15 +- ...va => TridentMinMaxOfDevicesTopology.java} | 135 +++++++------ .../TridentMinMaxOfVehiclesTopology.java | 180 ++++++++++++++++++ .../jvm/org/apache/storm/trident/Stream.java | 24 ++- .../builtin/ComparisonAggregator.java | 23 ++- .../storm/trident/operation/builtin/Max.java | 6 - .../operation/builtin/MaxWithComparator.java | 7 + .../storm/trident/operation/builtin/Min.java | 8 - .../operation/builtin/MinWithComparator.java | 7 + 9 files changed, 302 insertions(+), 103 deletions(-) rename storm-core/src/jvm/org/apache/storm/trident/testing/NumberGeneratorSpout.java => examples/storm-starter/src/jvm/org/apache/storm/starter/spout/RandomNumberGeneratorSpout.java (81%) rename examples/storm-starter/src/jvm/org/apache/storm/starter/trident/{TridentMinMaxOperationsTopology.java => TridentMinMaxOfDevicesTopology.java} (52%) create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfVehiclesTopology.java diff --git a/storm-core/src/jvm/org/apache/storm/trident/testing/NumberGeneratorSpout.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/spout/RandomNumberGeneratorSpout.java similarity index 81% rename from storm-core/src/jvm/org/apache/storm/trident/testing/NumberGeneratorSpout.java rename to examples/storm-starter/src/jvm/org/apache/storm/starter/spout/RandomNumberGeneratorSpout.java index a4a9a7998b7..1d1b0829dbb 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/testing/NumberGeneratorSpout.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/spout/RandomNumberGeneratorSpout.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.storm.trident.testing; +package org.apache.storm.starter.spout; import org.apache.storm.Config; import org.apache.storm.task.TopologyContext; @@ -25,23 +25,22 @@ import org.apache.storm.tuple.Fields; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.concurrent.ThreadLocalRandom; /** + * This spout generates random whole numbers with given {@code maxNumber} value as maximum with the given {@code fields}. * */ -public class NumberGeneratorSpout implements IBatchSpout { +public class RandomNumberGeneratorSpout implements IBatchSpout { private final Fields fields; private final int batchSize; private final int maxNumber; private final Map>> batches = new HashMap<>(); - public NumberGeneratorSpout(Fields fields, int batchSize, int maxNumber) { + public RandomNumberGeneratorSpout(Fields fields, int batchSize, int maxNumber) { this.fields = fields; this.batchSize = batchSize; this.maxNumber = maxNumber; @@ -59,7 +58,11 @@ public void emitBatch(long batchId, TridentCollector collector) { } else { values = new ArrayList<>(); for (int i = 0; i < batchSize; i++) { - values.add(Collections.singletonList((Object) ThreadLocalRandom.current().nextInt(0, maxNumber + 1))); + List numbers = new ArrayList<>(); + for (int x=0; x, Serializable { @Override public int compare(TridentTuple tuple1, TridentTuple tuple2) { - Vehicle vehicle1 = (Vehicle) tuple1.getValueByField("vehicle"); - Vehicle vehicle2 = (Vehicle) tuple2.getValueByField("vehicle"); + Vehicle vehicle1 = (Vehicle) tuple1.getValueByField(Vehicle.FIELD_NAME); + Vehicle vehicle2 = (Vehicle) tuple2.getValueByField(Vehicle.FIELD_NAME); return Integer.compare(vehicle1.maxSpeed, vehicle2.maxSpeed); } } @@ -147,14 +138,15 @@ static class EfficiencyComparator implements Comparator, Serializa @Override public int compare(TridentTuple tuple1, TridentTuple tuple2) { - Vehicle vehicle1 = (Vehicle) tuple1.getValueByField("vehicle"); - Vehicle vehicle2 = (Vehicle) tuple2.getValueByField("vehicle"); + Vehicle vehicle1 = (Vehicle) tuple1.getValueByField(Vehicle.FIELD_NAME); + Vehicle vehicle2 = (Vehicle) tuple2.getValueByField(Vehicle.FIELD_NAME); return Double.compare(vehicle1.efficiency, vehicle2.efficiency); } } static class Driver implements Serializable { + static final String FIELD_NAME = "driver"; final String name; final int id; @@ -173,6 +165,7 @@ public String toString() { } static class Vehicle implements Serializable { + static final String FIELD_NAME = "vehicle"; final String name; final int maxSpeed; final double efficiency; @@ -194,12 +187,12 @@ public String toString() { public static List[] generateVehicles(int count) { List[] vehicles = new List[count]; - for(int i=0; i + * 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 org.apache.storm.starter.trident; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.trident.Stream; +import org.apache.storm.trident.TridentTopology; +import org.apache.storm.trident.operation.builtin.Debug; +import org.apache.storm.trident.testing.FixedBatchSpout; +import org.apache.storm.trident.tuple.TridentTuple; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; + +/** + * This class demonstrates different usages of + * * {@link Stream#minBy(String, Comparator)} + * * {@link Stream#min(Comparator)} + * * {@link Stream#maxBy(String, Comparator)} + * * {@link Stream#max(Comparator)} + * operations on trident {@link Stream}. + */ +public class TridentMinMaxOfVehiclesTopology { + + /** + * Creates a topology which demonstrates min/max operations on tuples of stream which contain vehicle and driver fields + * with values {@link TridentMinMaxOfVehiclesTopology.Vehicle} and {@link TridentMinMaxOfVehiclesTopology.Driver} respectively. + */ + public static StormTopology buildVehiclesTopology() { + Fields driverField = new Fields(Driver.FIELD_NAME); + Fields vehicleField = new Fields(Vehicle.FIELD_NAME); + Fields allFields = new Fields(Vehicle.FIELD_NAME, Driver.FIELD_NAME); + + FixedBatchSpout spout = new FixedBatchSpout(allFields, 10, Vehicle.generateVehicles(20)); + spout.setCycle(true); + + TridentTopology topology = new TridentTopology(); + Stream vehiclesStream = topology.newStream("spout1", spout). + each(allFields, new Debug("##### vehicles")); + + Stream slowVehiclesStream = + vehiclesStream + .min(new SpeedComparator()) + .each(vehicleField, new Debug("#### slowest vehicle")); + + Stream slowDriversStream = + slowVehiclesStream + .project(driverField) + .each(driverField, new Debug("##### slowest driver")); + + vehiclesStream + .max(new SpeedComparator()) + .each(vehicleField, new Debug("#### fastest vehicle")) + .project(driverField) + .each(driverField, new Debug("##### fastest driver")); + + vehiclesStream + .minBy(Vehicle.FIELD_NAME, new EfficiencyComparator()). + each(vehicleField, new Debug("#### least efficient vehicle")); + + vehiclesStream + .maxBy(Vehicle.FIELD_NAME, new EfficiencyComparator()). + each(vehicleField, new Debug("#### most efficient vehicle")); + + return topology.build(); + } + + public static void main(String[] args) throws Exception { + + StormTopology topology = buildVehiclesTopology(); + Config conf = new Config(); + conf.setMaxSpoutPending(20); + if (args.length == 0) { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("vehicles-topology", conf, topology); + Utils.sleep(60 * 1000); + cluster.shutdown(); + System.exit(0); + } else { + conf.setNumWorkers(3); + StormSubmitter.submitTopologyWithProgressBar("vehicles-topology", conf, topology); + } + } + + static class SpeedComparator implements Comparator, Serializable { + + @Override + public int compare(TridentTuple tuple1, TridentTuple tuple2) { + Vehicle vehicle1 = (Vehicle) tuple1.getValueByField(Vehicle.FIELD_NAME); + Vehicle vehicle2 = (Vehicle) tuple2.getValueByField(Vehicle.FIELD_NAME); + return Integer.compare(vehicle1.maxSpeed, vehicle2.maxSpeed); + } + } + + static class EfficiencyComparator implements Comparator, Serializable { + + @Override + public int compare(Vehicle vehicle1, Vehicle vehicle2) { + return Double.compare(vehicle1.efficiency, vehicle2.efficiency); + } + + } + + static class Driver implements Serializable { + static final String FIELD_NAME = "driver"; + final String name; + final int id; + + Driver(String name, int id) { + this.name = name; + this.id = id; + } + + @Override + public String toString() { + return "Driver{" + + "name='" + name + '\'' + + ", id=" + id + + '}'; + } + } + + static class Vehicle implements Serializable { + static final String FIELD_NAME = "vehicle"; + final String name; + final int maxSpeed; + final double efficiency; + + public Vehicle(String name, int maxSpeed, double efficiency) { + this.name = name; + this.maxSpeed = maxSpeed; + this.efficiency = efficiency; + } + + @Override + public String toString() { + return "Vehicle{" + + "name='" + name + '\'' + + ", maxSpeed=" + maxSpeed + + ", efficiency=" + efficiency + + '}'; + } + + public static List[] generateVehicles(int count) { + List[] vehicles = new List[count]; + for (int i = 0; i < count; i++) { + int id = i - 1; + vehicles[i] = + (new Values( + new Vehicle("Vehicle-" + id, ThreadLocalRandom.current().nextInt(0, 100), ThreadLocalRandom.current().nextDouble(1, 5)), + new Driver("Driver-" + id, id) + )); + } + return vehicles; + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/Stream.java b/storm-core/src/jvm/org/apache/storm/trident/Stream.java index fa62b72826b..d313678476a 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/Stream.java +++ b/storm-core/src/jvm/org/apache/storm/trident/Stream.java @@ -448,10 +448,11 @@ public Stream partitionAggregate(Fields inputFields, ReducerAggregator agg, Fiel /** * This aggregator operation computes the minimum of tuples by the given {@code inputFieldName} and it is - * assumed that its value is an instance of {@code Comparable}. + * assumed that its value is an instance of {@code Comparable}. If the value of tuple with field {@code inputFieldName} is not an + * instance of {@code Comparable} then it throws {@code ClassCastException} * * @param inputFieldName input field name - * @return + * @return the new stream with this operation. */ public Stream minBy(String inputFieldName) { Aggregator min = new Min(inputFieldName); @@ -460,12 +461,13 @@ public Stream minBy(String inputFieldName) { /** * This aggregator operation computes the minimum of tuples by the given {@code inputFieldName} in a stream by - * using the given {@code comparator}. + * using the given {@code comparator}. If the value of tuple with field {@code inputFieldName} is not an + * instance of {@code T} then it throws {@code ClassCastException} * * @param inputFieldName input field name * @param comparator comparator used in for finding minimum of two tuple values of {@code inputFieldName}. * @param type of tuple's given input field value. - * @return + * @return the new stream with this operation. */ public Stream minBy(String inputFieldName, Comparator comparator) { Aggregator min = new MinWithComparator<>(inputFieldName, comparator); @@ -477,7 +479,7 @@ public Stream minBy(String inputFieldName, Comparator comparator) { * {@code TridentTuple}s. * * @param comparator comparator used in for finding minimum of two tuple values. - * @return + * @return the new stream with this operation. */ public Stream min(Comparator comparator) { Aggregator min = new MinWithComparator<>(comparator); @@ -486,10 +488,11 @@ public Stream min(Comparator comparator) { /** * This aggregator operation computes the maximum of tuples by the given {@code inputFieldName} and it is - * assumed that its value is an instance of {@code Comparable}. + * assumed that its value is an instance of {@code Comparable}. If the value of tuple with field {@code inputFieldName} is not an + * instance of {@code Comparable} then it throws {@code ClassCastException} * * @param inputFieldName input field name - * @return + * @return the new stream with this operation. */ public Stream maxBy(String inputFieldName) { Aggregator max = new Max(inputFieldName); @@ -498,12 +501,13 @@ public Stream maxBy(String inputFieldName) { /** * This aggregator operation computes the maximum of tuples by the given {@code inputFieldName} in a stream by - * using the given {@code comparator}. + * using the given {@code comparator}. If the value of tuple with field {@code inputFieldName} is not an + * instance of {@code T} then it throws {@code ClassCastException} * * @param inputFieldName input field name * @param comparator comparator used in for finding maximum of two tuple values of {@code inputFieldName}. * @param type of tuple's given input field value. - * @return + * @return the new stream with this operation. */ public Stream maxBy(String inputFieldName, Comparator comparator) { Aggregator max = new MaxWithComparator<>(inputFieldName, comparator); @@ -515,7 +519,7 @@ public Stream maxBy(String inputFieldName, Comparator comparator) { * {@code TridentTuple}s. * * @param comparator comparator used in for finding maximum of two tuple values. - * @return + * @return the new stream with this operation. */ public Stream max(Comparator comparator) { Aggregator max = new MaxWithComparator<>(comparator); diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/ComparisonAggregator.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/ComparisonAggregator.java index 0109bb59acf..82b657ac9d9 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/ComparisonAggregator.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/ComparisonAggregator.java @@ -21,12 +21,16 @@ import org.apache.storm.trident.operation.BaseAggregator; import org.apache.storm.trident.operation.TridentCollector; import org.apache.storm.trident.tuple.TridentTuple; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Abstract {@code Aggregator} for comparing two values in a stream. * */ public abstract class ComparisonAggregator extends BaseAggregator { + private static final Logger log = LoggerFactory.getLogger(ComparisonAggregator.class); + private Object batchId; public static class State { TridentTuple previousTuple; @@ -42,6 +46,8 @@ public ComparisonAggregator(String inputFieldName) { @Override public State init(Object batchId, TridentCollector collector) { + this.batchId = batchId; + log.debug("Started comparison aggregation for batch: [{}] in operation [{}]", batchId, this); return new State(); } @@ -50,6 +56,8 @@ public void aggregate(State state, TridentTuple tuple, TridentCollector collecto T value1 = valueFromTuple(state.previousTuple); T value2 = valueFromTuple(tuple); + log.debug("Aggregated tuple value in state [{}], and received tuple value [{}] in operation [{}]", value1, value2, this); + if(value2 == null) { return; } @@ -62,11 +70,22 @@ public void aggregate(State state, TridentTuple tuple, TridentCollector collecto protected T valueFromTuple(TridentTuple tuple) { // when there is no input field then the whole tuple is considered for comparison. - return (T) (inputFieldName != null && tuple != null ? tuple.getValueByField(inputFieldName) : tuple); + Object value = null; + if (inputFieldName != null && tuple != null) { + value = tuple.getValueByField(inputFieldName); + } else { + value = tuple; + } + + log.debug("value from tuple is [{}] with input field [{}] and tuple [{}]", value, inputFieldName, tuple); + + return (T) value; } @Override public void complete(State state, TridentCollector collector) { - collector.emit(state.previousTuple.getValues()); + log.debug("Completed comparison aggregation for batch [{}] with resultant tuple: [{}] in operation [{}]", batchId, state.previousTuple, this); + + collector.emit(state.previousTuple != null ? state.previousTuple.getValues() : null); } } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Max.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Max.java index 5385dfb6aaa..f1221b03a34 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Max.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Max.java @@ -34,10 +34,4 @@ protected Comparable compare(Comparable value1, Comparable 0 ? value1 : value2; } - /** - * Returns an aggregator computes the maximum of aggregated tuples in a stream. It assumes that the tuple has one value and - * it is an instance of {@code Comparable}. - * - * @return - */ } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java index 172aa58cf59..0e8ae900e69 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java @@ -41,4 +41,11 @@ public MaxWithComparator(String inputFieldName, Comparator comparator) { protected T compare(T value1, T value2) { return comparator.compare(value1, value2) > 0 ? value1 : value2; } + + @Override + public String toString() { + return "MaxWithComparator{" + + "comparator=" + comparator + + '}'; + } } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Min.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Min.java index 0757d7ce984..010a9195da1 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Min.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/Min.java @@ -33,12 +33,4 @@ public Min(String inputFieldName) { protected Comparable compare(Comparable value1, Comparable value2) { return value1.compareTo(value2) < 0 ? value1 : value2; } - - /** - * Returns an aggregator computes the maximum of aggregated tuples in a stream. It assumes that the tuple has one value and - * it is an instance of {@code Comparable}. - * - * @return - * @param inputFieldName - */ } diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java index d33e0001d52..64144cb9ab8 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java @@ -41,4 +41,11 @@ public MinWithComparator(Comparator comparator) { protected T compare(T value1, T value2) { return comparator.compare(value1, value2) < 0 ? value1 : value2; } + + @Override + public String toString() { + return "MinWithComparator{" + + "comparator=" + comparator + + '}'; + } } From 7715ad10c317f8d1695dd279433f4d44ae53154d Mon Sep 17 00:00:00 2001 From: Longda Feng Date: Fri, 19 Feb 2016 14:10:34 +0800 Subject: [PATCH 062/387] Add Longda to committer list --- README.markdown | 1 + pom.xml | 9 +++++++++ 2 files changed, 10 insertions(+) diff --git a/README.markdown b/README.markdown index d0b94924ed9..3f1eb1750d9 100644 --- a/README.markdown +++ b/README.markdown @@ -91,6 +91,7 @@ under the License. * Boyang Jerry Peng ([@jerrypeng](https://github.com/jerrypeng)) * Zhuo Liu ([@zhuoliu](https://github.com/zhuoliu)) * Haohui Mai ([@haohui](https://github.com/haohui)) +* Longda Feng ([@longda](https://github.com/longdafeng)) ## Contributors diff --git a/pom.xml b/pom.xml index e85e4c155c7..0f690ff09ff 100644 --- a/pom.xml +++ b/pom.xml @@ -165,6 +165,15 @@ -6 + + longda + Longda Feng + longda@apache.org + + Committer + + +8 + From 29bda16bc37804c74924fff25b2ba2d75e584b68 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 12 Feb 2016 09:43:45 +0900 Subject: [PATCH 063/387] STORM-1541 Change scope of 'hadoop-minicluster' to test --- external/storm-hdfs/pom.xml | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 5e8cce4fb40..a3fe871f79f 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -113,17 +113,6 @@ - - org.apache.hadoop - hadoop-minicluster - ${hadoop.version} - - - org.slf4j - slf4j-log4j12 - - - org.apache.hadoop hadoop-auth @@ -194,6 +183,18 @@ 4.11 test + + org.apache.hadoop + hadoop-minicluster + ${hadoop.version} + + + org.slf4j + slf4j-log4j12 + + + test + io.confluent kafka-avro-serializer From 69cf90de467693d4949fcc272ac9ebd893fb18a2 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 19 Feb 2016 15:02:25 -0500 Subject: [PATCH 064/387] add STORM-1541 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3b91c20464d..7457843ecb0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1541: Change scope of 'hadoop-minicluster' to test * STORM-1539: Improve Storm ACK-ing performance * STORM-1519: Storm syslog logging not confirming to RFC5426 3.1 * STORM-1533: IntegerValidator for metric consumer parallelism hint From cb4636c59c84db59912b50cbd95178f8c3532534 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 19 Feb 2016 15:10:45 -0500 Subject: [PATCH 065/387] add STORM-1532 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7457843ecb0..475a1a4c703 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1532: Fix readCommandLineOpts to parse JSON correctly * STORM-1541: Change scope of 'hadoop-minicluster' to test * STORM-1539: Improve Storm ACK-ing performance * STORM-1519: Storm syslog logging not confirming to RFC5426 3.1 From f6f05afb16336ef3a8dd16b33577e6fc23e50bf6 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 19 Feb 2016 15:19:50 -0500 Subject: [PATCH 066/387] add STORM-1522 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 475a1a4c703..718c98972d9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1522: REST API throws invalid worker log links * STORM-1532: Fix readCommandLineOpts to parse JSON correctly * STORM-1541: Change scope of 'hadoop-minicluster' to test * STORM-1539: Improve Storm ACK-ing performance From 2c6fc60054b1cb7115bf53d3d4f592888c263961 Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Fri, 19 Feb 2016 16:50:58 -0600 Subject: [PATCH 067/387] fix broken component error times --- storm-core/src/clj/org/apache/storm/ui/core.clj | 13 ++++--------- storm-core/src/ui/public/component.html | 2 +- storm-core/src/ui/public/topology.html | 2 +- 3 files changed, 6 insertions(+), 11 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj index 9bd7a26b7dc..9e339841ac5 100644 --- a/storm-core/src/clj/org/apache/storm/ui/core.clj +++ b/storm-core/src/clj/org/apache/storm/ui/core.clj @@ -157,11 +157,6 @@ (defn supervisor-log-link [host] (url-format "http://%s:%s/daemonlog?file=supervisor.log" host (*STORM-CONF* LOGVIEWER-PORT))) -(defn get-error-time - [error] - (if error - (time-delta (.get_error_time_secs ^ErrorInfo error)))) - (defn get-error-data [error] (if error @@ -184,7 +179,7 @@ [error] (if error (.get_error_time_secs ^ErrorInfo error) - "")) + 0)) (defn worker-dump-link [host port topology-id] (url-format "http://%s:%s/dumps/%s/%s" @@ -518,7 +513,7 @@ "errorTime" (get-error-time error-info) "errorHost" host "errorPort" port - "errorLapsedSecs" (get-error-time error-info) + "errorLapsedSecs" (time-delta (get-error-time error-info)) "errorWorkerLogLink" (worker-log-link host port topo-id secure?)})) (defn- common-agg-stats-json @@ -644,14 +639,14 @@ reverse)] {"componentErrors" (for [^ErrorInfo e errors] - {"errorTime" (* 1000 (long (.get_error_time_secs e))) + {"errorTime" (get-error-time e) "errorHost" (.get_host e) "errorPort" (.get_port e) "errorWorkerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id secure?) - "errorLapsedSecs" (get-error-time e) + "errorLapsedSecs" (time-delta (get-error-time e)) "error" (.get_error e)})})) (defmulti unpack-comp-agg-stat diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html index 88187b673ba..6d5465f17de 100644 --- a/storm-core/src/ui/public/component.html +++ b/storm-core/src/ui/public/component.html @@ -301,7 +301,7 @@

Storm UI

var errorTimeCells = document.getElementsByClassName("errorTimeSpan"); for (i = 0; i < errorTimeCells.length; i++) { - var timeInMilliseconds = errorTimeCells[i].id; + var timeInMilliseconds = errorTimeCells[i].id * 1000; var time = parseInt(timeInMilliseconds); var date = new Date(time); errorTimeCells[i].innerHTML = date.toJSON(); diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html index 5869d9a9a9b..feb81f8f271 100644 --- a/storm-core/src/ui/public/topology.html +++ b/storm-core/src/ui/public/topology.html @@ -350,7 +350,7 @@

Topology resources

{ if((errorTime[i].id)) { - var a = new Date(parseInt(errorTime[i].id)); + var a = new Date(parseInt(errorTime[i].id) * 1000); var months = ['Jan','Feb','Mar','Apr','May','Jun','Jul','Aug','Sep','Oct','Nov','Dec']; var days = ['Sun', 'Mon', 'Tue', 'Wed', 'Thur', 'Fri', 'Sat']; var year = a.getFullYear(); From 4a1b879f616a789b0230a7388ffaf026d892fd26 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sat, 20 Feb 2016 18:52:51 -0800 Subject: [PATCH 068/387] Added STORM-1511 to CHANGELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 718c98972d9..d85c002d440 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1511: min/max operations support on a trident stream * STORM-1522: REST API throws invalid worker log links * STORM-1532: Fix readCommandLineOpts to parse JSON correctly * STORM-1541: Change scope of 'hadoop-minicluster' to test From 8d6096154ac30bbd3bc390604b6b9376a740c83e Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Mon, 15 Feb 2016 20:41:03 +0530 Subject: [PATCH 069/387] [STORM-1552] Fix topology event sampling log dir Currently the events are logged under "storm-local/workers-artifacts/{storm-id}/{port}/events.log". and the "events" link in UI does not display the log file. The events.log should be kept under "logs/workers-artifacts/{storm-id}/{port}/events.log" so that its viewable via logviewer. --- .../storm/metric/FileBasedEventLogger.java | 31 ++++++++++--------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/metric/FileBasedEventLogger.java b/storm-core/src/jvm/org/apache/storm/metric/FileBasedEventLogger.java index a56a596b647..076a68c3fb2 100644 --- a/storm-core/src/jvm/org/apache/storm/metric/FileBasedEventLogger.java +++ b/storm-core/src/jvm/org/apache/storm/metric/FileBasedEventLogger.java @@ -17,6 +17,7 @@ */ package org.apache.storm.metric; +import org.apache.storm.Config; import org.apache.storm.task.TopologyContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +46,7 @@ public class FileBasedEventLogger implements IEventLogger { private void initLogWriter(Path logFilePath) { try { - LOG.info("logFilePath {}", logFilePath); + LOG.info("Event log path {}", logFilePath); eventLogPath = logFilePath; eventLogWriter = Files.newBufferedWriter(eventLogPath, StandardCharsets.UTF_8, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.APPEND); @@ -76,33 +77,35 @@ public void run() { scheduler.scheduleAtFixedRate(task, FLUSH_INTERVAL_MILLIS, FLUSH_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); } + private String getLogDir(Map stormConf) { + String logDir; + if ((logDir = System.getProperty("storm.log.dir")) == null + && (logDir = (String) stormConf.get("storm.log.dir")) == null) { + logDir = Paths.get(System.getProperty("storm.home"), "logs").toString(); + } + return logDir; + } @Override public void prepare(Map stormConf, TopologyContext context) { - String logDir; // storm local directory + String workersArtifactDir; // workers artifact directory String stormId = context.getStormId(); int port = context.getThisWorkerPort(); - if ((logDir = System.getProperty("storm.local.dir")) == null && - (logDir = (String)stormConf.get("storm.local.dir")) == null) { - String msg = "Could not determine the directory to log events."; - LOG.error(msg); - throw new RuntimeException(msg); - } else { - LOG.info("FileBasedEventLogger log directory {}.", logDir); + if ((workersArtifactDir = (String) stormConf.get(Config.STORM_WORKERS_ARTIFACTS_DIR)) == null) { + workersArtifactDir = "workers-artifacts"; } - /* * Include the topology name & worker port in the file name so that * multiple event loggers can log independently. */ - Path path = Paths.get(logDir, "workers-artifacts", stormId, Integer.toString(port), "events.log"); + Path path = Paths.get(workersArtifactDir, stormId, Integer.toString(port), "events.log"); if (!path.isAbsolute()) { - path = Paths.get(System.getProperty("storm.home"), logDir, "workers-artifacts", - stormId, Integer.toString(port), "events.log"); + path = Paths.get(getLogDir(stormConf), workersArtifactDir, + stormId, Integer.toString(port), "events.log"); } File dir = path.toFile().getParentFile(); if (!dir.exists()) { - dir.mkdirs(); + dir.mkdirs(); } initLogWriter(path); setUpFlushTask(); From f3baed0d4bb9c415657dfa97f33519994e991e17 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Mon, 22 Feb 2016 00:03:29 +0530 Subject: [PATCH 070/387] Fixing event log dir to fall back on storm.local.dir if storm.home is null --- .../storm/metric/FileBasedEventLogger.java | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/metric/FileBasedEventLogger.java b/storm-core/src/jvm/org/apache/storm/metric/FileBasedEventLogger.java index 076a68c3fb2..6a07c0ece21 100644 --- a/storm-core/src/jvm/org/apache/storm/metric/FileBasedEventLogger.java +++ b/storm-core/src/jvm/org/apache/storm/metric/FileBasedEventLogger.java @@ -17,6 +17,7 @@ */ package org.apache.storm.metric; +import org.apache.commons.lang.StringUtils; import org.apache.storm.Config; import org.apache.storm.task.TopologyContext; import org.slf4j.Logger; @@ -77,11 +78,24 @@ public void run() { scheduler.scheduleAtFixedRate(task, FLUSH_INTERVAL_MILLIS, FLUSH_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); } + private String getFirstNonNull(String... strings) { + for (String str : strings) { + if (str != null) { + return str; + } + } + return null; + } + private String getLogDir(Map stormConf) { - String logDir; - if ((logDir = System.getProperty("storm.log.dir")) == null - && (logDir = (String) stormConf.get("storm.log.dir")) == null) { - logDir = Paths.get(System.getProperty("storm.home"), "logs").toString(); + String logDir = getFirstNonNull(System.getProperty("storm.log.dir"), + (String) stormConf.get("storm.log.dir")); + if (logDir == null) { + logDir = Paths.get(getFirstNonNull(System.getProperty("storm.home"), + System.getProperty("storm.local.dir"), + (String) stormConf.get("storm.local.dir"), + StringUtils.EMPTY), + "logs").toString(); } return logDir; } From 63bac16c2a27c3a3d0c372dc0925f4f606ffe9dd Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 23 Feb 2016 12:52:23 -0800 Subject: [PATCH 071/387] Added STORM-1552 to CHANGELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index d85c002d440..319dc858f1e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1552: Fix topology event sampling log dir * STORM-1511: min/max operations support on a trident stream * STORM-1522: REST API throws invalid worker log links * STORM-1532: Fix readCommandLineOpts to parse JSON correctly From 31985cc09537534ed8179446b6493c947e741a48 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 23 Feb 2016 10:28:05 -0800 Subject: [PATCH 072/387] STORM-1569: Adding option in nimbus to specify request queue size in config. --- conf/defaults.yaml | 2 +- storm-core/src/jvm/org/apache/storm/Config.java | 9 +++++++++ .../apache/storm/security/auth/ThriftConnectionType.java | 2 +- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index d381f0d72b6..7128cfe9430 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -77,7 +77,7 @@ topology.min.replication.count: 1 topology.max.replication.wait.time.sec: 60 nimbus.credential.renewers.freq.secs: 600 nimbus.impersonation.authorizer: "org.apache.storm.security.auth.authorizer.ImpersonationAuthorizer" - +nimbus.queue.size: 100000 scheduler.display.resource: false ### ui.* configs are for the master diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index 74231a06f0d..3eedc63ca0e 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -674,6 +674,15 @@ public class Config extends HashMap { @isStringList public static final String NIMBUS_AUTO_CRED_PLUGINS = "nimbus.autocredential.plugins.classes"; + /** + * Nimbus thrift server queue size, default is 100000. This is the request queue size , when there are more requests + * than number of threads to serve the requests, those requests will be queued to this queue. If the request queue + * size > this config, then the incoming requests will be rejected. + */ + @isInteger + @isPositiveNumber + public static final String NIMBUS_QUEUE_SIZE = "nimbus.queue.size"; + /** * FQCN of a class that implements {@code ISubmitterHook} @see ISubmitterHook for details. */ diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java b/storm-core/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java index 6d05a8ae8b8..27db1430742 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java @@ -26,7 +26,7 @@ * The purpose for which the Thrift server is created. */ public enum ThriftConnectionType { - NIMBUS(Config.NIMBUS_THRIFT_TRANSPORT_PLUGIN, Config.NIMBUS_THRIFT_PORT, null, + NIMBUS(Config.NIMBUS_THRIFT_TRANSPORT_PLUGIN, Config.NIMBUS_THRIFT_PORT, Config.NIMBUS_QUEUE_SIZE, 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), From 7c534bd69adf0f7c4562ef6b59d341d55d31876f Mon Sep 17 00:00:00 2001 From: Kyle Nusbaum Date: Wed, 24 Feb 2016 15:40:25 -0600 Subject: [PATCH 073/387] Adding STORM-1569 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 319dc858f1e..6cef6490850 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1569: Allowing users to specify the nimbus thrift server queue size. * STORM-1552: Fix topology event sampling log dir * STORM-1511: min/max operations support on a trident stream * STORM-1522: REST API throws invalid worker log links From 1397246d2b2721783bb43445a84996ec552a05d5 Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Wed, 24 Feb 2016 16:12:19 -0600 Subject: [PATCH 074/387] Topo page last error time blank when no errors --- storm-core/src/clj/org/apache/storm/ui/core.clj | 7 +++---- .../src/ui/public/templates/topology-page-template.html | 6 +++++- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj index 9e339841ac5..c35e0517ae9 100644 --- a/storm-core/src/clj/org/apache/storm/ui/core.clj +++ b/storm-core/src/clj/org/apache/storm/ui/core.clj @@ -178,8 +178,7 @@ (defn get-error-time [error] (if error - (.get_error_time_secs ^ErrorInfo error) - 0)) + (.get_error_time_secs ^ErrorInfo error))) (defn worker-dump-link [host port topology-id] (url-format "http://%s:%s/dumps/%s/%s" @@ -513,7 +512,7 @@ "errorTime" (get-error-time error-info) "errorHost" host "errorPort" port - "errorLapsedSecs" (time-delta (get-error-time error-info)) + "errorLapsedSecs" (if-let [t (get-error-time error-info)] (time-delta t)) "errorWorkerLogLink" (worker-log-link host port topo-id secure?)})) (defn- common-agg-stats-json @@ -646,7 +645,7 @@ (.get_port e) topology-id secure?) - "errorLapsedSecs" (time-delta (get-error-time e)) + "errorLapsedSecs" (if-let [t (get-error-time e)] (time-delta t)) "error" (.get_error e)})})) (defmulti unpack-comp-agg-stat 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 1f81f1b8e3e..02b3c7693b1 100644 --- a/storm-core/src/ui/public/templates/topology-page-template.html +++ b/storm-core/src/ui/public/templates/topology-page-template.html @@ -323,7 +323,9 @@

Spouts ({{windowHint}})

{{lastError}} + {{#errorTime}} {{errorTime}} + {{/errorTime}} {{/spouts}} @@ -417,7 +419,9 @@

Bolts ({{windowHint}})

{{lastError}} + {{#errorTime}} {{errorTime}} + {{/errorTime}} {{/bolts}} @@ -512,4 +516,4 @@

Change Log Level

{{/loggers}} - \ No newline at end of file + From d6cc7a0bcfabb685d3549882795deeb0d336b904 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 25 Feb 2016 13:04:18 -0600 Subject: [PATCH 075/387] Merge branch 'master' of https://github.com/DigitalPebble/storm into PULL-1122 PULL-1122: storm-hdfs change visibility of create and closeOutputFile methods to protected --- .../org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java | 10 +++++----- .../apache/storm/hdfs/bolt/AvroGenericRecordBolt.java | 8 ++++---- .../main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java | 8 ++++---- .../org/apache/storm/hdfs/bolt/SequenceFileBolt.java | 8 ++++---- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java index ae5d5d7e251..c56f486c968 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java @@ -231,7 +231,7 @@ public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) { * @param tuple * @throws IOException */ - abstract void writeTuple(Tuple tuple) throws IOException; + abstract protected void writeTuple(Tuple tuple) throws IOException; /** * Make the best effort to sync written data to the underlying file system. Concrete classes should very clearly @@ -240,12 +240,12 @@ public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) { * * @throws IOException */ - abstract void syncTuples() throws IOException; + abstract protected void syncTuples() throws IOException; - abstract void closeOutputFile() throws IOException; + abstract protected void closeOutputFile() throws IOException; - abstract Path createOutputFile() throws IOException; + abstract protected Path createOutputFile() throws IOException; - abstract void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector collector) throws IOException; + abstract protected void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector collector) throws IOException; } diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java index 1fd2e2ff543..cdeb2f8c578 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java @@ -92,7 +92,7 @@ public AvroGenericRecordBolt withTickTupleIntervalSeconds(int interval) { } @Override - void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector collector) throws IOException { + protected void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector collector) throws IOException { LOG.info("Preparing AvroGenericRecord Bolt..."); this.fs = FileSystem.get(URI.create(this.fsUrl), hdfsConfig); Schema.Parser parser = new Schema.Parser(); @@ -100,14 +100,14 @@ void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector collec } @Override - void writeTuple(Tuple tuple) throws IOException { + protected void writeTuple(Tuple tuple) throws IOException { GenericRecord avroRecord = (GenericRecord) tuple.getValue(0); avroWriter.append(avroRecord); offset = this.out.getPos(); } @Override - void syncTuples() throws IOException { + protected void syncTuples() throws IOException { avroWriter.flush(); LOG.debug("Attempting to sync all data to filesystem"); @@ -127,7 +127,7 @@ protected void closeOutputFile() throws IOException } @Override - Path createOutputFile() throws IOException { + protected Path createOutputFile() throws IOException { Path path = new Path(this.fileNameFormat.getPath(), this.fileNameFormat.getName(this.rotation, System.currentTimeMillis())); this.out = this.fs.create(path); diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java index b351adc1ff3..0299f43e37e 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java @@ -96,7 +96,7 @@ public void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector } @Override - void syncTuples() throws IOException { + protected void syncTuples() throws IOException { LOG.debug("Attempting to sync all data to filesystem"); if (this.out instanceof HdfsDataOutputStream) { ((HdfsDataOutputStream) this.out).hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); @@ -106,19 +106,19 @@ void syncTuples() throws IOException { } @Override - void writeTuple(Tuple tuple) throws IOException { + protected void writeTuple(Tuple tuple) throws IOException { byte[] bytes = this.format.format(tuple); out.write(bytes); this.offset += bytes.length; } @Override - void closeOutputFile() throws IOException { + protected void closeOutputFile() throws IOException { this.out.close(); } @Override - Path createOutputFile() throws IOException { + protected Path createOutputFile() throws IOException { Path path = new Path(this.fileNameFormat.getPath(), this.fileNameFormat.getName(this.rotation, System.currentTimeMillis())); this.out = this.fs.create(path); return path; diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileBolt.java index 2a266c16611..e0db7c9ac21 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileBolt.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileBolt.java @@ -114,18 +114,18 @@ public void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector } @Override - void syncTuples() throws IOException { + protected void syncTuples() throws IOException { LOG.debug("Attempting to sync all data to filesystem"); this.writer.hsync(); } @Override - void writeTuple(Tuple tuple) throws IOException { + protected void writeTuple(Tuple tuple) throws IOException { this.writer.append(this.format.key(tuple), this.format.value(tuple)); this.offset = this.writer.getLength(); } - Path createOutputFile() throws IOException { + protected Path createOutputFile() throws IOException { Path p = new Path(this.fsUrl + this.fileNameFormat.getPath(), this.fileNameFormat.getName(this.rotation, System.currentTimeMillis())); this.writer = SequenceFile.createWriter( this.hdfsConfig, @@ -137,7 +137,7 @@ Path createOutputFile() throws IOException { return p; } - void closeOutputFile() throws IOException { + protected void closeOutputFile() throws IOException { this.writer.close(); } } From d56a22567e9905b88a4af79050658d7cb389674b Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Tue, 16 Feb 2016 15:55:25 +0530 Subject: [PATCH 076/387] [STORM-1540] Fix Debug/Sampling for Trident When ConsList emitted by a trident spout has to be transferred over the network, it fails during Serialization. The proposed fix is to make ConsList kyro serializable. --- .../serialization/SerializationFactory.java | 2 ++ .../apache/storm/trident/tuple/ConsList.java | 20 +++++++++++++++++-- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/serialization/SerializationFactory.java b/storm-core/src/jvm/org/apache/storm/serialization/SerializationFactory.java index 678a74255f1..32d28fed1ed 100644 --- a/storm-core/src/jvm/org/apache/storm/serialization/SerializationFactory.java +++ b/storm-core/src/jvm/org/apache/storm/serialization/SerializationFactory.java @@ -24,6 +24,7 @@ import org.apache.storm.serialization.types.HashMapSerializer; import org.apache.storm.serialization.types.HashSetSerializer; import org.apache.storm.transactional.TransactionAttempt; +import org.apache.storm.trident.tuple.ConsList; import org.apache.storm.tuple.Values; import org.apache.storm.utils.ListDelegate; import org.apache.storm.utils.Utils; @@ -68,6 +69,7 @@ public static Kryo getKryo(Map conf) { k.register(Values.class); k.register(org.apache.storm.metric.api.IMetricsConsumer.DataPoint.class); k.register(org.apache.storm.metric.api.IMetricsConsumer.TaskInfo.class); + k.register(ConsList.class); try { JavaBridge.registerPrimitives(k); JavaBridge.registerCollections(k); diff --git a/storm-core/src/jvm/org/apache/storm/trident/tuple/ConsList.java b/storm-core/src/jvm/org/apache/storm/trident/tuple/ConsList.java index 55c1e79c134..cef2cc473ae 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/tuple/ConsList.java +++ b/storm-core/src/jvm/org/apache/storm/trident/tuple/ConsList.java @@ -18,12 +18,18 @@ package org.apache.storm.trident.tuple; import java.util.AbstractList; +import java.util.ArrayList; import java.util.List; public class ConsList extends AbstractList { List _elems; Object _first; - + + // for kryo + private ConsList() { + _elems = new ArrayList<>(); + } + public ConsList(Object o, List elems) { _elems = elems; _first = o; @@ -39,6 +45,16 @@ public Object get(int i) { @Override public int size() { - return _elems.size() + 1; + return _first == null ? _elems.size() : _elems.size() + 1; + } + + // for kryo + @Override + public void add(int index, Object element) { + if (index == 0) { + _first = element; + } else { + _elems.add(index - 1, element); + } } } From f99fe58508e80a39253091534123bcb22a005695 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 25 Feb 2016 15:33:16 -0800 Subject: [PATCH 077/387] apply STORM-1540 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6cef6490850..473a4d1aefd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1540: Fix Debug/Sampling for Trident * STORM-1569: Allowing users to specify the nimbus thrift server queue size. * STORM-1552: Fix topology event sampling log dir * STORM-1511: min/max operations support on a trident stream From 8ce34faa409895f5e8db3a10fbc89e7371e7210b Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal Date: Mon, 22 Feb 2016 18:09:25 +0530 Subject: [PATCH 078/387] STORM-1542: Remove profile action retry in case of non-zero exit code --- bin/flight.bash | 4 ++-- storm-core/src/clj/org/apache/storm/daemon/supervisor.clj | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/flight.bash b/bin/flight.bash index 957c9ac22c8..36b98ba7f7e 100755 --- a/bin/flight.bash +++ b/bin/flight.bash @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -JDKPATH="/home/y/share/yjava_jdk/java" +JDKPATH=$JAVA_HOME BINPATH="/usr/bin" USER=`whoami` @@ -59,7 +59,7 @@ function dump_record { function jstack_record { FILENAME=jstack-$1-${NOW}.txt - $BINPATH/jstack $1 > "$2/${FILENAME}" + $BINPATH/jstack $1 > "$2/${FILENAME}" 2>&1 } function jmap_record { diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj index d27a6090c68..b504e570970 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj @@ -741,7 +741,7 @@ (and stop? (= action ProfileAction/JPROFILE_STOP)) (jprofile-stop profile-cmd worker-pid target-dir)) action-on-exit (fn [exit-code] (log-message log-prefix " profile-action exited for code: " exit-code) - (if (and (= exit-code 0) stop?) + (if stop? (delete-topology-profiler-action storm-cluster-state storm-id pro-action))) command (->> command (map str) (filter (complement empty?)))] From a24b8967844bd5eef5ee7e9c5f385161418cda87 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 25 Feb 2016 16:59:43 -0800 Subject: [PATCH 079/387] add STORM-1542 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 473a4d1aefd..aa7bcd23e4b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1542: Remove profile action retry in case of non-zero exit code * STORM-1540: Fix Debug/Sampling for Trident * STORM-1569: Allowing users to specify the nimbus thrift server queue size. * STORM-1552: Fix topology event sampling log dir From 7fc342cbc2bd8f757985038650d005c3de00c642 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 25 Feb 2016 18:28:15 -0800 Subject: [PATCH 080/387] add STORM-1488 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index aa7bcd23e4b..63b3c6580f5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1488: UI Topology Page component last error timestamp is from 1970 * STORM-1542: Remove profile action retry in case of non-zero exit code * STORM-1540: Fix Debug/Sampling for Trident * STORM-1569: Allowing users to specify the nimbus thrift server queue size. From 32721e8470805282620eda75f0b560c895cf4669 Mon Sep 17 00:00:00 2001 From: zhuol Date: Fri, 26 Feb 2016 10:07:30 -0600 Subject: [PATCH 081/387] [STORM-1574][1.x-branch] Better handle backpressure exception & clear dir --- .../src/clj/org/apache/storm/cluster.clj | 7 ++- .../clj/org/apache/storm/daemon/nimbus.clj | 6 ++- .../clj/org/apache/storm/daemon/worker.clj | 8 +-- .../utils/WorkerBackpressureCallback.java | 3 +- .../storm/utils/WorkerBackpressureThread.java | 38 ++++++++++---- .../utils/WorkerBackpressureThreadTest.java | 50 +++++++++++++++++++ 6 files changed, 96 insertions(+), 16 deletions(-) create mode 100644 storm-core/test/jvm/org/apache/storm/utils/WorkerBackpressureThreadTest.java diff --git a/storm-core/src/clj/org/apache/storm/cluster.clj b/storm-core/src/clj/org/apache/storm/cluster.clj index 5aef2661148..9b01df98345 100644 --- a/storm-core/src/clj/org/apache/storm/cluster.clj +++ b/storm-core/src/clj/org/apache/storm/cluster.clj @@ -81,6 +81,7 @@ (worker-backpressure! [this storm-id node port info]) (topology-backpressure [this storm-id callback]) (setup-backpressure! [this storm-id]) + (remove-backpressure! [this storm-id]) (remove-worker-backpressure! [this storm-id node port]) (activate-storm! [this storm-id storm-base]) (update-storm! [this storm-id new-elems]) @@ -485,7 +486,7 @@ (topology-backpressure [this storm-id callback] - "if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not." + "if the backpresure/storm-id dir is not empty, this topology has throttle-on, otherwise throttle-off." (when callback (swap! backpressure-callback assoc storm-id callback)) (let [path (backpressure-storm-root storm-id) @@ -496,6 +497,10 @@ [this storm-id] (.mkdirs cluster-state (backpressure-storm-root storm-id) acls)) + (remove-backpressure! + [this storm-id] + (.delete_node cluster-state (backpressure-storm-root storm-id))) + (remove-worker-backpressure! [this storm-id node port] (.delete_node cluster-state (backpressure-path storm-id node port))) diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj index 9376d6e2bfb..ec5a624409b 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj @@ -1523,7 +1523,8 @@ (setup-storm-code nimbus conf storm-id uploadedJarLocation total-storm-conf topology) (wait-for-desired-code-replication nimbus total-storm-conf storm-id) (.setup-heartbeats! storm-cluster-state storm-id) - (.setup-backpressure! storm-cluster-state storm-id) + (if (total-storm-conf TOPOLOGY-BACKPRESSURE-ENABLE) + (.setup-backpressure! storm-cluster-state storm-id)) (notify-topology-action-listener nimbus storm-name "submitTopology") (let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive TopologyInitialStatus/ACTIVE :active}] @@ -1546,6 +1547,7 @@ (mark! nimbus:num-killTopologyWithOpts-calls) (check-storm-active! nimbus storm-name true) (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus) + storm-id (topology-conf STORM-ID) operation "killTopology"] (check-authorization! nimbus storm-name topology-conf operation) (let [wait-amt (if (.is_set_wait_secs options) @@ -1553,6 +1555,8 @@ )] (transition-name! nimbus storm-name [:kill wait-amt] true) (notify-topology-action-listener nimbus storm-name operation)) + (if (topology-conf TOPOLOGY-BACKPRESSURE-ENABLE) + (.remove-backpressure! (:storm-cluster-state nimbus) storm-id)) (add-topology-to-history-log (get-storm-id (:storm-cluster-state nimbus) storm-name) nimbus topology-conf))) diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj index edef2c45230..9d9c4826689 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj @@ -146,7 +146,10 @@ ;; update the worker's backpressure flag to zookeeper only when it has changed (log-debug "BP " @(:backpressure worker) " WAS " prev-backpressure-flag) (when (not= prev-backpressure-flag @(:backpressure worker)) - (.worker-backpressure! storm-cluster-state storm-id assignment-id port @(:backpressure worker))) + (try + (.worker-backpressure! storm-cluster-state storm-id assignment-id port @(:backpressure worker)) + (catch Exception exc + (log-error exc "workerBackpressure update failed when connecting to ZK ... will retry")))) )))) (defn- mk-disruptor-backpressure-handler [worker] @@ -669,8 +672,7 @@ (.interrupt transfer-thread) (.join transfer-thread) (log-message "Shut down transfer thread") - (.interrupt backpressure-thread) - (.join backpressure-thread) + (.terminate backpressure-thread) (log-message "Shut down backpressure thread") (cancel-timer (:heartbeat-timer worker)) (cancel-timer (:refresh-connections-timer worker)) diff --git a/storm-core/src/jvm/org/apache/storm/utils/WorkerBackpressureCallback.java b/storm-core/src/jvm/org/apache/storm/utils/WorkerBackpressureCallback.java index 0b3e45213e5..36f62a9c563 100755 --- a/storm-core/src/jvm/org/apache/storm/utils/WorkerBackpressureCallback.java +++ b/storm-core/src/jvm/org/apache/storm/utils/WorkerBackpressureCallback.java @@ -21,6 +21,5 @@ public interface WorkerBackpressureCallback { - void onEvent(Object obj) throws Exception; - + void onEvent(Object obj); } diff --git a/storm-core/src/jvm/org/apache/storm/utils/WorkerBackpressureThread.java b/storm-core/src/jvm/org/apache/storm/utils/WorkerBackpressureThread.java index 6271198b9f5..f3b5a66cc5f 100644 --- a/storm-core/src/jvm/org/apache/storm/utils/WorkerBackpressureThread.java +++ b/storm-core/src/jvm/org/apache/storm/utils/WorkerBackpressureThread.java @@ -16,21 +16,26 @@ * limitations under the License. */ - package org.apache.storm.utils; -import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class WorkerBackpressureThread extends Thread { - Object trigger; - Object workerData; - WorkerBackpressureCallback callback; + private static final Logger LOG = LoggerFactory.getLogger(WorkerBackpressureThread.class); + private Object trigger; + private Object workerData; + private WorkerBackpressureCallback callback; + private volatile boolean running = true; public WorkerBackpressureThread(Object trigger, Object workerData, WorkerBackpressureCallback callback) { this.trigger = trigger; this.workerData = workerData; this.callback = callback; + this.setName("WorkerBackpressureThread"); + this.setDaemon(true); + this.setUncaughtExceptionHandler(new BackpressureUncaughtExceptionHandler()); } static public void notifyBackpressureChecker(Object trigger) { @@ -43,17 +48,32 @@ static public void notifyBackpressureChecker(Object trigger) { } } + public void terminate() throws InterruptedException { + running = false; + interrupt(); + join(); + } + public void run() { - try { - while (true) { + while (running) { + try { synchronized(trigger) { trigger.wait(100); } callback.onEvent(workerData); // check all executors and update zk backpressure throttle for the worker if needed + } catch (InterruptedException interEx) { + // ignored, we are shutting down. } - } catch (Exception e) { - throw new RuntimeException(e); } } } +class BackpressureUncaughtExceptionHandler implements Thread.UncaughtExceptionHandler { + private static final Logger LOG = LoggerFactory.getLogger(BackpressureUncaughtExceptionHandler.class); + @Override + public void uncaughtException(Thread t, Throwable e) { + // note that exception that happens during connecting to ZK has been ignored in the callback implementation + LOG.error("Received error or exception in WorkerBackpressureThread.. terminating the worker...", e); + Runtime.getRuntime().exit(1); + } +} diff --git a/storm-core/test/jvm/org/apache/storm/utils/WorkerBackpressureThreadTest.java b/storm-core/test/jvm/org/apache/storm/utils/WorkerBackpressureThreadTest.java new file mode 100644 index 00000000000..b8e1770cfb2 --- /dev/null +++ b/storm-core/test/jvm/org/apache/storm/utils/WorkerBackpressureThreadTest.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 org.apache.storm.utils; + +import java.util.concurrent.atomic.AtomicLong; +import org.junit.Assert; +import org.junit.Test; +import junit.framework.TestCase; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class WorkerBackpressureThreadTest extends TestCase { + private static final Logger LOG = LoggerFactory.getLogger(WorkerBackpressureThreadTest.class); + + @Test + public void testNormalEvent() throws Exception { + Object trigger = new Object(); + AtomicLong workerData = new AtomicLong(0); + WorkerBackpressureCallback callback = new WorkerBackpressureCallback() { + @Override + public void onEvent(Object obj) { + ((AtomicLong) obj).getAndDecrement(); + } + }; + WorkerBackpressureThread workerBackpressureThread = new WorkerBackpressureThread(trigger, workerData, callback); + workerBackpressureThread.start(); + WorkerBackpressureThread.notifyBackpressureChecker(trigger); + long start = System.currentTimeMillis(); + while (workerData.get() == 0) { + assertTrue("Timeout", (System.currentTimeMillis() - start) < 1000); + Thread.sleep(100); + } + } +} From 98bc3ae9b6ba1c03d33d7bb1ec61af920ca4659c Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Fri, 26 Feb 2016 00:56:11 +0530 Subject: [PATCH 082/387] [STORM-1576] fix ConcurrentModificationException in addCheckpointInputs Proposed patch addresses the ConcurrentModificationException while creating a topology with an IStatefulBolt having more than one input. --- .../storm/topology/TopologyBuilder.java | 13 ++-- .../storm/topology/TopologyBuilderTest.java | 65 +++++++++++++++++++ 2 files changed, 74 insertions(+), 4 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java b/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java index 6fa953231f1..af415537465 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java +++ b/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java @@ -35,8 +35,11 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; + import org.apache.storm.windowing.TupleWindow; import org.json.simple.JSONValue; import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_COMPONENT_ID; @@ -357,16 +360,18 @@ private IRichBolt maybeAddCheckpointTupleForwarder(IRichBolt bolt) { * add checkpoint stream from the previous bolt to its input. */ private void addCheckPointInputs(ComponentCommon component) { + Set checkPointInputs = new HashSet<>(); for (GlobalStreamId inputStream : component.get_inputs().keySet()) { String sourceId = inputStream.get_componentId(); if (_spouts.containsKey(sourceId)) { - GlobalStreamId checkPointStream = new GlobalStreamId(CHECKPOINT_COMPONENT_ID, CHECKPOINT_STREAM_ID); - component.put_to_inputs(checkPointStream, Grouping.all(new NullStruct())); + checkPointInputs.add(new GlobalStreamId(CHECKPOINT_COMPONENT_ID, CHECKPOINT_STREAM_ID)); } else { - GlobalStreamId checkPointStream = new GlobalStreamId(sourceId, CHECKPOINT_STREAM_ID); - component.put_to_inputs(checkPointStream, Grouping.all(new NullStruct())); + checkPointInputs.add(new GlobalStreamId(sourceId, CHECKPOINT_STREAM_ID)); } } + for (GlobalStreamId streamId : checkPointInputs) { + component.put_to_inputs(streamId, Grouping.all(new NullStruct())); + } } private ComponentCommon getComponentCommon(String id, IComponent component) { diff --git a/storm-core/test/jvm/org/apache/storm/topology/TopologyBuilderTest.java b/storm-core/test/jvm/org/apache/storm/topology/TopologyBuilderTest.java index 0637da38777..56e6f99b585 100644 --- a/storm-core/test/jvm/org/apache/storm/topology/TopologyBuilderTest.java +++ b/storm-core/test/jvm/org/apache/storm/topology/TopologyBuilderTest.java @@ -17,8 +17,21 @@ */ package org.apache.storm.topology; +import com.google.common.collect.ImmutableSet; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.state.State; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.topology.base.BaseStatefulBolt; +import org.apache.storm.tuple.Tuple; +import org.junit.Assert; import org.junit.Test; +import java.util.Map; +import java.util.Set; + import static org.mockito.Mockito.mock; public class TopologyBuilderTest { @@ -50,4 +63,56 @@ public void testAddWorkerHook() { // builder.setStateSpout("stateSpout", mock(IRichStateSpout.class), 0); // } + @Test + public void testStatefulTopology() { + builder.setSpout("spout1", makeDummySpout()); + builder.setSpout("spout2", makeDummySpout()); + builder.setBolt("bolt1", makeDummyStatefulBolt(), 1) + .shuffleGrouping("spout1").shuffleGrouping("spout2"); + builder.setBolt("bolt2", makeDummyStatefulBolt(), 1).shuffleGrouping("spout1"); + builder.setBolt("bolt3", makeDummyStatefulBolt(), 1) + .shuffleGrouping("bolt1").shuffleGrouping("bolt2"); + StormTopology topology = builder.createTopology(); + + Assert.assertNotNull(topology); + Set spouts = topology.get_spouts().keySet(); + // checkpoint spout should 've been added + Assert.assertEquals(ImmutableSet.of("spout1", "spout2", "$checkpointspout"), spouts); + // bolt1, bolt2 should also receive from checkpoint spout + Assert.assertEquals(ImmutableSet.of(new GlobalStreamId("spout1", "default"), + new GlobalStreamId("spout2", "default"), + new GlobalStreamId("$checkpointspout", "$checkpoint")), + topology.get_bolts().get("bolt1").get_common().get_inputs().keySet()); + Assert.assertEquals(ImmutableSet.of(new GlobalStreamId("spout1", "default"), + new GlobalStreamId("$checkpointspout", "$checkpoint")), + topology.get_bolts().get("bolt2").get_common().get_inputs().keySet()); + // bolt3 should also receive from checkpoint streams of bolt1, bolt2 + Assert.assertEquals(ImmutableSet.of(new GlobalStreamId("bolt1", "default"), + new GlobalStreamId("bolt1", "$checkpoint"), + new GlobalStreamId("bolt2", "default"), + new GlobalStreamId("bolt2", "$checkpoint")), + topology.get_bolts().get("bolt3").get_common().get_inputs().keySet()); + } + + private IRichSpout makeDummySpout() { + return new BaseRichSpout() { + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) {} + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {} + @Override + public void nextTuple() {} + private void writeObject(java.io.ObjectOutputStream stream) {} + }; + } + + private IStatefulBolt makeDummyStatefulBolt() { + return new BaseStatefulBolt() { + @Override + public void execute(Tuple input) {} + @Override + public void initState(State state) {} + private void writeObject(java.io.ObjectOutputStream stream) {} + }; + } } From 17da503f49e09fbe309416a434cd7f394a7b5470 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Wed, 10 Feb 2016 11:55:53 -0800 Subject: [PATCH 083/387] Merge branch 'master' of https://github.com/dbahir/storm into STORM-1521 --- .../hbase/security/HBaseSecurityUtil.java | 36 +++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/external/storm-hbase/src/main/java/org/apache/storm/hbase/security/HBaseSecurityUtil.java b/external/storm-hbase/src/main/java/org/apache/storm/hbase/security/HBaseSecurityUtil.java index f306a51d031..e579015c2e0 100644 --- a/external/storm-hbase/src/main/java/org/apache/storm/hbase/security/HBaseSecurityUtil.java +++ b/external/storm-hbase/src/main/java/org/apache/storm/hbase/security/HBaseSecurityUtil.java @@ -39,26 +39,34 @@ public class HBaseSecurityUtil { public static final String STORM_KEYTAB_FILE_KEY = "storm.keytab.file"; public static final String STORM_USER_NAME_KEY = "storm.kerberos.principal"; + private static UserProvider legacyProvider = null; public static UserProvider login(Map conf, Configuration hbaseConfig) throws IOException { //Allowing keytab based login for backward compatibility. - UserProvider provider = UserProvider.instantiate(hbaseConfig); - if (conf.get(TOPOLOGY_AUTO_CREDENTIALS) == null || - !(((List) conf.get(TOPOLOGY_AUTO_CREDENTIALS)).contains(AutoHBase.class.getName()))) { + if (UserGroupInformation.isSecurityEnabled() && (conf.get(TOPOLOGY_AUTO_CREDENTIALS) == null || + !(((List) conf.get(TOPOLOGY_AUTO_CREDENTIALS)).contains(AutoHBase.class.getName())))) { LOG.info("Logging in using keytab as AutoHBase is not specified for " + TOPOLOGY_AUTO_CREDENTIALS); - if (UserGroupInformation.isSecurityEnabled()) { - String keytab = (String) conf.get(STORM_KEYTAB_FILE_KEY); - if (keytab != null) { - hbaseConfig.set(STORM_KEYTAB_FILE_KEY, keytab); + //insure that if keytab is used only one login per process executed + if(legacyProvider == null) { + synchronized (HBaseSecurityUtil.class) { + if(legacyProvider == null) { + legacyProvider = UserProvider.instantiate(hbaseConfig); + String keytab = (String) conf.get(STORM_KEYTAB_FILE_KEY); + if (keytab != null) { + hbaseConfig.set(STORM_KEYTAB_FILE_KEY, keytab); + } + String userName = (String) conf.get(STORM_USER_NAME_KEY); + if (userName != null) { + hbaseConfig.set(STORM_USER_NAME_KEY, userName); + } + legacyProvider.login(STORM_KEYTAB_FILE_KEY, STORM_USER_NAME_KEY, + InetAddress.getLocalHost().getCanonicalHostName()); + } } - String userName = (String) conf.get(STORM_USER_NAME_KEY); - if (userName != null) { - hbaseConfig.set(STORM_USER_NAME_KEY, userName); - } - provider.login(STORM_KEYTAB_FILE_KEY, STORM_USER_NAME_KEY, - InetAddress.getLocalHost().getCanonicalHostName()); } + return legacyProvider; + } else { + return UserProvider.instantiate(hbaseConfig); } - return provider; } } From f8887470b4ffa439a8c45a4ff4b5d93036d7da30 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 29 Feb 2016 08:58:47 -0800 Subject: [PATCH 084/387] Added STORM-1521 to CHANGELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 63b3c6580f5..3761a982027 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1521: When using Kerberos login from keytab with multiple bolts/executors ticket is not renewed * STORM-1488: UI Topology Page component last error timestamp is from 1970 * STORM-1542: Remove profile action retry in case of non-zero exit code * STORM-1540: Fix Debug/Sampling for Trident From d971801a3c4ab7ac52bc22a6f3e198b57e1f2601 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 29 Feb 2016 11:03:14 -0800 Subject: [PATCH 085/387] add STORM-1576 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3761a982027..1f88d1a64b3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1576: fix ConcurrentModificationException in addCheckpointInputs * STORM-1521: When using Kerberos login from keytab with multiple bolts/executors ticket is not renewed * STORM-1488: UI Topology Page component last error timestamp is from 1970 * STORM-1542: Remove profile action retry in case of non-zero exit code From 0b3d732a0cda4f9c837941acd6af8f4acce2ad2c Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 29 Feb 2016 19:57:38 -0800 Subject: [PATCH 086/387] Merge branch 'STORM-1570' of http://github.com/arunmahadevan/storm into STORM-1570 --- .../storm/sql/compiler/CompilerUtil.java | 7 +- .../storm/sql/compiler/ExprCompiler.java | 32 ++++++++-- .../backends/standalone/RelNodeCompiler.java | 6 +- .../apache/storm/sql/parser/StormParser.java | 5 ++ .../org/apache/storm/sql/TestStormSql.java | 64 +++++++++++++++++-- .../storm/sql/compiler/TestCompilerUtils.java | 62 ++++++++++++++++-- .../storm/sql/compiler/TestExprSemantic.java | 18 ++++++ .../backends/standalone/TestPlanCompiler.java | 20 ++++++ .../backends/trident/TestPlanCompiler.java | 4 +- .../test/org/apache/storm/sql/TestUtils.java | 32 +++++++++- 10 files changed, 223 insertions(+), 27 deletions(-) diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java index 30ea0e35753..7f9258c74e9 100644 --- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java +++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java @@ -79,8 +79,11 @@ private FieldType(String name, RelDataType relDataType) { private Statistic stats; public TableBuilderInfo field(String name, SqlTypeName type) { - RelDataType dataType = typeFactory.createSqlType(type); - fields.add(new FieldType(name, dataType)); + return field(name, typeFactory.createSqlType(type)); + } + + public TableBuilderInfo field(String name, RelDataType type) { + fields.add(new FieldType(name, type)); return this; } diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java index 01024f03e28..c43c32fa58f 100644 --- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java +++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java @@ -26,8 +26,12 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.*; import org.apache.calcite.runtime.SqlFunctions; +import org.apache.calcite.schema.Function; +import org.apache.calcite.schema.impl.ReflectiveFunctionBase; +import org.apache.calcite.schema.impl.ScalarFunctionImpl; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.validate.SqlUserDefinedFunction; import org.apache.calcite.util.BuiltInMethod; import org.apache.calcite.util.NlsString; import org.apache.calcite.util.Util; @@ -179,6 +183,7 @@ private ImpTable() { .put(builtInMethod(CHARACTER_LENGTH, BuiltInMethod.CHAR_LENGTH, NullPolicy.STRICT)) .put(builtInMethod(CHAR_LENGTH, BuiltInMethod.CHAR_LENGTH, NullPolicy.STRICT)) .put(builtInMethod(CONCAT, BuiltInMethod.STRING_CONCAT, NullPolicy.STRICT)) + .put(builtInMethod(ITEM, BuiltInMethod.ANY_ITEM, NullPolicy.STRICT)) .put(infixBinary(LESS_THAN, "<", "lt")) .put(infixBinary(LESS_THAN_OR_EQUAL, "<=", "le")) .put(infixBinary(GREATER_THAN, ">", "gt")) @@ -198,7 +203,8 @@ private ImpTable() { .put(expectNot(IS_NOT_FALSE, false)) .put(AND, AND_EXPR) .put(OR, OR_EXPR) - .put(NOT, NOT_EXPR); + .put(NOT, NOT_EXPR) + .put(CAST, CAST_EXPR); this.translators = builder.build(); } @@ -213,7 +219,7 @@ private String compile(ExprCompiler compiler, RexCall call) { } private Map.Entry builtInMethod( - final SqlOperator op, final BuiltInMethod method, NullPolicy nullPolicy) { + final SqlOperator op, final BuiltInMethod method, NullPolicy nullPolicy) { if (nullPolicy != NullPolicy.STRICT) { throw new UnsupportedOperationException(); } @@ -369,8 +375,8 @@ public String translate( String s; if (rhsNullable) { s = foldNullExpr( - String.format("(%2$s != null && !(%2$s)) ? false : %1$s", lhs, - rhs), "null", op1); + String.format("(%2$s != null && !(%2$s)) ? Boolean.FALSE : ((%1$s == null || %2$s == null) ? null : Boolean.TRUE)", + lhs, rhs), "null", op1); } else { s = String.format("!(%2$s) ? Boolean.FALSE : %1$s", lhs, rhs); } @@ -410,7 +416,8 @@ public String translate( String s; if (rhsNullable) { s = foldNullExpr( - String.format("(%2$s != null && %2$s) ? true : %1$s", lhs, rhs), + String.format("(%2$s != null && %2$s) ? Boolean.TRUE : ((%1$s == null || %2$s == null) ? null : Boolean.FALSE)", + lhs, rhs), "null", op1); } else { s = String.format("%2$s ? Boolean.valueOf(%2$s) : %1$s", lhs, rhs); @@ -443,6 +450,21 @@ public String translate( return val; } }; + + + private static final CallExprPrinter CAST_EXPR = new CallExprPrinter() { + @Override + public String translate( + ExprCompiler compiler, RexCall call) { + String val = compiler.reserveName(); + PrintWriter pw = compiler.pw; + RexNode op = call.getOperands().get(0); + String lhs = op.accept(compiler); + pw.print(String.format("final %1$s %2$s = (%1$s) %3$s;\n", + compiler.javaTypeName(call), val, lhs)); + return val; + } + }; } private static String foldNullExpr(String notNullExpr, String diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java index 6d51a116ca1..845bb3ab00c 100644 --- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java +++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java @@ -64,7 +64,11 @@ public Void visitFilter(Filter filter) throws Exception { beginStage(filter); ExprCompiler compiler = new ExprCompiler(pw, typeFactory); String r = filter.getCondition().accept(compiler); - pw.print(String.format(" if (%s) { ctx.emit(_data); }\n", r)); + if (filter.getCondition().getType().isNullable()) { + pw.print(String.format(" if (%s != null && %s) { ctx.emit(_data); }\n", r, r)); + } else { + pw.print(String.format(" if (%s) { ctx.emit(_data); }\n", r, r)); + } endStage(); return null; } diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java index 670901ea3c7..8444e1e8ae4 100644 --- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java +++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java @@ -33,6 +33,11 @@ public StormParser(String s) { this.impl.setQuotedCasing(Lex.ORACLE.quotedCasing); this.impl.setUnquotedCasing(Lex.ORACLE.unquotedCasing); this.impl.setIdentifierMaxLength(DEFAULT_IDENTIFIER_MAX_LENGTH); + /* + * By default parser uses [ ] for quoting identifiers. Switching to DQID (double quoted identifiers) + * is needed for array and map access (m['x'] = 1 or arr[2] = 10 etc) to work. + */ + this.impl.switchTo("DQID"); } @VisibleForTesting diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java index 511e5ab38b6..a85a90781dc 100644 --- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java +++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java @@ -17,11 +17,7 @@ */ package org.apache.storm.sql; -import org.apache.storm.Config; -import org.apache.storm.ILocalCluster; -import org.apache.storm.StormSubmitter; -import org.apache.storm.generated.SubmitOptions; -import org.apache.storm.generated.TopologyInitialStatus; +import com.google.common.collect.ImmutableMap; import org.apache.storm.tuple.Values; import org.apache.storm.sql.runtime.*; import org.junit.AfterClass; @@ -31,9 +27,9 @@ import java.net.URI; import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; +import java.util.Arrays; import java.util.List; +import java.util.Map; public class TestStormSql { private static class MockDataSourceProvider implements DataSourcesProvider { @@ -56,14 +52,37 @@ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, } } + private static class MockNestedDataSourceProvider implements DataSourcesProvider { + @Override + public String scheme() { + return "mocknested"; + } + + @Override + public DataSource construct( + URI uri, String inputFormatClass, String outputFormatClass, + List fields) { + return new TestUtils.MockNestedDataSource(); + } + + @Override + public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, + String properties, List fields) { + throw new UnsupportedOperationException("Not supported"); + } + } + + @BeforeClass public static void setUp() { DataSourcesRegistry.providerMap().put("mock", new MockDataSourceProvider()); + DataSourcesRegistry.providerMap().put("mocknested", new MockNestedDataSourceProvider()); } @AfterClass public static void tearDown() { DataSourcesRegistry.providerMap().remove("mock"); + DataSourcesRegistry.providerMap().remove("mocknested"); } @Test @@ -79,4 +98,35 @@ public void testExternalDataSource() throws Exception { Assert.assertEquals(4, values.get(0).get(0)); Assert.assertEquals(5, values.get(1).get(0)); } + + @Test + public void testExternalDataSourceNested() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'"); + stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " + + "FROM FOO " + + "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[1] = 200"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + System.out.println(values); + Map map = ImmutableMap.of("b", 2, "c", 4); + Map> nestedMap = ImmutableMap.of("a", map); + Assert.assertEquals(new Values(2, map, nestedMap, Arrays.asList(100, 200, 300)), values.get(0)); + } + + @Test + public void testExternalNestedInvalidAccess() throws Exception { + List stmt = new ArrayList<>(); + stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'"); + stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " + + "FROM FOO " + + "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD['a'] = 200"); + StormSql sql = StormSql.construct(); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + sql.execute(stmt, h); + Assert.assertEquals(0, values.size()); + } } diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java index 994e419229c..43b54f72e7e 100644 --- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java +++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java @@ -17,31 +17,74 @@ */ package org.apache.storm.sql.compiler; +import com.google.common.collect.ImmutableList; import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.prepare.CalciteCatalogReader; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.StreamableTable; import org.apache.calcite.schema.Table; +import org.apache.calcite.schema.impl.ScalarFunctionImpl; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.tools.*; +import org.apache.calcite.sql.util.ChainedSqlOperatorTable; +import org.apache.calcite.tools.FrameworkConfig; +import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.tools.Planner; +import org.apache.calcite.tools.RelConversionException; +import org.apache.calcite.tools.ValidationException; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; public class TestCompilerUtils { + public static CalciteState sqlOverDummyTable(String sql) - throws RelConversionException, ValidationException, SqlParseException { + throws RelConversionException, ValidationException, SqlParseException { + SchemaPlus schema = Frameworks.createRootSchema(true); + JavaTypeFactory typeFactory = new JavaTypeFactoryImpl + (RelDataTypeSystem.DEFAULT); + StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory) + .field("ID", SqlTypeName.INTEGER) + .field("NAME", typeFactory.createType(String.class)) + .field("ADDR", typeFactory.createType(String.class)) + .build(); + Table table = streamableTable.stream(); + schema.add("FOO", table); + schema.add("BAR", table); + FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema( + schema).build(); + Planner planner = Frameworks.getPlanner(config); + SqlNode parse = planner.parse(sql); + SqlNode validate = planner.validate(parse); + RelNode tree = planner.convert(validate); + return new CalciteState(schema, tree); + } + + public static CalciteState sqlOverNestedTable(String sql) + throws RelConversionException, ValidationException, SqlParseException { SchemaPlus schema = Frameworks.createRootSchema(true); JavaTypeFactory typeFactory = new JavaTypeFactoryImpl - (RelDataTypeSystem.DEFAULT); + (RelDataTypeSystem.DEFAULT); + StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory) - .field("ID", SqlTypeName.INTEGER).build(); + .field("ID", SqlTypeName.INTEGER) + .field("MAPFIELD", SqlTypeName.ANY) + .field("NESTEDMAPFIELD", SqlTypeName.ANY) + .field("ARRAYFIELD", SqlTypeName.ANY) + .build(); Table table = streamableTable.stream(); schema.add("FOO", table); schema.add("BAR", table); FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema( - schema).build(); + schema).build(); Planner planner = Frameworks.getPlanner(config); SqlNode parse = planner.parse(sql); SqlNode validate = planner.validate(parse); @@ -58,7 +101,12 @@ private CalciteState(SchemaPlus schema, RelNode tree) { this.tree = tree; } - public SchemaPlus schema() { return schema; } - public RelNode tree() { return tree; } + public SchemaPlus schema() { + return schema; + } + + public RelNode tree() { + return tree; + } } } diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java index 8304a3389ac..f2ac0814966 100644 --- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java +++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java @@ -89,6 +89,24 @@ public void testAndWithNull() throws Exception { false, false), v); } + @Test + public void testAndWithNullable() throws Exception { + Values v = testExpr( + Lists.newArrayList( + "ADDR = 'a' AND NAME = 'a'", "NAME = 'a' AND ADDR = 'a'", "NAME = 'x' AND ADDR = 'a'", "ADDR = 'a' AND NAME = 'x'" + )); + assertEquals(new Values(false, false, null, null), v); + } + + @Test + public void testOrWithNullable() throws Exception { + Values v = testExpr( + Lists.newArrayList( + "ADDR = 'a' OR NAME = 'a'", "NAME = 'a' OR ADDR = 'a' ", "NAME = 'x' OR ADDR = 'a' ", "ADDR = 'a' OR NAME = 'x'" + )); + assertEquals(new Values(null, null, true, true), v); + } + @Test public void testOrWithNull() throws Exception { Values v = testExpr( diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java index ff282310e2d..414aeee6234 100644 --- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java +++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java @@ -17,6 +17,7 @@ */ package org.apache.storm.sql.compiler.backends.standalone; +import com.google.common.collect.ImmutableMap; import org.apache.storm.tuple.Values; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.jdbc.JavaTypeFactoryImpl; @@ -30,6 +31,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -66,4 +68,22 @@ public void testLogicalExpr() throws Exception { proc.initialize(data, h); Assert.assertEquals(new Values(true, false, true), values.get(0)); } + + @Test + public void testNested() throws Exception { + String sql = "SELECT ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " + + "FROM FOO " + + "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[1] = 200"; + TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql); + PlanCompiler compiler = new PlanCompiler(typeFactory); + AbstractValuesProcessor proc = compiler.compile(state.tree()); + Map data = new HashMap<>(); + data.put("FOO", new TestUtils.MockNestedDataSource()); + List values = new ArrayList<>(); + ChannelHandler h = new TestUtils.CollectDataChannelHandler(values); + proc.initialize(data, h); + Map map = ImmutableMap.of("b", 2, "c", 4); + Map> nestedMap = ImmutableMap.of("a", map); + Assert.assertEquals(new Values(2, map, nestedMap, Arrays.asList(100, 200, 300)), values.get(0)); + } } diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java index ddc671a0b72..0f8daa92ab2 100644 --- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java +++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java @@ -73,7 +73,7 @@ public void testCompile() throws Exception { @Test public void testInsert() throws Exception { final int EXPECTED_VALUE_SIZE = 1; - String sql = "INSERT INTO BAR SELECT ID FROM FOO WHERE ID > 3"; + String sql = "INSERT INTO BAR SELECT ID, NAME, ADDR FROM FOO WHERE ID > 3"; TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql); PlanCompiler compiler = new PlanCompiler(typeFactory); final AbstractTridentProcessor proc = compiler.compile(state.tree()); @@ -82,7 +82,7 @@ public void testInsert() throws Exception { data.put("BAR", new TestUtils.MockSqlTridentDataSource()); final TridentTopology topo = proc.build(data); runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo); - Assert.assertArrayEquals(new Values[] { new Values(4)}, getCollectedValues().toArray()); + Assert.assertArrayEquals(new Values[] { new Values(4, "x", "y")}, getCollectedValues().toArray()); } private void runTridentTopology(final int expectedValueSize, AbstractTridentProcessor proc, diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java index c5a4043b788..da763a7934e 100644 --- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java +++ b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java @@ -35,6 +35,8 @@ import org.apache.storm.trident.tuple.TridentTuple; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -44,7 +46,31 @@ public static class MockDataSource implements DataSource { public MockDataSource() { for (int i = 0; i < 5; ++i) { - RECORDS.add(new Values(i)); + RECORDS.add(new Values(i, "x", null)); + } + } + + @Override + public void open(ChannelContext ctx) { + for (Values v : RECORDS) { + ctx.emit(v); + } + ctx.fireChannelInactive(); + } + } + + public static class MockNestedDataSource implements DataSource { + private final ArrayList RECORDS = new ArrayList<>(); + + public MockNestedDataSource() { + List ints = Arrays.asList(100, 200, 300); + for (int i = 0; i < 5; ++i) { + Map map = new HashMap<>(); + map.put("b", i); + map.put("c", i*i); + Map> mm = new HashMap<>(); + mm.put("a", map); + RECORDS.add(new Values(i, map, mm, ints)); } } @@ -85,11 +111,11 @@ public void execute(TridentTuple tuple, TridentCollector collector) { private static class MockSpout implements IBatchSpout { private final ArrayList RECORDS = new ArrayList<>(); - private final Fields OUTPUT_FIELDS = new Fields("ID"); + private final Fields OUTPUT_FIELDS = new Fields("ID", "NAME", "ADDR"); public MockSpout() { for (int i = 0; i < 5; ++i) { - RECORDS.add(new Values(i)); + RECORDS.add(new Values(i, "x", "y")); } } From 98ef884e427be98c1572c4fcfe85139d1ffc691c Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 29 Feb 2016 19:59:48 -0800 Subject: [PATCH 087/387] Added STORM-1570 to CHANGELOG --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1f88d1a64b3..3f33c8ca21b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1570: Storm SQL support for nested fields and array * STORM-1576: fix ConcurrentModificationException in addCheckpointInputs * STORM-1521: When using Kerberos login from keytab with multiple bolts/executors ticket is not renewed * STORM-1488: UI Topology Page component last error timestamp is from 1970 From e60eb194c0ab5c1df2cfc42b19d3defa3a85182f Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 1 Mar 2016 12:36:31 -0600 Subject: [PATCH 088/387] Merge branch 'storm1587' of https://github.com/kishorvpatil/incubator-storm into STORM-1587 STORM-1587: Avoid NPE while prining Metrics --- .../src/jvm/org/apache/storm/starter/ThroughputVsLatency.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java index 8ee48c91f1a..8ecfb3a1b03 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java @@ -273,7 +273,7 @@ public static void printMetrics(C client, String name) throws Exception { long acked = 0; long failed = 0; for (ExecutorSummary exec: info.get_executors()) { - if ("spout".equals(exec.get_component_id())) { + if ("spout".equals(exec.get_component_id()) && exec.get_stats() != null && exec.get_stats().get_specific() != null) { SpoutStats stats = exec.get_stats().get_specific().get_spout(); Map failedMap = stats.get_failed().get(":all-time"); Map ackedMap = stats.get_acked().get(":all-time"); From f665cf2035f2dea2d169d3b3869f68565e8c3cda Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 1 Mar 2016 12:53:04 -0600 Subject: [PATCH 089/387] Added STORM-1587 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3f33c8ca21b..49fac0730a8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1587: Avoid NPE while prining Metrics * STORM-1570: Storm SQL support for nested fields and array * STORM-1576: fix ConcurrentModificationException in addCheckpointInputs * STORM-1521: When using Kerberos login from keytab with multiple bolts/executors ticket is not renewed From 3a6e3e47c3154f5d7576e56234b936bfff8491bb Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 1 Mar 2016 12:56:49 -0600 Subject: [PATCH 090/387] Added STORM-1574 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 49fac0730a8..316fa2eaea6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1574: Better handle backpressure exception etc. * STORM-1587: Avoid NPE while prining Metrics * STORM-1570: Storm SQL support for nested fields and array * STORM-1576: fix ConcurrentModificationException in addCheckpointInputs From 4a75d61822dcf9e744d14e808a051e8b2379f938 Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal Date: Wed, 2 Mar 2016 12:58:50 +0530 Subject: [PATCH 091/387] STORM-1588: Do not add event logger details if number of event loggers is zero --- .../clj/org/apache/storm/daemon/nimbus.clj | 33 ++++++++++--------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj index 9376d6e2bfb..5062d0d2046 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj @@ -2135,22 +2135,23 @@ (.set_debug_options comp-page-info (converter/thriftify-debugoptions debug-options))) - ;; Add the event logger details. - (let [component->tasks (reverse-map (:task->component info)) - eventlogger-tasks (sort (get component->tasks - EVENTLOGGER-COMPONENT-ID)) - ;; Find the task the events from this component route to. - task-index (mod (TupleUtils/listHashCode [component-id]) - (count eventlogger-tasks)) - task-id (nth eventlogger-tasks task-index) - eventlogger-exec (first (filter (fn [[start stop]] - (between? task-id start stop)) - (keys executor->host+port))) - [host port] (get executor->host+port eventlogger-exec)] - (if (and host port) - (doto comp-page-info - (.set_eventlog_host host) - (.set_eventlog_port port)))) + ;; Add the event logger details + (let [component->tasks (reverse-map (:task->component info))] + (if (contains? component->tasks EVENTLOGGER-COMPONENT-ID) + (let [eventlogger-tasks (sort (get component->tasks + EVENTLOGGER-COMPONENT-ID)) + ;; Find the task the events from this component route to. + task-index (mod (TupleUtils/listHashCode [component-id]) + (count eventlogger-tasks)) + task-id (nth eventlogger-tasks task-index) + eventlogger-exec (first (filter (fn [[start stop]] + (between? task-id start stop)) + (keys executor->host+port))) + [host port] (get executor->host+port eventlogger-exec)] + (if (and host port) + (doto comp-page-info + (.set_eventlog_host host) + (.set_eventlog_port port)))))) comp-page-info)) (^TopologyHistoryInfo getTopologyHistory [this ^String user] From 94a93d7e6f7d7e1ed3eac4b367c5378c95e3ca2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20D=C3=B8ssing?= Date: Wed, 2 Mar 2016 17:58:52 +0100 Subject: [PATCH 092/387] Add javadoc about reset timeout being expensive --- .../src/jvm/org/apache/storm/task/OutputCollector.java | 1 + .../jvm/org/apache/storm/topology/BasicOutputCollector.java | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/storm-core/src/jvm/org/apache/storm/task/OutputCollector.java b/storm-core/src/jvm/org/apache/storm/task/OutputCollector.java index 071d8aaa9a0..4db87f0d3f5 100644 --- a/storm-core/src/jvm/org/apache/storm/task/OutputCollector.java +++ b/storm-core/src/jvm/org/apache/storm/task/OutputCollector.java @@ -221,6 +221,7 @@ public void fail(Tuple input) { /** * Resets the message timeout for any tuple trees to which the given tuple belongs. * The timeout is reset to Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS. + * Note that this is an expensive operation, and should be used sparingly. * @param input the tuple to reset timeout for */ @Override diff --git a/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java b/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java index 343c349ec06..1d1e5ffff2f 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java +++ b/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java @@ -52,6 +52,12 @@ public void emitDirect(int taskId, List tuple) { emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); } + /** + * Resets the message timeout for any tuple trees to which the given tuple belongs. + * The timeout is reset to Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS. + * Note that this is an expensive operation, and should be used sparingly. + * @param input the tuple to reset timeout for + */ public void resetTimeout(Tuple tuple){ out.resetTimeout(tuple); } From e2251c4506f681abffa5e59baed02a567f8d4264 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20D=C3=B8ssing?= Date: Wed, 2 Mar 2016 18:02:46 +0100 Subject: [PATCH 093/387] Fix javadoc param name --- .../src/jvm/org/apache/storm/topology/BasicOutputCollector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java b/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java index 1d1e5ffff2f..2cf1e82c173 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java +++ b/storm-core/src/jvm/org/apache/storm/topology/BasicOutputCollector.java @@ -56,7 +56,7 @@ public void emitDirect(int taskId, List tuple) { * Resets the message timeout for any tuple trees to which the given tuple belongs. * The timeout is reset to Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS. * Note that this is an expensive operation, and should be used sparingly. - * @param input the tuple to reset timeout for + * @param tuple the tuple to reset timeout for */ public void resetTimeout(Tuple tuple){ out.resetTimeout(tuple); From 59b29b218357aaf39ae70e829ad5ff20e0732431 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 2 Mar 2016 23:36:32 -0600 Subject: [PATCH 094/387] Decommissioning SimpleTransportPlugin --- storm-core/src/jvm/org/apache/storm/Config.java | 4 ++++ .../apache/storm/security/auth/SimpleTransportPlugin.java | 2 ++ .../apache/storm/security/auth/ThriftConnectionType.java | 6 ++++++ .../org/apache/storm/testing/SingleUserSimpleTransport.java | 5 ++++- 4 files changed, 16 insertions(+), 1 deletion(-) diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java index 3eedc63ca0e..37565b8c24a 100644 --- a/storm-core/src/jvm/org/apache/storm/Config.java +++ b/storm-core/src/jvm/org/apache/storm/Config.java @@ -550,7 +550,9 @@ public class Config extends HashMap { /** * The maximum buffer size thrift should use when reading messages. + * @deprecated As SimpleTrasportPlugin is deprecated, no other thrift transport plugin uses this configuration. */ + @Deprecated @isInteger @isPositiveNumber public static final String NIMBUS_THRIFT_MAX_BUFFER_SIZE = "nimbus.thrift.max_buffer_size"; @@ -1089,7 +1091,9 @@ public class Config extends HashMap { /** * The maximum buffer size thrift should use when reading messages for DRPC. + * @deprecated As SimpleTrasportPlugin is deprecated, no other thrift transport plugin uses this configuration. */ + @Deprecated @isNumber @isPositiveNumber public static final String DRPC_MAX_BUFFER_SIZE = "drpc.max_buffer_size"; diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/SimpleTransportPlugin.java b/storm-core/src/jvm/org/apache/storm/security/auth/SimpleTransportPlugin.java index 339bae9fe7b..7343a2d6254 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/SimpleTransportPlugin.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/SimpleTransportPlugin.java @@ -49,7 +49,9 @@ * Simple transport for Thrift plugin. * * This plugin is designed to be backward compatible with existing Storm code. + * @deprecated Use PlainSaslTransportPlugin instead as default insecure transport plugin */ +@Deprecated public class SimpleTransportPlugin implements ITransportPlugin { protected ThriftConnectionType type; protected Map storm_conf; diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java b/storm-core/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java index 27db1430742..f627956a6c9 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java @@ -71,6 +71,12 @@ public int getNumThreads(Map conf) { return Utils.getInt(conf.get(_threadsConf)); } + /** + * @deprecated As SimpleTrasportPlugin is deprecated, no other thrift transport plugin uses this method. + * @param conf + * @return + */ + @Deprecated public int getMaxBufferSize(Map conf) { return Utils.getInt(conf.get(_buffConf)); } diff --git a/storm-core/src/jvm/org/apache/storm/testing/SingleUserSimpleTransport.java b/storm-core/src/jvm/org/apache/storm/testing/SingleUserSimpleTransport.java index 0196a23aab5..97bb6c06f34 100644 --- a/storm-core/src/jvm/org/apache/storm/testing/SingleUserSimpleTransport.java +++ b/storm-core/src/jvm/org/apache/storm/testing/SingleUserSimpleTransport.java @@ -23,7 +23,10 @@ import java.security.Principal; import java.util.HashSet; - +/** + * @deprecated This is no longer available for testing purpose. + */ +@Deprecated public class SingleUserSimpleTransport extends SimpleTransportPlugin { @Override protected Subject getDefaultSubject() { From 4fdb14a0b24ca52ad6a3f4616965e75613472452 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Thu, 3 Mar 2016 12:38:01 -0600 Subject: [PATCH 095/387] Check if /backpressure/storm-id znode exists before requesting children --- storm-core/src/clj/org/apache/storm/cluster.clj | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/cluster.clj b/storm-core/src/clj/org/apache/storm/cluster.clj index 9b01df98345..5315f1a671b 100644 --- a/storm-core/src/clj/org/apache/storm/cluster.clj +++ b/storm-core/src/clj/org/apache/storm/cluster.clj @@ -486,11 +486,13 @@ (topology-backpressure [this storm-id callback] - "if the backpresure/storm-id dir is not empty, this topology has throttle-on, otherwise throttle-off." + "if the backpresure/storm-id dir is not empty, this topology has throttle-on, otherwise throttle-off. + The backpressure/storm-id dir may not exist if nimbus has shutdown the topology" (when callback (swap! backpressure-callback assoc storm-id callback)) (let [path (backpressure-storm-root storm-id) - children (.get_children cluster-state path (not-nil? callback))] + children (if (.node_exists cluster-state path false) + (.get_children cluster-state path (not-nil? callback))) ] (> (count children) 0))) (setup-backpressure! From 51889ae618bef14b77e6e9f621f4941b0095e9cd Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 3 Mar 2016 13:41:32 -0600 Subject: [PATCH 096/387] Merge branch 'storm1596' of https://github.com/kishorvpatil/incubator-storm into STORM-1596 STORM-1596: Do not use single Kerberos TGT instance between multiple threads. --- .../apache/storm/security/auth/AuthUtils.java | 40 ++++++++++++ .../storm/security/auth/kerberos/AutoTGT.java | 64 ++++++++----------- .../auth/kerberos/AutoTGTKrb5LoginModule.java | 8 ++- .../security/auth/auto_login_module_test.clj | 24 ++++++- 4 files changed, 96 insertions(+), 40 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java index 86e11484434..72b7d7c0b01 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/AuthUtils.java @@ -17,10 +17,16 @@ */ package org.apache.storm.security.auth; +import javax.security.auth.kerberos.KerberosTicket; import org.apache.storm.Config; import javax.security.auth.login.Configuration; import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.Subject; +import javax.xml.bind.DatatypeConverter; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.security.URIParameter; import java.security.MessageDigest; @@ -345,4 +351,38 @@ public static String makeDigestPayload(Configuration login_config, String config throw new RuntimeException(e); } } + + public static byte[] serializeKerberosTicket(KerberosTicket tgt) throws Exception { + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bao); + out.writeObject(tgt); + out.flush(); + out.close(); + return bao.toByteArray(); + } + + public static KerberosTicket deserializeKerberosTicket(byte[] tgtBytes) { + KerberosTicket ret; + try { + + ByteArrayInputStream bin = new ByteArrayInputStream(tgtBytes); + ObjectInputStream in = new ObjectInputStream(bin); + ret = (KerberosTicket)in.readObject(); + in.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return ret; + } + + public static KerberosTicket cloneKerberosTicket(KerberosTicket kerberosTicket) { + if(kerberosTicket != null) { + try { + return (deserializeKerberosTicket(serializeKerberosTicket(kerberosTicket))); + } catch (Exception e) { + throw new RuntimeException("Failed to clone KerberosTicket TGT!!", e); + } + } + return null; + } } diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java index 2590ce4634b..c3f85607637 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java @@ -24,10 +24,6 @@ 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; @@ -110,12 +106,9 @@ public void populateCredentials(Map credentials) { 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())); + + byte[] bytes = AuthUtils.serializeKerberosTicket(tgt); + credentials.put("TGT", DatatypeConverter.printBase64Binary(bytes)); } catch (Exception e) { throw new RuntimeException(e); } @@ -123,15 +116,8 @@ public static void saveTGT(KerberosTicket tgt, Map credentials) 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); - } + if (credentials != null && credentials.containsKey("TGT") && credentials.get("TGT") != null) { + ret = AuthUtils.deserializeKerberosTicket(DatatypeConverter.parseBase64Binary(credentials.get("TGT"))); } return ret; } @@ -150,23 +136,7 @@ public void populateSubject(Subject subject, Map credentials) { private void populateSubjectWithTGT(Subject subject, Map credentials) { KerberosTicket tgt = getTGT(credentials); if (tgt != null) { - 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 destroy ticket ", e); - } - } - } - creds.add(tgt); - } + clearCredentials(subject, tgt); subject.getPrincipals().add(tgt.getClient()); kerbTicket.set(tgt); } else { @@ -174,6 +144,28 @@ private void populateSubjectWithTGT(Subject subject, Map credent } } + public static void clearCredentials(Subject subject, KerberosTicket tgt) { + 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); + } + } + } + if(tgt != null) { + creds.add(tgt); + } + } + } + /** * Hadoop does not just go off of a TGT, it needs a bit more. This * should fill in the rest. diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java index fd01297733d..c2b37e385db 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java @@ -21,6 +21,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.storm.security.auth.AuthUtils; import java.security.Principal; import java.util.Map; import javax.security.auth.Subject; @@ -79,7 +80,10 @@ public boolean commit() throws LoginException { throw new LoginException("Authentication failed because the Subject is invalid."); } // Let us add the kerbClientPrinc and kerbTicket - subject.getPrivateCredentials().add(kerbTicket); + // We need to clone the ticket because java.security.auth.kerberos assumes TGT is unique for each subject + // So, sharing TGT with multiple subjects can cause expired TGT to never refresh. + KerberosTicket kerbTicketCopy = AuthUtils.cloneKerberosTicket(kerbTicket); + subject.getPrivateCredentials().add(kerbTicketCopy); subject.getPrincipals().add(getKerbTicketClient()); LOG.debug("Commit Succeeded."); return true; @@ -96,7 +100,7 @@ public boolean abort() throws LoginException { public boolean logout() throws LoginException { if (subject != null && !subject.isReadOnly() && kerbTicket != null) { subject.getPrincipals().remove(kerbTicket.getClient()); - subject.getPrivateCredentials().remove(kerbTicket); + AutoTGT.clearCredentials(subject, null); } kerbTicket = null; return true; diff --git a/storm-core/test/clj/org/apache/storm/security/auth/auto_login_module_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/auto_login_module_test.clj index d976c7982a5..518bb74bdef 100644 --- a/storm-core/test/clj/org/apache/storm/security/auth/auto_login_module_test.clj +++ b/storm-core/test/clj/org/apache/storm/security/auth/auto_login_module_test.clj @@ -19,8 +19,12 @@ (:import [org.apache.storm.security.auth.kerberos AutoTGT AutoTGTKrb5LoginModule AutoTGTKrb5LoginModuleTest]) (:import [javax.security.auth Subject Subject]) - (:import [javax.security.auth.kerberos KerberosTicket]) + (:import [javax.security.auth.kerberos KerberosTicket KerberosPrincipal]) (:import [org.mockito Mockito]) + (:import [java.text SimpleDateFormat]) + (:import [java.util Date]) + (:import [java.util Arrays]) + (:import [java.net InetAddress]) ) (deftest login-module-no-subj-no-tgt-test @@ -82,7 +86,23 @@ (let [login-module (AutoTGTKrb5LoginModuleTest.) _ (set! (. login-module client) (Mockito/mock java.security.Principal)) - ticket (Mockito/mock KerberosTicket)] + endTime (.parse (java.text.SimpleDateFormat. "ddMMyyyy") "31122030") + asn1Enc (byte-array 10) + _ (Arrays/fill asn1Enc (byte 122)) + sessionKey (byte-array 10) + _ (Arrays/fill sessionKey (byte 123)) + ticket (KerberosTicket. + asn1Enc + (KerberosPrincipal. "client/localhost@local.com") + (KerberosPrincipal. "server/localhost@local.com") + sessionKey + 234 + (boolean-array (map even? (range 3 10))) + (Date.) + (Date.) + endTime, + endTime, + (into-array InetAddress [(InetAddress/getByName "localhost")]))] (.initialize login-module (Subject.) nil nil nil) (.setKerbTicket login-module ticket) (is (.login login-module)) From 66ce7cee99eb6f82d762cda80ed3169d11e08cfd Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 3 Mar 2016 14:19:39 -0600 Subject: [PATCH 097/387] Added STORM-1596 ot Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 316fa2eaea6..491e894f774 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -280,6 +280,7 @@ ## 0.10.1 + * STORM-1596: Do not use single Kerberos TGT instance between multiple threads * STORM-1481: avoid Math.abs(Integer) get a negative value * STORM-1121: Deprecate test only configuraton nimbus.reassign * STORM-1180: FLUX logo wasn't appearing quite right From 84e3444ec470a6fbe1039326bda9bae808a3afe8 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 4 Mar 2016 09:20:58 -0600 Subject: [PATCH 098/387] Added STORM-1601 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 491e894f774..53c6d0ba10e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1601: Check if /backpressure/storm-id node exists before requesting children * STORM-1574: Better handle backpressure exception etc. * STORM-1587: Avoid NPE while prining Metrics * STORM-1570: Storm SQL support for nested fields and array From d5e8bab2a83850c1eeabc89a8981f30adeefe3ef Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 4 Mar 2016 16:45:54 +0000 Subject: [PATCH 099/387] Merge branch 'storm1561' of https://github.com/kishorvpatil/incubator-storm Conflicts: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj --- .../src/clj/org/apache/storm/daemon/supervisor.clj | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj index b504e570970..2b1f4b83230 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj @@ -524,6 +524,16 @@ (rm-topo-files conf storm-id localizer false) storm-id))))) +(defn kill-existing-workers-with-change-in-components [supervisor existing-assignment new-assignment] + (let [assigned-executors (or (ls-local-assignments (:local-state supervisor)) {}) + allocated (read-allocated-workers supervisor assigned-executors (Time/currentTimeSecs)) + valid-allocated (filter-val (fn [[state _]] (= state :valid)) allocated) + port->worker-id (clojure.set/map-invert (map-val #((nth % 1) :port) valid-allocated))] + (doseq [p (set/intersection (set (keys existing-assignment)) + (set (keys new-assignment)))] + (if (not= (:executors (existing-assignment p)) (:executors (new-assignment p))) + (shutdown-worker supervisor (port->worker-id p)))))) + (defn mk-synchronize-supervisor [supervisor sync-processes event-manager processes-event-manager] (fn this [] (let [conf (:conf supervisor) @@ -581,6 +591,7 @@ (doseq [p (set/difference (set (keys existing-assignment)) (set (keys new-assignment)))] (.killedWorker isupervisor (int p))) + (kill-existing-workers-with-change-in-components supervisor existing-assignment new-assignment) (.assigned isupervisor (keys new-assignment)) (ls-local-assignments! local-state new-assignment) From af50c25385e7ea48bf21f292bd91c253b547854e Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 4 Mar 2016 17:06:55 +0000 Subject: [PATCH 100/387] Added STORM-1561 ot Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 53c6d0ba10e..481b80383fa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1561: Supervisor should relaunch worker if assignments have changed * STORM-1601: Check if /backpressure/storm-id node exists before requesting children * STORM-1574: Better handle backpressure exception etc. * STORM-1587: Avoid NPE while prining Metrics From 5ad3737366907a0fa0c8198d93d2951911a45dad Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 4 Mar 2016 17:34:41 +0000 Subject: [PATCH 101/387] Merge branch 'storm1543' of https://github.com/kishorvpatil/incubator-storm --- storm-core/src/jvm/org/apache/storm/drpc/DRPCSpout.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/storm-core/src/jvm/org/apache/storm/drpc/DRPCSpout.java b/storm-core/src/jvm/org/apache/storm/drpc/DRPCSpout.java index e500c7d4dcf..791fc917da0 100644 --- a/storm-core/src/jvm/org/apache/storm/drpc/DRPCSpout.java +++ b/storm-core/src/jvm/org/apache/storm/drpc/DRPCSpout.java @@ -182,6 +182,8 @@ public void nextTuple() { client = _clients.get(i); } if (!client.isConnected()) { + LOG.warn("DRPCInvocationsClient [{}:{}] is not connected.", client.getHost(), client.getPort()); + reconnect(client); continue; } try { From 8d216926ade3d1e358b016a851fd6348ec54a286 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 4 Mar 2016 17:46:17 +0000 Subject: [PATCH 102/387] Added STORM-1543 ot Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 481b80383fa..7257aa828e1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1543: DRPCSpout should always try to reconnect disconnected DRPCInvocationsClient * STORM-1561: Supervisor should relaunch worker if assignments have changed * STORM-1601: Check if /backpressure/storm-id node exists before requesting children * STORM-1574: Better handle backpressure exception etc. From 275bae83958e8c137f0faa1142a01639631531b7 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 4 Mar 2016 12:11:21 -0600 Subject: [PATCH 103/387] Change default temp dir for workers to worker launch directory. --- storm-core/src/clj/org/apache/storm/config.clj | 4 ++++ storm-core/src/clj/org/apache/storm/daemon/supervisor.clj | 4 ++++ storm-core/test/clj/org/apache/storm/supervisor_test.clj | 3 +++ 3 files changed, 11 insertions(+) diff --git a/storm-core/src/clj/org/apache/storm/config.clj b/storm-core/src/clj/org/apache/storm/config.clj index d65c43996f9..0717a2233b1 100644 --- a/storm-core/src/clj/org/apache/storm/config.clj +++ b/storm-core/src/clj/org/apache/storm/config.clj @@ -295,6 +295,10 @@ [conf id] (str (worker-root conf id) file-path-separator "pids")) +(defn worker-tmp-root + [conf id] + (str (worker-root conf id) file-path-separator "tmp")) + (defn worker-pid-path [conf id pid] (str (worker-pids-root conf id) file-path-separator pid)) diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj index 2b1f4b83230..1287d775039 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj @@ -299,6 +299,7 @@ (rmr-as-user conf id (worker-pid-path conf id pid)) (try (rmpath (worker-pid-path conf id pid)) + (rmpath (worker-tmp-root conf id pid)) (catch Exception e)))) ;; on windows, the supervisor may still holds the lock on the worker directory (try-cleanup-worker conf id)) (log-message "Shut down " (:supervisor-id supervisor) ":" id)) @@ -375,6 +376,7 @@ (log-message "Launching worker with assignment " (get-worker-assignment-helper-msg assignment supervisor port id)) (local-mkdirs (worker-pids-root conf id)) + (local-mkdirs (worker-tmp-root conf id)) (local-mkdirs (worker-heartbeats-root conf id)) (launch-worker supervisor (:storm-id assignment) @@ -1051,6 +1053,7 @@ storm-home (System/getProperty "storm.home") storm-options (System/getProperty "storm.options") storm-conf-file (System/getProperty "storm.conf.file") + worker-tmp-dir (worker-tmp-root conf worker-id) storm-log-dir LOG-DIR storm-log-conf-dir (conf STORM-LOG4J2-CONF-DIR) storm-log4j2-conf-dir (if storm-log-conf-dir @@ -1120,6 +1123,7 @@ (str "-Dstorm.conf.file=" storm-conf-file) (str "-Dstorm.options=" storm-options) (str "-Dstorm.log.dir=" storm-log-dir) + (str "-Djava.io.tmpdir=" worker-tmp-dir) (str "-Dlogging.sensitivity=" logging-sensitivity) (str "-Dlog4j.configurationFile=" log4j-configuration-file) (str "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector") diff --git a/storm-core/test/clj/org/apache/storm/supervisor_test.clj b/storm-core/test/clj/org/apache/storm/supervisor_test.clj index d9daf329948..ef9f883cf12 100644 --- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj +++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj @@ -297,6 +297,8 @@ "-Dstorm.conf.file=" "-Dstorm.options=" (str "-Dstorm.log.dir=" file-path-separator "logs") + (str "-Djava.io.tmpdir=" file-path-separator "storm-local" file-path-separator + "workers" file-path-separator mock-worker-id file-path-separator "tmp") (str "-Dlogging.sensitivity=" mock-sensitivity) (str "-Dlog4j.configurationFile=" file-path-separator "log4j2" file-path-separator "worker.xml") "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector" @@ -441,6 +443,7 @@ " '-Dstorm.conf.file='" " '-Dstorm.options='" " '-Dstorm.log.dir=/logs'" + " '-Djava.io.tmpdir=" (str storm-local "/workers/" mock-worker-id "/tmp'") " '-Dlogging.sensitivity=" mock-sensitivity "'" " '-Dlog4j.configurationFile=/log4j2/worker.xml'" " '-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector'" From 00d12e26d72ca8e32089606bbdf59af9a376ed8c Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 4 Mar 2016 18:48:47 +0000 Subject: [PATCH 104/387] Added STORM-1529 ot Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7257aa828e1..f6fbefb1e2e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1529: Change default worker temp directory location for workers * STORM-1543: DRPCSpout should always try to reconnect disconnected DRPCInvocationsClient * STORM-1561: Supervisor should relaunch worker if assignments have changed * STORM-1601: Check if /backpressure/storm-id node exists before requesting children From ce6b4681365187e4844a91260bc23221f29837b5 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 9 Feb 2016 16:30:34 +0000 Subject: [PATCH 105/387] STORM-1436: Set Travis Heap size to fit in memory limits. --- dev-tools/travis/travis-script.sh | 4 +++- external/storm-mqtt/core/pom.xml | 4 ++-- pom.xml | 2 ++ 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/dev-tools/travis/travis-script.sh b/dev-tools/travis/travis-script.sh index 7f00cb49121..4984a57d242 100755 --- a/dev-tools/travis/travis-script.sh +++ b/dev-tools/travis/travis-script.sh @@ -24,8 +24,10 @@ cd ${STORM_SRC_ROOT_DIR} # We should be concerned that Travis CI could be very slow because it uses VM export STORM_TEST_TIMEOUT_MS=150000 +# Travis only has 3GB of memory, lets use 1GB for build, and 1.5GB for forked JVMs +#export MAVEN_OPTS="-Xmx1024m" -mvn --batch-mode test -fae -Pnative,all-tests -Prat -pl $2 +mvn --batch-mode test -fae -Pnative,all-tests -Prat -pl "$2" BUILD_RET_VAL=$? for dir in `find . -type d -and -wholename \*/target/\*-reports`; diff --git a/external/storm-mqtt/core/pom.xml b/external/storm-mqtt/core/pom.xml index ee16dae5e8e..cd3001252ee 100644 --- a/external/storm-mqtt/core/pom.xml +++ b/external/storm-mqtt/core/pom.xml @@ -94,7 +94,7 @@ ${java.unit.test.include} - -Djava.net.preferIPv4Stack=true + -Djava.net.preferIPv4Stack=true -Xmx1536m @@ -109,7 +109,7 @@ ${java.integration.test.include} ${java.integration.test.group} - -Djava.net.preferIPv4Stack=true + -Djava.net.preferIPv4Stack=true -Xmx1536m diff --git a/pom.xml b/pom.xml index 0f690ff09ff..f507265e4b3 100644 --- a/pom.xml +++ b/pom.xml @@ -876,6 +876,7 @@ ${java.unit.test.include} + -Xmx1536m @@ -887,6 +888,7 @@ ${java.integration.test.include} ${java.integration.test.group} + -Xmx1536m From 39c34aeb0cf95b0fca50997cd4182c629971b8e6 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 10 Feb 2016 15:59:38 +0000 Subject: [PATCH 106/387] Fixed missed heap settings --- dev-tools/travis/travis-script.sh | 2 +- storm-core/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev-tools/travis/travis-script.sh b/dev-tools/travis/travis-script.sh index 4984a57d242..c6625cb57eb 100755 --- a/dev-tools/travis/travis-script.sh +++ b/dev-tools/travis/travis-script.sh @@ -25,7 +25,7 @@ cd ${STORM_SRC_ROOT_DIR} # We should be concerned that Travis CI could be very slow because it uses VM export STORM_TEST_TIMEOUT_MS=150000 # Travis only has 3GB of memory, lets use 1GB for build, and 1.5GB for forked JVMs -#export MAVEN_OPTS="-Xmx1024m" +export MAVEN_OPTS="-Xmx1024m" mvn --batch-mode test -fae -Pnative,all-tests -Prat -pl "$2" BUILD_RET_VAL=$? diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 37da95dc65c..5386d855572 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -438,7 +438,7 @@ true test/resources/test_runner.clj - ${argLine} ${test.extra.args} + -Xmx1536m ${argLine} ${test.extra.args} ${clojure.test.set} From e5c23a936b5be3b6c68b51577e5e17760167b350 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Fri, 4 Mar 2016 19:01:32 +0000 Subject: [PATCH 107/387] Fixing Travis Build failures by configuring heap size --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index f6fbefb1e2e..0f353d8671f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1436: Set Travis Heap size to fit in memory limits in travis builds. * STORM-1529: Change default worker temp directory location for workers * STORM-1543: DRPCSpout should always try to reconnect disconnected DRPCInvocationsClient * STORM-1561: Supervisor should relaunch worker if assignments have changed From 1f2ddec68ae34f88f16d5ad6e0336355aa1394dd Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 7 Mar 2016 13:05:21 -0600 Subject: [PATCH 108/387] Merge branch 'travis' of https://github.com/hustfxj/storm into STORM-1606 STORM-1606: print the information of testcase which is on failure --- dev-tools/travis/print-errors-from-test-reports.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dev-tools/travis/print-errors-from-test-reports.py b/dev-tools/travis/print-errors-from-test-reports.py index a91f49d00ef..72af6d5e19d 100644 --- a/dev-tools/travis/print-errors-from-test-reports.py +++ b/dev-tools/travis/print-errors-from-test-reports.py @@ -55,6 +55,10 @@ def print_error_reports_from_report_file(file_path): if fail is not None: print_detail_information(testcase, fail) + failure = testcase.find("failure") + if failure is not None: + print_detail_information(testcase, failure) + def main(report_dir_path): for test_report in glob.iglob(report_dir_path + '/*.xml'): From 140a6a4fbfc92c83f4d14518146ccc5afbefd607 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 7 Mar 2016 13:07:16 -0600 Subject: [PATCH 109/387] Added STORM-1606 to Changelog Conflicts: CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0f353d8671f..e7c125c1b2b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1606: print the information of testcase which is on failure * STORM-1436: Set Travis Heap size to fit in memory limits in travis builds. * STORM-1529: Change default worker temp directory location for workers * STORM-1543: DRPCSpout should always try to reconnect disconnected DRPCInvocationsClient From 8679384f1526420b8abd6b8c7fad40c9d2420318 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 7 Mar 2016 13:12:53 -0600 Subject: [PATCH 110/387] Added STORM-1588 to Changelog Conflicts: CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e7c125c1b2b..acb72a0a1ec 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1588: Do not add event logger details if number of event loggers is zero * STORM-1606: print the information of testcase which is on failure * STORM-1436: Set Travis Heap size to fit in memory limits in travis builds. * STORM-1529: Change default worker temp directory location for workers From a70e344a3e3c7abc056bca0fa4d2d48f5d5a223f Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 7 Mar 2016 13:58:03 -0600 Subject: [PATCH 111/387] Merge branch 'storm1469' of https://github.com/kishorvpatil/incubator-storm into STORM-1469 STORM-1469: Adding Plain Sasl Transport Plugin --- .../AbstractSaslClientCallbackHandler.java | 76 +++++++++ .../AbstractSaslServerCallbackHandler.java | 94 +++++++++++ .../auth/digest/ClientCallbackHandler.java | 60 +------ .../auth/digest/ServerCallbackHandler.java | 61 +------ .../plain/PlainClientCallbackHandler.java | 31 ++++ .../auth/plain/PlainSaslTransportPlugin.java | 71 ++++++++ .../plain/PlainServerCallbackHandler.java | 55 ++++++ .../security/auth/plain/SaslPlainServer.java | 158 ++++++++++++++++++ 8 files changed, 497 insertions(+), 109 deletions(-) create mode 100644 storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.java create mode 100644 storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java create mode 100644 storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java create mode 100644 storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java create mode 100644 storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java create mode 100644 storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.java new file mode 100644 index 00000000000..04710bab201 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslClientCallbackHandler.java @@ -0,0 +1,76 @@ +/** + * 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 org.apache.storm.security.auth; + +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.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import java.io.IOException; + +public abstract class AbstractSaslClientCallbackHandler implements CallbackHandler { + protected static final String USERNAME = "username"; + protected static final String PASSWORD = "password"; + private static final Logger LOG = LoggerFactory.getLogger(AbstractSaslClientCallbackHandler.class); + protected String _username = null; + protected String _password = null; + + /** + * 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"); + NameCallback nc = (NameCallback) c; + nc.setName(_username); + } else if (c instanceof PasswordCallback) { + LOG.debug("password callback"); + PasswordCallback pc = (PasswordCallback)c; + if (_password != null) { + pc.setPassword(_password.toCharArray()); + } + } 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 if (c instanceof RealmCallback) { + RealmCallback rc = (RealmCallback) c; + ((RealmCallback) c).setText(rc.getDefaultText()); + } else { + throw new UnsupportedCallbackException(c); + } + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java new file mode 100644 index 00000000000..ebbe2ea3175 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/security/auth/AbstractSaslServerCallbackHandler.java @@ -0,0 +1,94 @@ +/** + * 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 org.apache.storm.security.auth; + +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.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import java.util.HashMap; +import java.util.Map; + +public abstract class AbstractSaslServerCallbackHandler implements CallbackHandler { + private static final Logger LOG = LoggerFactory.getLogger(AbstractSaslServerCallbackHandler.class); + protected final Map credentials = new HashMap<>(); + protected String userName; + + 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 RealmCallback) { + handleRealmCallback((RealmCallback) callback); + } else if (callback instanceof AuthorizeCallback) { + handleAuthorizeCallback((AuthorizeCallback) callback); + } + } + } + + private void handleNameCallback(NameCallback nc) { + LOG.debug("handleNameCallback"); + userName = nc.getDefaultName(); + nc.setName(nc.getDefaultName()); + } + + protected void handlePasswordCallback(PasswordCallback pc) { + LOG.debug("handlePasswordCallback"); + if (credentials.containsKey(userName) ) { + pc.setPassword(credentials.get(userName).toCharArray()); + } else { + LOG.warn("No password found for user: {}", userName); + } + } + + private void handleRealmCallback(RealmCallback rc) { + LOG.debug("handleRealmCallback: {}", rc.getDefaultText()); + rc.setText(rc.getDefaultText()); + } + + private void handleAuthorizeCallback(AuthorizeCallback ac) { + String authenticationID = ac.getAuthenticationID(); + LOG.info("Successfully authenticated client: authenticationID = {} authorizationID = {}", + authenticationID, ac.getAuthorizationID()); + + //if authorizationId is not set, set it to authenticationId. + if(ac.getAuthorizationID() == null) { + ac.setAuthorizedID(authenticationID); + } + + //When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We + //add the authNid as the real user in reqContext's subject which will be used during authorization. + if(!authenticationID.equals(ac.getAuthorizationID())) { + LOG.info("Impersonation attempt authenticationID = {} authorizationID = {}", + ac.getAuthenticationID(), ac.getAuthorizationID()); + ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID())); + } else { + ReqContext.context().setRealPrincipal(null); + } + + ac.setAuthorized(true); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java index 013ce065b3d..312e4abcbe5 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/digest/ClientCallbackHandler.java @@ -17,30 +17,17 @@ */ package org.apache.storm.security.auth.digest; -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.AuthorizeCallback; -import javax.security.sasl.RealmCallback; +import org.apache.storm.security.auth.AbstractSaslClientCallbackHandler; +import org.apache.storm.security.auth.AuthUtils; + import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.storm.security.auth.AuthUtils; +import java.io.IOException; /** * client side callback handler. */ -public class ClientCallbackHandler implements CallbackHandler { - private static final String USERNAME = "username"; - private static final String PASSWORD = "password"; - private static final Logger LOG = LoggerFactory.getLogger(ClientCallbackHandler.class); - private String _username = null; - private String _password = null; +public class ClientCallbackHandler extends AbstractSaslClientCallbackHandler { /** * Constructor based on a JAAS configuration @@ -68,41 +55,4 @@ public ClientCallbackHandler(Configuration configuration) throws IOException { } } - /** - * 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"); - NameCallback nc = (NameCallback) c; - nc.setName(_username); - } else if (c instanceof PasswordCallback) { - LOG.debug("password callback"); - PasswordCallback pc = (PasswordCallback)c; - if (_password != null) { - pc.setPassword(_password.toCharArray()); - } - } 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 if (c instanceof RealmCallback) { - RealmCallback rc = (RealmCallback) c; - ((RealmCallback) c).setText(rc.getDefaultText()); - } else { - throw new UnsupportedCallbackException(c); - } - } - } } diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java index 4fe21c25633..7c4414f257f 100644 --- a/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java +++ b/storm-core/src/jvm/org/apache/storm/security/auth/digest/ServerCallbackHandler.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.storm.security.auth.AbstractSaslServerCallbackHandler; import org.apache.storm.security.auth.ReqContext; import org.apache.storm.security.auth.SaslTransportPlugin; import org.slf4j.Logger; @@ -41,13 +42,10 @@ /** * SASL server side callback handler */ -public class ServerCallbackHandler implements CallbackHandler { - private static final String USER_PREFIX = "user_"; +public class ServerCallbackHandler extends AbstractSaslServerCallbackHandler { private static final Logger LOG = LoggerFactory.getLogger(ServerCallbackHandler.class); - private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; - - private String userName; - private final Map credentials = new HashMap<>(); + private static final String USER_PREFIX = "user_"; + public static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; public ServerCallbackHandler(Configuration configuration) throws IOException { if (configuration==null) return; @@ -72,61 +70,16 @@ public ServerCallbackHandler(Configuration configuration) throws IOException { } } - 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 RealmCallback) { - handleRealmCallback((RealmCallback) 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) { + @Override + protected void handlePasswordCallback(PasswordCallback pc) { LOG.debug("handlePasswordCallback"); if ("super".equals(this.userName) && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) { // superuser: use Java system property for password, if available. pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD).toCharArray()); - } else if (credentials.containsKey(userName) ) { - pc.setPassword(credentials.get(userName).toCharArray()); } else { - LOG.warn("No password found for user: " + userName); + super.handlePasswordCallback(pc); } - } - private void handleRealmCallback(RealmCallback rc) { - LOG.debug("handleRealmCallback: "+ rc.getDefaultText()); - rc.setText(rc.getDefaultText()); } - private void handleAuthorizeCallback(AuthorizeCallback ac) { - String authenticationID = ac.getAuthenticationID(); - LOG.info("Successfully authenticated client: authenticationID = " + authenticationID + " authorizationID = " + ac.getAuthorizationID()); - - //if authorizationId is not set, set it to authenticationId. - if(ac.getAuthorizationID() == null) { - ac.setAuthorizedID(authenticationID); - } - - //When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We - //add the authNid as the real user in reqContext's subject which will be used during authorization. - if(!authenticationID.equals(ac.getAuthorizationID())) { - LOG.info("Impersonation attempt authenticationID = " + ac.getAuthenticationID() + " authorizationID = " + ac.getAuthorizationID()); - ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID())); - } else { - ReqContext.context().setRealPrincipal(null); - } - - ac.setAuthorized(true); - } } diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.java new file mode 100644 index 00000000000..13340dfc4ea --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainClientCallbackHandler.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 org.apache.storm.security.auth.plain; + +import org.apache.storm.security.auth.AbstractSaslClientCallbackHandler; + +public class PlainClientCallbackHandler extends AbstractSaslClientCallbackHandler { + + /* + * For plain, using constants for a pair of user name and password. + */ + public PlainClientCallbackHandler() { + _username = System.getProperty("user.name"); + _password = PASSWORD; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java new file mode 100644 index 00000000000..eaef91a1546 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java @@ -0,0 +1,71 @@ +/** + * 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 org.apache.storm.security.auth.plain; + +import org.apache.storm.security.auth.AuthUtils; +import org.apache.storm.security.auth.SaslTransportPlugin; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.callback.CallbackHandler; +import java.io.IOException; +import java.security.Security; + +public class PlainSaslTransportPlugin extends SaslTransportPlugin { + public static final String PLAIN = "PLAIN"; + private static final Logger LOG = LoggerFactory.getLogger(PlainSaslTransportPlugin.class); + + @Override + protected TTransportFactory getServerTransportFactory() throws IOException { + //create an authentication callback handler + CallbackHandler serverCallbackHandler = new PlainServerCallbackHandler(); + if (Security.getProvider(SaslPlainServer.SecurityProvider.SASL_PLAIN_SERVER) == null) { + Security.addProvider(new SaslPlainServer.SecurityProvider()); + } + //create a transport factory that will invoke our auth callback for digest + TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory(); + factory.addServerDefinition(PLAIN, AuthUtils.SERVICE, "localhost", null, serverCallbackHandler); + + LOG.info("SASL PLAIN transport factory will be used"); + return factory; + } + + @Override + public TTransport connect(TTransport transport, String serverHost, String asUser) throws IOException, TTransportException { + PlainClientCallbackHandler clientCallbackHandler = new PlainClientCallbackHandler(); + TSaslClientTransport wrapperTransport = new TSaslClientTransport(PLAIN, + null, + AuthUtils.SERVICE, + serverHost, + null, + clientCallbackHandler, + transport); + + wrapperTransport.open(); + LOG.debug("SASL PLAIN client transport has been established"); + + return wrapperTransport; + + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.java new file mode 100644 index 00000000000..c646fc925b2 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/PlainServerCallbackHandler.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 org.apache.storm.security.auth.plain; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.storm.security.auth.AbstractSaslServerCallbackHandler; +import org.apache.storm.security.auth.ReqContext; +import org.apache.storm.security.auth.SaslTransportPlugin; +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.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + +/** + * SASL server side callback handler + */ +public class PlainServerCallbackHandler extends AbstractSaslServerCallbackHandler { + private static final Logger LOG = LoggerFactory.getLogger(PlainServerCallbackHandler.class); + public static final String PASSWORD = "password"; + + public PlainServerCallbackHandler() throws IOException { + userName=null; + } + + protected void handlePasswordCallback(PasswordCallback pc) { + LOG.debug("handlePasswordCallback"); + pc.setPassword(PASSWORD.toCharArray()); + + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java b/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java new file mode 100644 index 00000000000..c84ce77f34d --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/security/auth/plain/SaslPlainServer.java @@ -0,0 +1,158 @@ +/** + * 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 org.apache.storm.security.auth.plain; + +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.sasl.AuthorizeCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import javax.security.sasl.SaslServerFactory; +import java.security.Provider; +import java.util.Map; + +public class SaslPlainServer implements SaslServer { + @SuppressWarnings("serial") + public static class SecurityProvider extends Provider { + + public static final String SASL_PLAIN_SERVER = "SaslPlainServer"; + + public SecurityProvider() { + super(SASL_PLAIN_SERVER, 1.0, "SASL PLAIN Authentication Server"); + put("SaslServerFactory.PLAIN", + SaslPlainServerFactory.class.getName()); + } + } + + public static class SaslPlainServerFactory implements SaslServerFactory { + @Override + public SaslServer createSaslServer(String mechanism, String protocol, + String serverName, Map props, CallbackHandler cbh) + throws SaslException { + return "PLAIN".equals(mechanism) ? new SaslPlainServer(cbh) : null; + } + @Override + public String[] getMechanismNames(Map props){ + return (props == null) || "false".equals(props.get(Sasl.POLICY_NOPLAINTEXT)) + ? new String[]{"PLAIN"} + : new String[0]; + } + } + + private CallbackHandler cbh; + private boolean completed; + private String authz; + + SaslPlainServer(CallbackHandler callback) { + this.cbh = callback; + } + + @Override + public String getMechanismName() { + return "PLAIN"; + } + + @Override + public byte[] evaluateResponse(byte[] response) throws SaslException { + if (completed) { + throw new IllegalStateException("PLAIN authentication has completed"); + } + if (response == null) { + throw new IllegalArgumentException("Received null response"); + } + try { + String payload; + try { + payload = new String(response, "UTF-8"); + } catch (Exception e) { + throw new IllegalArgumentException("Received corrupt response", e); + } + // [ authz, authn, password ] + String[] parts = payload.split("\u0000", 3); + if (parts.length != 3) { + throw new IllegalArgumentException("Received corrupt response"); + } + if (parts[0].isEmpty()) { // authz = authn + parts[0] = parts[1]; + } + + NameCallback nc = new NameCallback("SASL PLAIN"); + nc.setName(parts[1]); + PasswordCallback pc = new PasswordCallback("SASL PLAIN", false); + pc.setPassword(parts[2].toCharArray()); + AuthorizeCallback ac = new AuthorizeCallback(parts[1], parts[0]); + cbh.handle(new Callback[]{nc, pc, ac}); + if (ac.isAuthorized()) { + authz = ac.getAuthorizedID(); + } + } catch (Exception e) { + throw new SaslException("PLAIN auth failed: " + e.toString(), e); + } finally { + completed = true; + } + return null; + } + + private void throwIfNotComplete() { + if (!completed) { + throw new IllegalStateException("PLAIN authentication not completed"); + } + } + + @Override + public boolean isComplete() { + return completed; + } + + @Override + public String getAuthorizationID() { + throwIfNotComplete(); + return authz; + } + + @Override + public Object getNegotiatedProperty(String propName) { + throwIfNotComplete(); + return Sasl.QOP.equals(propName) ? "auth" : null; + } + + @Override + public byte[] wrap(byte[] outgoing, int offset, int len) + throws SaslException { + throwIfNotComplete(); + throw new IllegalStateException( + "PLAIN supports neither integrity nor privacy"); + } + + @Override + public byte[] unwrap(byte[] incoming, int offset, int len) + throws SaslException { + throwIfNotComplete(); + throw new IllegalStateException( + "PLAIN supports neither integrity nor privacy"); + } + + @Override + public void dispose() throws SaslException { + cbh = null; + authz = null; + } +} From 23a5392522dd5ffc82c40ad5c2a5366bf40ce0fd Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 7 Mar 2016 14:09:52 -0600 Subject: [PATCH 112/387] Added STORM-1469 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index acb72a0a1ec..898a17efbd1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1469: Adding Plain Sasl Transport Plugin * STORM-1588: Do not add event logger details if number of event loggers is zero * STORM-1606: print the information of testcase which is on failure * STORM-1436: Set Travis Heap size to fit in memory limits in travis builds. From 514ba4e2a3aa8b13b46b9426d3619922428c93db Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal Date: Wed, 9 Mar 2016 01:43:23 +0530 Subject: [PATCH 113/387] STORM-1537: Upgrade to kryo 3 --- pom.xml | 6 ++--- storm-core/pom.xml | 2 +- .../BlowfishTupleSerializer.java | 24 +++++++++---------- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/pom.xml b/pom.xml index f507265e4b3..f7ebcbe4604 100644 --- a/pom.xml +++ b/pom.xml @@ -216,7 +216,7 @@ 7.6.13.v20130916 0.2.3 0.0.1 - 1.4.0 + 1.5.0 1.11 4.3.3 0.2.4 @@ -237,7 +237,7 @@ 1.2.0 0.14.0 2.6.1 - 2.21 + 3.0.3 2.5 2.3 0.9.3 @@ -495,7 +495,7 @@ ${clojure.version} - com.esotericsoftware.kryo + com.esotericsoftware kryo ${kryo.version} diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 5386d855572..21a3857e44f 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -48,7 +48,7 @@ test - com.esotericsoftware.kryo + com.esotericsoftware kryo diff --git a/storm-core/src/jvm/org/apache/storm/security/serialization/BlowfishTupleSerializer.java b/storm-core/src/jvm/org/apache/storm/security/serialization/BlowfishTupleSerializer.java index f459b354f49..8cc2f23a07f 100644 --- a/storm-core/src/jvm/org/apache/storm/security/serialization/BlowfishTupleSerializer.java +++ b/storm-core/src/jvm/org/apache/storm/security/serialization/BlowfishTupleSerializer.java @@ -17,23 +17,23 @@ */ package org.apache.storm.security.serialization; -import java.util.Map; -import org.apache.commons.codec.binary.Hex; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.crypto.KeyGenerator; -import javax.crypto.SecretKey; - import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; -import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.serializers.BlowfishSerializer; +import org.apache.commons.codec.binary.Hex; +import org.apache.storm.Config; import org.apache.storm.serialization.types.ListDelegateSerializer; import org.apache.storm.utils.ListDelegate; -import org.apache.storm.Config; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +import javax.crypto.KeyGenerator; +import javax.crypto.SecretKey; /** * Apply Blowfish encryption for tuple communication to bolts @@ -66,12 +66,12 @@ public BlowfishTupleSerializer(Kryo kryo, Map storm_conf) { @Override public void write(Kryo kryo, Output output, ListDelegate object) { - _serializer.write(kryo, output, object); + kryo.writeObject(output, object, _serializer); } @Override public ListDelegate read(Kryo kryo, Input input, Class type) { - return (ListDelegate)_serializer.read(kryo, input, type); + return kryo.readObject(input, ListDelegate.class, _serializer); } /** From 821d4efe04f8db766d6f0c5f8579a7cda249e8e3 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 9 Mar 2016 15:38:15 -0500 Subject: [PATCH 114/387] add STORM-1469 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 898a17efbd1..7f12405b143 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1469: Decommission SimpleTransportPlugin and configuration * STORM-1469: Adding Plain Sasl Transport Plugin * STORM-1588: Do not add event logger details if number of event loggers is zero * STORM-1606: print the information of testcase which is on failure From 28b96d7db2313e4fc5d47729416594928886c696 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Fri, 11 Mar 2016 08:25:58 -0600 Subject: [PATCH 115/387] STORM-1614: backpressure init and cleanup changes for 1.x branch --- .../src/clj/org/apache/storm/cluster.clj | 14 +- .../clj/org/apache/storm/daemon/nimbus.clj | 15 +- .../clj/org/apache/storm/daemon/worker.clj | 2 + .../test/clj/org/apache/storm/nimbus_test.clj | 129 ++++++++++++++++++ 4 files changed, 151 insertions(+), 9 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/cluster.clj b/storm-core/src/clj/org/apache/storm/cluster.clj index 5315f1a671b..760f3304ec5 100644 --- a/storm-core/src/clj/org/apache/storm/cluster.clj +++ b/storm-core/src/clj/org/apache/storm/cluster.clj @@ -73,6 +73,7 @@ (teardown-topology-errors! [this storm-id]) (heartbeat-storms [this]) (error-topologies [this]) + (backpressure-topologies [this]) (set-topology-log-config! [this storm-id log-config]) (topology-log-config [this storm-id cb]) (worker-heartbeat! [this storm-id node port info]) @@ -277,7 +278,7 @@ ;; this should never happen (exit-process! 30 "Unknown callback for subtree " subtree args)))))] (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE BLOBSTORE-SUBTREE NIMBUSES-SUBTREE - LOGCONFIG-SUBTREE]] + LOGCONFIG-SUBTREE BACKPRESSURE-ROOT]] (.mkdirs cluster-state p acls)) (reify StormClusterState @@ -368,6 +369,10 @@ [this] (.get_children cluster-state ERRORS-SUBTREE false)) + (backpressure-topologies + [this] + (.get_children cluster-state BACKPRESSURE-SUBTREE false)) + (get-worker-heartbeat [this storm-id node port] (let [worker-hb (.get_worker_hb cluster-state (workerbeat-path storm-id node port) false)] @@ -505,8 +510,11 @@ (remove-worker-backpressure! [this storm-id node port] - (.delete_node cluster-state (backpressure-path storm-id node port))) - + (let [path (backpressure-path storm-id node port) + existed (.node_exists cluster-state path false)] + (if existed + (.delete_node cluster-state (backpressure-path storm-id node port))))) + (teardown-topology-errors! [this storm-id] (try-cause diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj index 61457254bc0..e79b94dd42f 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj @@ -1043,12 +1043,13 @@ (filter [this key] (get-id-from-blob-key key)))] (set (.filterAndListKeys blob-store to-id)))) -(defn cleanup-storm-ids [conf storm-cluster-state blob-store] +(defn cleanup-storm-ids [storm-cluster-state blob-store] (let [heartbeat-ids (set (.heartbeat-storms storm-cluster-state)) error-ids (set (.error-topologies storm-cluster-state)) code-ids (code-ids blob-store) + backpressure-ids (set (.backpressure-topologies storm-cluster-state)) assigned-ids (set (.active-storms storm-cluster-state))] - (set/difference (set/union heartbeat-ids error-ids code-ids) assigned-ids) + (set/difference (set/union heartbeat-ids error-ids backpressure-ids code-ids) assigned-ids) )) (defn extract-status-str [base] @@ -1120,6 +1121,9 @@ (blob-rm-key blob-store (master-stormconf-key id) storm-cluster-state) (blob-rm-key blob-store (master-stormcode-key id) storm-cluster-state)) +(defn force-delete-dir [conf id] + (rmr (master-stormdist-root conf id))) + (defn do-cleanup [nimbus] (if (is-leader nimbus :throw-exception false) (let [storm-cluster-state (:storm-cluster-state nimbus) @@ -1127,13 +1131,14 @@ submit-lock (:submit-lock nimbus) blob-store (:blob-store nimbus)] (let [to-cleanup-ids (locking submit-lock - (cleanup-storm-ids conf storm-cluster-state blob-store))] + (cleanup-storm-ids storm-cluster-state blob-store))] (when-not (empty? to-cleanup-ids) (doseq [id to-cleanup-ids] (log-message "Cleaning up " id) (.teardown-heartbeats! storm-cluster-state id) (.teardown-topology-errors! storm-cluster-state id) - (rmr (master-stormdist-root conf id)) + (.remove-backpressure! storm-cluster-state id) + (force-delete-dir conf id) (blob-rm-topology-keys id blob-store storm-cluster-state) (swap! (:heartbeats-cache nimbus) dissoc id))))) (log-message "not a leader, skipping cleanup"))) @@ -1555,8 +1560,6 @@ )] (transition-name! nimbus storm-name [:kill wait-amt] true) (notify-topology-action-listener nimbus storm-name operation)) - (if (topology-conf TOPOLOGY-BACKPRESSURE-ENABLE) - (.remove-backpressure! (:storm-cluster-state nimbus) storm-id)) (add-topology-to-history-log (get-storm-id (:storm-cluster-state nimbus) storm-name) nimbus topology-conf))) diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj index 9d9c4826689..778e83d0217 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj @@ -688,6 +688,8 @@ (run-worker-shutdown-hooks worker) (.remove-worker-heartbeat! (:storm-cluster-state worker) storm-id assignment-id port) + (.remove-worker-backpressure! (:storm-cluster-state worker) storm-id assignment-id port) + (log-message "Disconnecting from storm cluster state context") (.disconnect (:storm-cluster-state worker)) (.close (:cluster-state worker)) diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj index 6b51d3c5e81..0e6e4b69418 100644 --- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj +++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj @@ -1514,3 +1514,132 @@ (is (= (.get_action (.get levels "other-test")) LogLevelAction/UNCHANGED)) (is (= (.get_target_log_level (.get levels "other-test")) "DEBUG"))))))) + +(defn teardown-heartbeats [id]) +(defn teardown-topo-errors [id]) +(defn teardown-backpressure-dirs [id]) + +(defn mock-cluster-state + ([] + (mock-cluster-state nil nil)) + ([active-topos inactive-topos] + (mock-cluster-state active-topos inactive-topos inactive-topos inactive-topos)) + ([active-topos hb-topos error-topos bp-topos] + (reify cluster/StormClusterState + (teardown-heartbeats! [this id] (teardown-heartbeats id)) + (teardown-topology-errors! [this id] (teardown-topo-errors id)) + (remove-backpressure! [this id] (teardown-backpressure-dirs id)) + (active-storms [this] active-topos) + (heartbeat-storms [this] hb-topos) + (error-topologies [this] error-topos) + (backpressure-topologies [this] bp-topos)))) + +(deftest cleanup-storm-ids-returns-inactive-topos + (let [mock-state (mock-cluster-state (list "topo1") (list "topo1" "topo2" "topo3"))] + (stubbing [nimbus/is-leader true + nimbus/code-ids {}] + (is (= (nimbus/cleanup-storm-ids mock-state nil) #{"topo2" "topo3"}))))) + +(deftest cleanup-storm-ids-performs-union-of-storm-ids-with-active-znodes + (let [active-topos (list "hb1" "e2" "bp3") + hb-topos (list "hb1" "hb2" "hb3") + error-topos (list "e1" "e2" "e3") + bp-topos (list "bp1" "bp2" "bp3") + mock-state (mock-cluster-state active-topos hb-topos error-topos bp-topos)] + (stubbing [nimbus/is-leader true + nimbus/code-ids {}] + (is (= (nimbus/cleanup-storm-ids mock-state nil) + #{"hb2" "hb3" "e1" "e3" "bp1" "bp2"}))))) + +(deftest cleanup-storm-ids-returns-empty-set-when-all-topos-are-active + (let [active-topos (list "hb1" "hb2" "hb3" "e1" "e2" "e3" "bp1" "bp2" "bp3") + hb-topos (list "hb1" "hb2" "hb3") + error-topos (list "e1" "e2" "e3") + bp-topos (list "bp1" "bp2" "bp3") + mock-state (mock-cluster-state active-topos hb-topos error-topos bp-topos)] + (stubbing [nimbus/is-leader true + nimbus/code-ids {}] + (is (= (nimbus/cleanup-storm-ids mock-state nil) + #{}))))) + +(deftest do-cleanup-removes-inactive-znodes + (let [inactive-topos (list "topo2" "topo3") + hb-cache (atom (into {}(map vector inactive-topos '(nil nil)))) + mock-state (mock-cluster-state) + mock-blob-store {} + conf {} + nimbus {:conf conf + :submit-lock mock-blob-store + :blob-store {} + :storm-cluster-state mock-state + :heartbeats-cache hb-cache}] + + (stubbing [nimbus/is-leader true + nimbus/blob-rm-topology-keys nil + nimbus/cleanup-storm-ids inactive-topos] + (mocking + [teardown-heartbeats + teardown-topo-errors + teardown-backpressure-dirs + nimbus/force-delete-dir + nimbus/blob-rm-topology-keys] + + (nimbus/do-cleanup nimbus) + + ;; removed heartbeats znode + (verify-nth-call-args-for 1 teardown-heartbeats "topo2") + (verify-nth-call-args-for 2 teardown-heartbeats "topo3") + + ;; removed topo errors znode + (verify-nth-call-args-for 1 teardown-topo-errors "topo2") + (verify-nth-call-args-for 2 teardown-topo-errors "topo3") + + ;; removed backpressure znodes + (verify-nth-call-args-for 1 teardown-backpressure-dirs "topo2") + (verify-nth-call-args-for 2 teardown-backpressure-dirs "topo3") + + ;; removed topo directories + (verify-nth-call-args-for 1 nimbus/force-delete-dir conf "topo2") + (verify-nth-call-args-for 2 nimbus/force-delete-dir conf "topo3") + + ;; removed blob store topo keys + (verify-nth-call-args-for 1 nimbus/blob-rm-topology-keys "topo2" mock-blob-store mock-state) + (verify-nth-call-args-for 2 nimbus/blob-rm-topology-keys "topo3" mock-blob-store mock-state) + + ;; remove topos from heartbeat cache + (is (= (count @hb-cache) 0)))))) + +(deftest do-cleanup-does-not-teardown-active-topos + (let [inactive-topos () + hb-cache (atom {"topo1" nil "topo2" nil}) + mock-state (mock-cluster-state) + mock-blob-store {} + conf {} + nimbus {:conf conf + :submit-lock mock-blob-store + :blob-store {} + :storm-cluster-state mock-state + :heartbeats-cache hb-cache}] + + (stubbing [nimbus/is-leader true + nimbus/blob-rm-topology-keys nil + nimbus/cleanup-storm-ids inactive-topos] + (mocking + [teardown-heartbeats + teardown-topo-errors + teardown-backpressure-dirs + nimbus/force-delete-dir + nimbus/blob-rm-topology-keys] + + (nimbus/do-cleanup nimbus) + + (verify-call-times-for teardown-heartbeats 0) + (verify-call-times-for teardown-topo-errors 0) + (verify-call-times-for teardown-backpressure-dirs 0) + (verify-call-times-for nimbus/force-delete-dir 0) + (verify-call-times-for nimbus/blob-rm-topology-keys 0) + + ;; hb-cache goes down to 1 because only one topo was inactive + (is (= (count @hb-cache) 2)) + (is (contains? @hb-cache "topo1")) + (is (contains? @hb-cache "topo2")))))) From f7a1d1e5fc622b31319ffb060039730ae3118a6b Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sat, 12 Mar 2016 12:46:40 -0800 Subject: [PATCH 116/387] Added logs & build to .gitignore --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index 08b217afd23..54bd2893097 100644 --- a/.gitignore +++ b/.gitignore @@ -38,3 +38,5 @@ metastore_db .settings/ .project .classpath +logs +build From 421685c95db3bfc2d82b61c049583605d166d7ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20D=C3=B8ssing?= Date: Fri, 11 Mar 2016 14:13:58 +0100 Subject: [PATCH 117/387] STORM-1620: Update curator to fix CURATOR-209 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f507265e4b3..facd824fd0e 100644 --- a/pom.xml +++ b/pom.xml @@ -209,7 +209,7 @@ 1.2.1 1.6 0.8.0 - 2.9.0 + 2.10.0 1.1 1.3.0 0.3.1 From e00151469bbd724bc0d9aaada06745062bedc8eb Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sat, 12 Mar 2016 12:49:17 -0800 Subject: [PATCH 118/387] Added STORM-1620 to CHANGELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7f12405b143..0fec6cd973d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1620: Update curator to fix CURATOR-209 * STORM-1469: Decommission SimpleTransportPlugin and configuration * STORM-1469: Adding Plain Sasl Transport Plugin * STORM-1588: Do not add event logger details if number of event loggers is zero From 2cfa136a4ac1dca7d3ef4169ebdb9b315bcf2cae Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sat, 12 Mar 2016 17:47:10 -0800 Subject: [PATCH 119/387] Merge branch 'netty-best-effort' of https://github.com/redsanket/storm into STORM-1609 --- .../apache/storm/messaging/netty/Client.java | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/storm-core/src/jvm/org/apache/storm/messaging/netty/Client.java b/storm-core/src/jvm/org/apache/storm/messaging/netty/Client.java index 976b55019d7..035eb1bcdf9 100644 --- a/storm-core/src/jvm/org/apache/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/org/apache/storm/messaging/netty/Client.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.Map; import java.util.HashMap; +import java.util.Timer; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -74,6 +75,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject, ISa private static final Logger LOG = LoggerFactory.getLogger(Client.class); private static final String PREFIX = "Netty-Client-"; private static final long NO_DELAY_MS = 0L; + private static final Timer timer = new Timer("Netty-ChannelAlive-Timer", true); private final Map stormConf; private final StormBoundedExponentialBackoffRetry retryPolicy; @@ -107,6 +109,12 @@ public class Client extends ConnectionWithStatus implements IStatefulObject, ISa */ private final AtomicInteger messagesLost = new AtomicInteger(0); + /** + * Periodically checks for connected channel in order to avoid loss + * of messages + */ + private final long CHANNEL_ALIVE_INTERVAL_MS = 30000L; + /** * Number of messages buffered in memory. */ @@ -151,10 +159,36 @@ public class Client extends ConnectionWithStatus implements IStatefulObject, ISa bootstrap = createClientBootstrap(factory, bufferSize, stormConf); dstAddress = new InetSocketAddress(host, port); dstAddressPrefixedName = prefixedName(dstAddress); + launchChannelAliveThread(); scheduleConnect(NO_DELAY_MS); batcher = new MessageBuffer(messageBatchSize); } + /** + * This thread helps us to check for channel connection periodically. + * This is performed just to know whether the destination address + * is alive or attempts to refresh connections if not alive. This + * solution is better than what we have now in case of a bad channel. + */ + private void launchChannelAliveThread() { + // netty TimerTask is already defined and hence a fully + // qualified name + timer.schedule(new java.util.TimerTask() { + public void run() { + try { + LOG.debug("running timer task, address {}", dstAddress); + if(closing) { + this.cancel(); + return; + } + getConnectedChannel(); + } catch (Exception exp) { + LOG.error("channel connection error {}", exp); + } + } + }, 0, CHANNEL_ALIVE_INTERVAL_MS); + } + private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize, Map stormConf) { ClientBootstrap bootstrap = new ClientBootstrap(factory); bootstrap.setOption("tcpNoDelay", true); From 8922f20377fec20c9b1941a6fe379fae39ffbde4 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sat, 12 Mar 2016 17:56:44 -0800 Subject: [PATCH 120/387] Added STORM-1609 to CHANGELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0fec6cd973d..fa32a0aa1aa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1609: Netty Client is not best effort delivery on failed Connection * STORM-1620: Update curator to fix CURATOR-209 * STORM-1469: Decommission SimpleTransportPlugin and configuration * STORM-1469: Adding Plain Sasl Transport Plugin From dd1577193d991489a7c46c50dcfa86db9fac5af2 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Mon, 7 Mar 2016 11:30:35 +0530 Subject: [PATCH 121/387] Fix stateful topology acking behavior Right now the acking is automatically taken care of for the non-stateful bolts in a stateful topology. This leads to double acking if BaseRichBolts are part of the topology. For the non-stateful bolts, its better to let the bolt do the acking rather than automatically acking. Signed-off-by: P. Taylor Goetz --- .../storm/starter/spout/RandomIntegerSpout.java | 15 ++++++++++++++- .../storm/topology/CheckpointTupleForwarder.java | 1 - 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/spout/RandomIntegerSpout.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/spout/RandomIntegerSpout.java index f6a35bf18f9..e031f6e545b 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/spout/RandomIntegerSpout.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/spout/RandomIntegerSpout.java @@ -24,6 +24,8 @@ import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Values; import org.apache.storm.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Map; import java.util.Random; @@ -33,6 +35,7 @@ * every 100 ms. The ts field can be used in tuple time based windowing. */ public class RandomIntegerSpout extends BaseRichSpout { + private static final Logger LOG = LoggerFactory.getLogger(RandomIntegerSpout.class); private SpoutOutputCollector collector; private Random rand; private long msgId = 0; @@ -51,6 +54,16 @@ public void open(Map conf, TopologyContext context, SpoutOutputCollector collect @Override public void nextTuple() { Utils.sleep(100); - collector.emit(new Values(rand.nextInt(1000), System.currentTimeMillis() - (24 * 60 * 60 * 1000), ++msgId)); + collector.emit(new Values(rand.nextInt(1000), System.currentTimeMillis() - (24 * 60 * 60 * 1000), ++msgId), msgId); + } + + @Override + public void ack(Object msgId) { + LOG.debug("Got ACK for msgId : " + msgId); + } + + @Override + public void fail(Object msgId) { + LOG.debug("Got FAIL for msgId : " + msgId); } } diff --git a/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java b/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java index 675be5706dc..cbb32152bcf 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java +++ b/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java @@ -116,7 +116,6 @@ protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) protected void handleTuple(Tuple input) { collector.setContext(input); bolt.execute(input); - collector.ack(input); } /** From 2ff57a541f0a38a2b3d537d45e6a2c8955f23b15 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Wed, 9 Mar 2016 16:45:17 +0530 Subject: [PATCH 122/387] Remove auto acking/anchoring for bolts in a stateful topology Signed-off-by: P. Taylor Goetz --- .../jvm/storm/starter/StatefulTopology.java | 1 + .../topology/CheckpointTupleForwarder.java | 21 ++++----- .../apache/storm/topology/IStatefulBolt.java | 7 ++- .../storm/topology/StatefulBoltExecutor.java | 46 +++++++++++++++---- .../storm/topology/TopologyBuilder.java | 5 +- .../topology/StatefulBoltExecutorTest.java | 1 + 6 files changed, 58 insertions(+), 23 deletions(-) diff --git a/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java b/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java index d09ceea2b3e..ba513dd0a2e 100644 --- a/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java +++ b/examples/storm-starter/src/jvm/storm/starter/StatefulTopology.java @@ -90,6 +90,7 @@ public void execute(Tuple input) { LOG.debug("{} sum = {}", name, sum); kvState.put("sum", sum); collector.emit(input, new Values(sum)); + collector.ack(input); } @Override diff --git a/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java b/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java index cbb32152bcf..11d03845367 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java +++ b/storm-core/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java @@ -51,7 +51,7 @@ public class CheckpointTupleForwarder implements IRichBolt { private final Map transactionRequestCount; private int checkPointInputTaskCount; private long lastTxid = Long.MIN_VALUE; - protected AnchoringOutputCollector collector; + private AnchoringOutputCollector collector; public CheckpointTupleForwarder(IRichBolt bolt) { this.bolt = bolt; @@ -60,9 +60,13 @@ public CheckpointTupleForwarder(IRichBolt bolt) { @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { - this.collector = new AnchoringOutputCollector(collector); + init(context, collector); bolt.prepare(stormConf, context, this.collector); - checkPointInputTaskCount = getCheckpointInputTaskCount(context); + } + + protected void init(TopologyContext context, OutputCollector collector) { + this.collector = new AnchoringOutputCollector(collector); + this.checkPointInputTaskCount = getCheckpointInputTaskCount(context); } @Override @@ -114,7 +118,6 @@ protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) * @param input the input tuple */ protected void handleTuple(Tuple input) { - collector.setContext(input); bolt.execute(input); } @@ -224,24 +227,18 @@ public String toString() { protected static class AnchoringOutputCollector extends OutputCollector { - private Tuple inputTuple; - AnchoringOutputCollector(IOutputCollector delegate) { super(delegate); } - void setContext(Tuple inputTuple) { - this.inputTuple = inputTuple; - } - @Override public List emit(String streamId, List tuple) { - return emit(streamId, inputTuple, tuple); + throw new UnsupportedOperationException("Bolts in a stateful topology must emit anchored tuples."); } @Override public void emitDirect(int taskId, String streamId, List tuple) { - emitDirect(taskId, streamId, inputTuple, tuple); + throw new UnsupportedOperationException("Bolts in a stateful topology must emit anchored tuples."); } } diff --git a/storm-core/src/jvm/org/apache/storm/topology/IStatefulBolt.java b/storm-core/src/jvm/org/apache/storm/topology/IStatefulBolt.java index 1c2c5fc6146..ed55e1d27a9 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/IStatefulBolt.java +++ b/storm-core/src/jvm/org/apache/storm/topology/IStatefulBolt.java @@ -20,7 +20,12 @@ import org.apache.storm.state.State; /** - * A bolt abstraction for supporting stateful computation. + * A bolt abstraction for supporting stateful computation. The state of the bolt is + * periodically checkpointed. + * + *

The framework provides at-least once guarantee for the + * state updates. The stateful bolts are expected to anchor the tuples while emitting + * and ack the input tuples once its processed.

*/ public interface IStatefulBolt extends IStatefulComponent, IRichBolt { } diff --git a/storm-core/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java b/storm-core/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java index c9c36eec59b..237305e4050 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java +++ b/storm-core/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java @@ -28,8 +28,12 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; import static org.apache.storm.spout.CheckPointState.Action; import static org.apache.storm.spout.CheckPointState.Action.COMMIT; @@ -47,7 +51,7 @@ public class StatefulBoltExecutor extends CheckpointTupleForwar private boolean boltInitialized = false; private List pendingTuples = new ArrayList<>(); private List preparedTuples = new ArrayList<>(); - private List executedTuples = new ArrayList<>(); + private AckTrackingOutputCollector collector; public StatefulBoltExecutor(IStatefulBolt bolt) { super(bolt); @@ -63,7 +67,9 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll // package access for unit tests void prepare(Map stormConf, TopologyContext context, OutputCollector collector, State state) { - super.prepare(stormConf, context, collector); + init(context, collector); + this.collector = new AckTrackingOutputCollector(collector); + bolt.prepare(stormConf, context, this.collector); this.state = state; } @@ -74,8 +80,7 @@ protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) if (boltInitialized) { bolt.prePrepare(txid); state.prepareCommit(txid); - preparedTuples.addAll(executedTuples); - executedTuples.clear(); + preparedTuples.addAll(collector.ackedTuples()); } else { /* * May be the task restarted in the middle and the state needs be initialized. @@ -93,7 +98,7 @@ protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) bolt.preRollback(); state.rollback(); fail(preparedTuples); - fail(executedTuples); + fail(collector.ackedTuples()); } else if (action == INITSTATE) { if (!boltInitialized) { bolt.initState((T) state); @@ -109,7 +114,7 @@ protected void handleCheckpoint(Tuple checkpointTuple, Action action, long txid) } } collector.emit(CheckpointSpout.CHECKPOINT_STREAM_ID, checkpointTuple, new Values(txid, action)); - collector.ack(checkpointTuple); + collector.delegate.ack(checkpointTuple); } @Override @@ -123,16 +128,14 @@ protected void handleTuple(Tuple input) { } private void doExecute(Tuple tuple) { - collector.setContext(tuple); bolt.execute(tuple); - executedTuples.add(tuple); } private void ack(List tuples) { if (!tuples.isEmpty()) { LOG.debug("Acking {} tuples", tuples.size()); for (Tuple tuple : tuples) { - collector.ack(tuple); + collector.delegate.ack(tuple); } tuples.clear(); } @@ -148,4 +151,29 @@ private void fail(List tuples) { } } + private static class AckTrackingOutputCollector extends AnchoringOutputCollector { + private final OutputCollector delegate; + private final Queue ackedTuples; + + AckTrackingOutputCollector(OutputCollector delegate) { + super(delegate); + this.delegate = delegate; + this.ackedTuples = new ConcurrentLinkedQueue<>(); + } + + List ackedTuples() { + List result = new ArrayList<>(); + Iterator it = ackedTuples.iterator(); + while(it.hasNext()) { + result.add(it.next()); + it.remove(); + } + return result; + } + + @Override + public void ack(Tuple input) { + ackedTuples.add(input); + } + } } diff --git a/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java b/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java index af415537465..5b7d499cef9 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java +++ b/storm-core/src/jvm/org/apache/storm/topology/TopologyBuilder.java @@ -234,7 +234,10 @@ public BoltDeclarer setBolt(String id, IWindowedBolt bolt, Number parallelism_hi * state (of computation) to be saved. When this bolt is initialized, the {@link IStatefulBolt#initState(State)} method * is invoked after {@link IStatefulBolt#prepare(Map, TopologyContext, OutputCollector)} but before {@link IStatefulBolt#execute(Tuple)} * with its previously saved state. - * + *

+ * The framework provides at-least once guarantee for the state updates. Bolts (both stateful and non-stateful) in a stateful topology + * are expected to anchor the tuples while emitting and ack the input tuples once its processed. + *

* @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. * @param bolt the stateful bolt * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somwehere around the cluster. diff --git a/storm-core/test/jvm/org/apache/storm/topology/StatefulBoltExecutorTest.java b/storm-core/test/jvm/org/apache/storm/topology/StatefulBoltExecutorTest.java index 69c541b7291..6606491beb1 100644 --- a/storm-core/test/jvm/org/apache/storm/topology/StatefulBoltExecutorTest.java +++ b/storm-core/test/jvm/org/apache/storm/topology/StatefulBoltExecutorTest.java @@ -170,6 +170,7 @@ public void testPrepareAndCommit() throws Exception { Mockito.when(mockCheckpointTuple.getValueByField(CHECKPOINT_FIELD_ACTION)).thenReturn(COMMIT); Mockito.when(mockCheckpointTuple.getLongByField(CHECKPOINT_FIELD_TXID)).thenReturn(new Long(100)); executor.execute(mockCheckpointTuple); + mockOutputCollector.ack(mockTuple); Mockito.verify(mockState, Mockito.times(1)).commit(new Long(100)); Mockito.verify(mockBolt, Mockito.times(2)).execute(mockTuple); Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple); From 80213bae74d2ca3c9079f26e3070d71b1256c70a Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Mon, 14 Mar 2016 15:24:34 -0400 Subject: [PATCH 123/387] add STORM-1608 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index fa32a0aa1aa..48254bd594f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1608: Fix stateful topology acking behavior * STORM-1609: Netty Client is not best effort delivery on failed Connection * STORM-1620: Update curator to fix CURATOR-209 * STORM-1469: Decommission SimpleTransportPlugin and configuration From 9322283a6c205697a50d1fadc3345e0af798d26b Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Tue, 15 Mar 2016 10:07:25 -0500 Subject: [PATCH 124/387] STORM-1614: force-delete-dir -> force-delete-topo-dist-dir --- storm-core/src/clj/org/apache/storm/daemon/nimbus.clj | 4 ++-- storm-core/test/clj/org/apache/storm/nimbus_test.clj | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj index e79b94dd42f..d34ddb17543 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj @@ -1121,7 +1121,7 @@ (blob-rm-key blob-store (master-stormconf-key id) storm-cluster-state) (blob-rm-key blob-store (master-stormcode-key id) storm-cluster-state)) -(defn force-delete-dir [conf id] +(defn force-delete-topo-dist-dir [conf id] (rmr (master-stormdist-root conf id))) (defn do-cleanup [nimbus] @@ -1138,7 +1138,7 @@ (.teardown-heartbeats! storm-cluster-state id) (.teardown-topology-errors! storm-cluster-state id) (.remove-backpressure! storm-cluster-state id) - (force-delete-dir conf id) + (force-delete-topo-dist-dir conf id) (blob-rm-topology-keys id blob-store storm-cluster-state) (swap! (:heartbeats-cache nimbus) dissoc id))))) (log-message "not a leader, skipping cleanup"))) diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj index 0e6e4b69418..0788fce7613 100644 --- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj +++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj @@ -1581,7 +1581,7 @@ [teardown-heartbeats teardown-topo-errors teardown-backpressure-dirs - nimbus/force-delete-dir + nimbus/force-delete-topo-dist-dir nimbus/blob-rm-topology-keys] (nimbus/do-cleanup nimbus) @@ -1599,8 +1599,8 @@ (verify-nth-call-args-for 2 teardown-backpressure-dirs "topo3") ;; removed topo directories - (verify-nth-call-args-for 1 nimbus/force-delete-dir conf "topo2") - (verify-nth-call-args-for 2 nimbus/force-delete-dir conf "topo3") + (verify-nth-call-args-for 1 nimbus/force-delete-topo-dist-dir conf "topo2") + (verify-nth-call-args-for 2 nimbus/force-delete-topo-dist-dir conf "topo3") ;; removed blob store topo keys (verify-nth-call-args-for 1 nimbus/blob-rm-topology-keys "topo2" mock-blob-store mock-state) @@ -1628,7 +1628,7 @@ [teardown-heartbeats teardown-topo-errors teardown-backpressure-dirs - nimbus/force-delete-dir + nimbus/force-delete-topo-dist-dir nimbus/blob-rm-topology-keys] (nimbus/do-cleanup nimbus) @@ -1636,7 +1636,7 @@ (verify-call-times-for teardown-heartbeats 0) (verify-call-times-for teardown-topo-errors 0) (verify-call-times-for teardown-backpressure-dirs 0) - (verify-call-times-for nimbus/force-delete-dir 0) + (verify-call-times-for nimbus/force-delete-topo-dist-dir 0) (verify-call-times-for nimbus/blob-rm-topology-keys 0) ;; hb-cache goes down to 1 because only one topo was inactive From 7b559ef3d8e02047a15860d261bc0a40d08834e0 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Wed, 16 Mar 2016 00:37:28 +0900 Subject: [PATCH 125/387] STORM-1629 Files/move doesn't work properly with non-empty directory in Windows * Use FileUtils/moveDirectory on Windows * It copies whole contents inside directory, and delete directory * Keep using Files/move on non-Windows * it's still better option since doesn't require copying contents inside directory --- .../src/clj/org/apache/storm/daemon/supervisor.clj | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj index 1287d775039..b37b9da2bd3 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj @@ -959,9 +959,13 @@ (if (download-blobs-for-topology-succeed? (supervisor-stormconf-path tmproot) tmproot) (do (log-message "Successfully downloaded blob resources for storm-id " storm-id) - (FileUtils/forceMkdir (File. stormroot)) - (Files/move (.toPath (File. tmproot)) (.toPath (File. stormroot)) - (doto (make-array StandardCopyOption 1) (aset 0 StandardCopyOption/ATOMIC_MOVE))) + (if on-windows? + ; Files/move with non-empty directory doesn't work well on Windows + (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) + (do + (FileUtils/forceMkdir (File. stormroot)) + (Files/move (.toPath (File. tmproot)) (.toPath (File. stormroot)) + (doto (make-array StandardCopyOption 1) (aset 0 StandardCopyOption/ATOMIC_MOVE))))) (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)) (do (log-message "Failed to download blob resources for storm-id " storm-id) From 6e742210a5aef41f385d74c8be5580a507382ef1 Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal Date: Sun, 13 Mar 2016 00:37:14 +0530 Subject: [PATCH 126/387] STORM-971: Metric for messages lost due to kafka retention Signed-off-by: P. Taylor Goetz --- .../jvm/org/apache/storm/kafka/PartitionManager.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java index dbf70a0a9a2..4db8af68adb 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java @@ -44,6 +44,8 @@ public class PartitionManager { private final ReducedMetric _fetchAPILatencyMean; private final CountMetric _fetchAPICallCount; private final CountMetric _fetchAPIMessageCount; + // Count of messages which could not be emitted or retried because they were deleted from kafka + private final CountMetric _lostMessageCount; Long _emittedToOffset; // _pending key = Kafka offset, value = time at which the message was first submitted to the topology private SortedMap _pending = new TreeMap(); @@ -117,6 +119,7 @@ public PartitionManager(DynamicPartitionConnections connections, String topology _fetchAPILatencyMean = new ReducedMetric(new MeanReducer()); _fetchAPICallCount = new CountMetric(); _fetchAPIMessageCount = new CountMetric(); + _lostMessageCount = new CountMetric(); } public Map getMetricsDataMap() { @@ -125,6 +128,7 @@ public Map getMetricsDataMap() { ret.put(_partition + "/fetchAPILatencyMean", _fetchAPILatencyMean.getValueAndReset()); ret.put(_partition + "/fetchAPICallCount", _fetchAPICallCount.getValueAndReset()); ret.put(_partition + "/fetchAPIMessageCount", _fetchAPIMessageCount.getValueAndReset()); + ret.put(_partition + "/lostMessageCount", _lostMessageCount.getValueAndReset()); return ret; } @@ -185,7 +189,7 @@ private void fill() { msgs = KafkaUtils.fetchMessages(_spoutConfig, _consumer, _partition, offset); } catch (TopicOffsetOutOfRangeException e) { offset = KafkaUtils.getOffset(_consumer, _partition.topic, _partition.partition, kafka.api.OffsetRequest.EarliestTime()); - // fetch failed, so don't update the metrics + // fetch failed, so don't update the fetch metrics //fix bug [STORM-643] : remove outdated failed offsets if (!processingNewTuples) { @@ -194,11 +198,17 @@ private void fill() { // offset, since they are anyway not there. // These calls to broker API will be then saved. Set omitted = this._failedMsgRetryManager.clearInvalidMessages(offset); + + // Omitted messages have not been acked and may be lost + if (null != omitted) { + _lostMessageCount.incrBy(omitted.size()); + } LOG.warn("Removing the failed offsets that are out of range: {}", omitted); } if (offset > _emittedToOffset) { + _lostMessageCount.incrBy(offset - _emittedToOffset); _emittedToOffset = offset; LOG.warn("{} Using new offset: {}", _partition.partition, _emittedToOffset); } From f0abfff92db39da900c3f052fe0ffd49747e2204 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Tue, 15 Mar 2016 15:39:17 -0400 Subject: [PATCH 127/387] add STORM-971 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 48254bd594f..a959dbeefd6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-971: Metric for messages lost due to kafka retention * STORM-1608: Fix stateful topology acking behavior * STORM-1609: Netty Client is not best effort delivery on failed Connection * STORM-1620: Update curator to fix CURATOR-209 From a33bc455db2c4cce0d355c9c0fa4da60471b859b Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Tue, 15 Mar 2016 21:26:12 -0700 Subject: [PATCH 128/387] STORM-1632) Disable event logging by default --- conf/defaults.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 7128cfe9430..3f6a1c9e72e 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -222,7 +222,7 @@ topology.enable.message.timeouts: true topology.debug: false topology.workers: 1 topology.acker.executors: null -topology.eventlogger.executors: null +topology.eventlogger.executors: 0 topology.tasks: null # maximum amount of time a message has to complete before it's considered failed topology.message.timeout.secs: 30 From 45fe4595c64bb7772cafd3d4bd9923b2429e835b Mon Sep 17 00:00:00 2001 From: Xin Wang Date: Sun, 24 Jan 2016 22:05:23 +0800 Subject: [PATCH 129/387] STORM-1483: add storm-mongodb connector --- external/storm-mongodb/README.md | 195 ++++++++++++++++++ external/storm-mongodb/pom.xml | 74 +++++++ .../storm/mongodb/bolt/AbstractMongoBolt.java | 56 +++++ .../storm/mongodb/bolt/MongoInsertBolt.java | 62 ++++++ .../storm/mongodb/bolt/MongoUpdateBolt.java | 75 +++++++ .../storm/mongodb/common/MongoDBClient.java | 91 ++++++++ .../mongodb/common/QueryFilterCreator.java | 38 ++++ .../common/SimpleQueryFilterCreator.java | 39 ++++ .../mongodb/common/mapper/MongoMapper.java | 38 ++++ .../common/mapper/SimpleMongoMapper.java | 40 ++++ .../mapper/SimpleMongoUpdateMapper.java | 41 ++++ .../mongodb/trident/state/MongoState.java | 97 +++++++++ .../trident/state/MongoStateFactory.java | 42 ++++ .../trident/state/MongoStateUpdater.java | 34 +++ .../mongodb/topology/InsertWordCount.java | 81 ++++++++ .../mongodb/topology/UpdateWordCount.java | 91 ++++++++ .../storm/mongodb/topology/WordCounter.java | 67 ++++++ .../storm/mongodb/topology/WordSpout.java | 88 ++++++++ .../mongodb/trident/WordCountTrident.java | 85 ++++++++ pom.xml | 1 + .../binary/src/main/assembly/binary.xml | 14 ++ 21 files changed, 1349 insertions(+) create mode 100644 external/storm-mongodb/README.md create mode 100644 external/storm-mongodb/pom.xml create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/AbstractMongoBolt.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/MongoInsertBolt.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/MongoUpdateBolt.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/MongoDBClient.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/QueryFilterCreator.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/SimpleQueryFilterCreator.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/MongoMapper.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/SimpleMongoMapper.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/SimpleMongoUpdateMapper.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoState.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoStateFactory.java create mode 100644 external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoStateUpdater.java create mode 100644 external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/InsertWordCount.java create mode 100644 external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/UpdateWordCount.java create mode 100644 external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/WordCounter.java create mode 100644 external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/WordSpout.java create mode 100644 external/storm-mongodb/src/test/java/org/apache/storm/mongodb/trident/WordCountTrident.java diff --git a/external/storm-mongodb/README.md b/external/storm-mongodb/README.md new file mode 100644 index 00000000000..614b52f9784 --- /dev/null +++ b/external/storm-mongodb/README.md @@ -0,0 +1,195 @@ +#Storm MongoDB + +Storm/Trident integration for [MongoDB](https://www.mongodb.org/). This package includes the core bolts and trident states that allows a storm topology to either insert storm tuples in a database collection or to execute update queries against a database collection in a storm topology. + +## Insert into Database +The bolt and trident state included in this package for inserting data into a database collection. + +### MongoMapper +The main API for inserting data in a collection using MongoDB is the `org.apache.storm.mongodb.common.mapper.MongoMapper` interface: + +```java +public interface MongoMapper extends Serializable { + Document toDocument(ITuple tuple); +} +``` + +### SimpleMongoMapper +`storm-mongodb` includes a general purpose `MongoMapper` implementation called `SimpleMongoMapper` that can map Storm tuple to a Database document. `SimpleMongoMapper` assumes that the storm tuple has fields with same name as the document field name in the database collection that you intend to write to. + +```java +public class SimpleMongoMapper implements MongoMapper { + private String[] fields; + + @Override + public Document toDocument(ITuple tuple) { + Document document = new Document(); + for(String field : fields){ + document.append(field, tuple.getValueByField(field)); + } + return document; + } + + public SimpleMongoMapper withFields(String... fields) { + this.fields = fields; + return this; + } +} +``` + +### MongoInsertBolt +To use the `MongoInsertBolt`, you construct an instance of it by specifying url, collectionName and a `MongoMapper` implementation that converts storm tuple to DB document. The following is the standard URI connection scheme: + `mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]][/[database][?options]]` + +More options information(eg: Write Concern Options) about Mongo URI, you can visit https://docs.mongodb.org/manual/reference/connection-string/#connections-connection-options + + ```java +String url = "mongodb://127.0.0.1:27017/test"; +String collectionName = "wordcount"; + +MongoMapper mapper = new SimpleMongoMapper() + .withFields("word", "count"); + +MongoInsertBolt insertBolt = new MongoInsertBolt(url, collectionName, mapper); + ``` + +### MongoTridentState +We also support a trident persistent state that can be used with trident topologies. To create a Mongo persistent trident state you need to initialize it with the url, collectionName, the `MongoMapper` instance. See the example below: + + ```java + MongoMapper mapper = new SimpleMongoMapper() + .withFields("word", "count"); + + MongoState.Options options = new MongoState.Options() + .withUrl(url) + .withCollectionName(collectionName) + .withMapper(mapper); + + StateFactory factory = new MongoStateFactory(options); + + TridentTopology topology = new TridentTopology(); + Stream stream = topology.newStream("spout1", spout); + + stream.partitionPersist(factory, fields, new MongoStateUpdater(), new Fields()); + ``` + **NOTE**: + >If there is no unique index provided, trident state inserts in the case of failures may result in duplicate documents. + +## Update from Database +The bolt included in this package for updating data from a database collection. + +### SimpleMongoUpdateMapper +`storm-mongodb` includes a general purpose `MongoMapper` implementation called `SimpleMongoUpdateMapper` that can map Storm tuple to a Database document. `SimpleMongoUpdateMapper` assumes that the storm tuple has fields with same name as the document field name in the database collection that you intend to write to. +`SimpleMongoUpdateMapper` uses `$set` operator for setting the value of a field in a document. More information about update operator, you can visit +https://docs.mongodb.org/manual/reference/operator/update/ + +```java +public class SimpleMongoUpdateMapper implements MongoMapper { + private String[] fields; + + @Override + public Document toDocument(ITuple tuple) { + Document document = new Document(); + for(String field : fields){ + document.append(field, tuple.getValueByField(field)); + } + return new Document("$set", document); + } + + public SimpleMongoUpdateMapper withFields(String... fields) { + this.fields = fields; + return this; + } +} +``` + + + +### QueryFilterCreator +The main API for creating a MongoDB query Filter is the `org.apache.storm.mongodb.common.QueryFilterCreator` interface: + + ```java +public interface QueryFilterCreator extends Serializable { + Bson createFilter(ITuple tuple); +} + ``` + +### SimpleQueryFilterCreator +`storm-mongodb` includes a general purpose `QueryFilterCreator` implementation called `SimpleQueryFilterCreator` that can create a MongoDB query Filter by given Tuple. `QueryFilterCreator` uses `$eq` operator for matching values that are equal to a specified value. More information about query operator, you can visit +https://docs.mongodb.org/manual/reference/operator/query/ + + ```java +public class SimpleQueryFilterCreator implements QueryFilterCreator { + private String field; + + @Override + public Bson createFilter(ITuple tuple) { + return Filters.eq(field, tuple.getValueByField(field)); + } + + public SimpleQueryFilterCreator withField(String field) { + this.field = field; + return this; + } + +} + ``` + +### MongoUpdateBolt +To use the `MongoUpdateBolt`, you construct an instance of it by specifying Mongo url, collectionName, a `QueryFilterCreator` implementation and a `MongoMapper` implementation that converts storm tuple to DB document. + + ```java + MongoMapper mapper = new SimpleMongoUpdateMapper() + .withFields("word", "count"); + + QueryFilterCreator updateQueryCreator = new SimpleQueryFilterCreator() + .withField("word"); + + MongoUpdateBolt updateBolt = new MongoUpdateBolt(url, collectionName, updateQueryCreator, mapper); + + //if a new document should be inserted if there are no matches to the query filter + //updateBolt.withUpsert(true); + ``` + + Or use a anonymous inner class implementation for `QueryFilterCreator`: + + ```java + MongoMapper mapper = new SimpleMongoUpdateMapper() + .withFields("word", "count"); + + QueryFilterCreator updateQueryCreator = new QueryFilterCreator() { + @Override + public Bson createFilter(ITuple tuple) { + return Filters.gt("count", 3); + } + }; + + MongoUpdateBolt updateBolt = new MongoUpdateBolt(url, collectionName, updateQueryCreator, mapper); + + //if a new document should be inserted if there are no matches to the query filter + //updateBolt.withUpsert(true); + ``` + +## License + +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. + +## Committer Sponsors + + * Sriharsha Chintalapani ([sriharsha@apache.org](mailto:sriharsha@apache.org)) + diff --git a/external/storm-mongodb/pom.xml b/external/storm-mongodb/pom.xml new file mode 100644 index 00000000000..7653ac846f5 --- /dev/null +++ b/external/storm-mongodb/pom.xml @@ -0,0 +1,74 @@ + + + + 4.0.0 + + + storm + org.apache.storm + 2.0.0-SNAPSHOT + ../../pom.xml + + + storm-mongodb + + + + vesense + Xin Wang + data.xinwang@gmail.com + + + + + 3.2.0 + + + + + org.apache.storm + storm-core + ${project.version} + provided + + + org.mongodb + mongo-java-driver + ${mongodb.version} + + + com.google.guava + guava + + + commons-lang + commons-lang + + + + junit + junit + test + + + org.mockito + mockito-all + test + + + diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/AbstractMongoBolt.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/AbstractMongoBolt.java new file mode 100644 index 00000000000..f730ec7b56a --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/AbstractMongoBolt.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 org.apache.storm.mongodb.bolt; + +import java.util.Map; + +import org.apache.commons.lang.Validate; +import org.apache.storm.mongodb.common.MongoDBClient; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.base.BaseRichBolt; + +public abstract class AbstractMongoBolt extends BaseRichBolt { + + private String url; + private String collectionName; + + protected OutputCollector collector; + protected MongoDBClient mongoClient; + + public AbstractMongoBolt(String url, String collectionName) { + Validate.notEmpty(url, "url can not be blank or null"); + Validate.notEmpty(collectionName, "collectionName can not be blank or null"); + + this.url = url; + this.collectionName = collectionName; + } + + @Override + public void prepare(Map stormConf, TopologyContext context, + OutputCollector collector) { + this.collector = collector; + this.mongoClient = new MongoDBClient(url, collectionName); + } + + @Override + public void cleanup() { + this.mongoClient.close(); + } + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/MongoInsertBolt.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/MongoInsertBolt.java new file mode 100644 index 00000000000..26cd1507e16 --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/MongoInsertBolt.java @@ -0,0 +1,62 @@ +/** + * 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 org.apache.storm.mongodb.bolt; + +import org.apache.commons.lang.Validate; +import org.apache.storm.mongodb.common.mapper.MongoMapper; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Tuple; +import org.bson.Document; + +/** + * Basic bolt for writing to MongoDB. + * + * Note: Each MongoInsertBolt defined in a topology is tied to a specific collection. + * + */ +public class MongoInsertBolt extends AbstractMongoBolt { + + private MongoMapper mapper; + + public MongoInsertBolt(String url, String collectionName, MongoMapper mapper) { + super(url, collectionName); + + Validate.notNull(mapper, "MongoMapper can not be null"); + + this.mapper = mapper; + } + + @Override + public void execute(Tuple tuple) { + try{ + //get document + Document doc = mapper.toDocument(tuple); + mongoClient.insert(doc); + this.collector.ack(tuple); + } catch (Exception e) { + this.collector.reportError(e); + this.collector.fail(tuple); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + + } + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/MongoUpdateBolt.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/MongoUpdateBolt.java new file mode 100644 index 00000000000..1994993b620 --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/bolt/MongoUpdateBolt.java @@ -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. + */ +package org.apache.storm.mongodb.bolt; + +import org.apache.commons.lang.Validate; +import org.apache.storm.mongodb.common.QueryFilterCreator; +import org.apache.storm.mongodb.common.mapper.MongoMapper; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Tuple; +import org.bson.Document; +import org.bson.conversions.Bson; + +/** + * Basic bolt for updating from MongoDB. + * + * Note: Each MongoUpdateBolt defined in a topology is tied to a specific collection. + * + */ +public class MongoUpdateBolt extends AbstractMongoBolt { + + private QueryFilterCreator queryCreator; + private MongoMapper mapper; + + private boolean upsert; //The default is false. + + public MongoUpdateBolt(String url, String collectionName, QueryFilterCreator queryCreator, MongoMapper mapper) { + super(url, collectionName); + + Validate.notNull(queryCreator, "QueryFilterCreator can not be null"); + Validate.notNull(mapper, "MongoMapper can not be null"); + + this.queryCreator = queryCreator; + this.mapper = mapper; + } + + @Override + public void execute(Tuple tuple) { + try{ + //get document + Document doc = mapper.toDocument(tuple); + //get query filter + Bson filter = queryCreator.createFilter(tuple); + mongoClient.update(filter, doc, upsert); + this.collector.ack(tuple); + } catch (Exception e) { + this.collector.reportError(e); + this.collector.fail(tuple); + } + } + + public void withUpsert(boolean upsert) { + this.upsert = upsert; + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + + } + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/MongoDBClient.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/MongoDBClient.java new file mode 100644 index 00000000000..be2e3763ce6 --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/MongoDBClient.java @@ -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. + */ +package org.apache.storm.mongodb.common; + +import java.util.List; + +import org.bson.Document; +import org.bson.conversions.Bson; + +import com.mongodb.MongoClient; +import com.mongodb.MongoClientURI; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.UpdateOptions; + +public class MongoDBClient { + + private MongoClient client; + private MongoCollection collection; + + public MongoDBClient(String url, String collectionName) { + //Creates a MongoURI from the given string. + MongoClientURI uri = new MongoClientURI(url); + //Creates a MongoClient described by a URI. + this.client = new MongoClient(uri); + //Gets a Database. + MongoDatabase db = client.getDatabase(uri.getDatabase()); + //Gets a collection. + this.collection = db.getCollection(collectionName); + } + + /** + * Inserts the provided document. + * + * @param document + */ + public void insert(Document document) { + collection.insertOne(document); + } + + /** + * Inserts one or more documents. + * This method is equivalent to a call to the bulkWrite method. + * The documents will be inserted in the order provided, + * stopping on the first failed insertion. + * + * @param documents + */ + public void insert(List documents) { + collection.insertMany(documents); + } + + /** + * Update all documents in the collection according to the specified query filter. + * When upsert set to true, the new document will be inserted if there are no matches to the query filter. + * + * @param filter + * @param update + * @param upsert + */ + public void update(Bson filter, Bson update, boolean upsert) { + UpdateOptions options = new UpdateOptions(); + if(upsert) { + options.upsert(true); + } + collection.updateMany(filter, update, options); + } + + /** + * Closes all resources associated with this instance. + */ + public void close(){ + client.close(); + } + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/QueryFilterCreator.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/QueryFilterCreator.java new file mode 100644 index 00000000000..d95f7176342 --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/QueryFilterCreator.java @@ -0,0 +1,38 @@ +/** + * 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 org.apache.storm.mongodb.common; + +import java.io.Serializable; + +import org.apache.storm.tuple.ITuple; +import org.bson.conversions.Bson; + +/** + * Create a MongoDB query Filter by given Tuple. + */ +public interface QueryFilterCreator extends Serializable { + + /** + * Create a query Filter by given Tuple + * + * @param tuple + * @return query Filter + */ + Bson createFilter(ITuple tuple); + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/SimpleQueryFilterCreator.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/SimpleQueryFilterCreator.java new file mode 100644 index 00000000000..8b4f1c31a2d --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/SimpleQueryFilterCreator.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 org.apache.storm.mongodb.common; + +import org.apache.storm.tuple.ITuple; +import org.bson.conversions.Bson; + +import com.mongodb.client.model.Filters; + +public class SimpleQueryFilterCreator implements QueryFilterCreator { + + private String field; + + @Override + public Bson createFilter(ITuple tuple) { + return Filters.eq(field, tuple.getValueByField(field)); + } + + public SimpleQueryFilterCreator withField(String field) { + this.field = field; + return this; + } + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/MongoMapper.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/MongoMapper.java new file mode 100644 index 00000000000..7bcd499bf2b --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/MongoMapper.java @@ -0,0 +1,38 @@ +/** + * 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 org.apache.storm.mongodb.common.mapper; + +import java.io.Serializable; + +import org.apache.storm.tuple.ITuple; +import org.bson.Document; + +/** + * Given a Tuple, converts it to an MongoDB document. + */ +public interface MongoMapper extends Serializable { + + /** + * Converts a Tuple to a Document + * + * @param tuple the incoming tuple + * @return the MongoDB document + */ + Document toDocument(ITuple tuple); + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/SimpleMongoMapper.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/SimpleMongoMapper.java new file mode 100644 index 00000000000..444096222b7 --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/SimpleMongoMapper.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 org.apache.storm.mongodb.common.mapper; + +import org.apache.storm.tuple.ITuple; +import org.bson.Document; + +public class SimpleMongoMapper implements MongoMapper { + + private String[] fields; + + @Override + public Document toDocument(ITuple tuple) { + Document document = new Document(); + for(String field : fields){ + document.append(field, tuple.getValueByField(field)); + } + return document; + } + + public SimpleMongoMapper withFields(String... fields) { + this.fields = fields; + return this; + } +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/SimpleMongoUpdateMapper.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/SimpleMongoUpdateMapper.java new file mode 100644 index 00000000000..f07d4dc7d89 --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/common/mapper/SimpleMongoUpdateMapper.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 org.apache.storm.mongodb.common.mapper; + +import org.apache.storm.tuple.ITuple; +import org.bson.Document; + +public class SimpleMongoUpdateMapper implements MongoMapper { + + private String[] fields; + + @Override + public Document toDocument(ITuple tuple) { + Document document = new Document(); + for(String field : fields){ + document.append(field, tuple.getValueByField(field)); + } + //$set operator: Sets the value of a field in a document. + return new Document("$set", document); + } + + public SimpleMongoUpdateMapper withFields(String... fields) { + this.fields = fields; + return this; + } +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoState.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoState.java new file mode 100644 index 00000000000..843fceee0ee --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoState.java @@ -0,0 +1,97 @@ +/** + * 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 org.apache.storm.mongodb.trident.state; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.Validate; +import org.apache.storm.mongodb.common.MongoDBClient; +import org.apache.storm.mongodb.common.mapper.MongoMapper; +import org.apache.storm.trident.operation.TridentCollector; +import org.apache.storm.trident.state.State; +import org.apache.storm.trident.tuple.TridentTuple; +import org.bson.Document; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; + +public class MongoState implements State { + + private static final Logger LOG = LoggerFactory.getLogger(MongoState.class); + + private Options options; + private MongoDBClient mongoClient; + private Map map; + + protected MongoState(Map map, Options options) { + this.options = options; + this.map = map; + } + + public static class Options implements Serializable { + private String url; + private String collectionName; + private MongoMapper mapper; + + public Options withUrl(String url) { + this.url = url; + return this; + } + + public Options withCollectionName(String collectionName) { + this.collectionName = collectionName; + return this; + } + + public Options withMapper(MongoMapper mapper) { + this.mapper = mapper; + return this; + } + } + + protected void prepare() { + Validate.notEmpty(options.url, "url can not be blank or null"); + Validate.notEmpty(options.collectionName, "collectionName can not be blank or null"); + Validate.notNull(options.mapper, "MongoMapper can not be null"); + + this.mongoClient = new MongoDBClient(options.url, options.collectionName); + } + + @Override + public void beginCommit(Long txid) { + LOG.debug("beginCommit is noop."); + } + + @Override + public void commit(Long txid) { + LOG.debug("commit is noop."); + } + + public void updateState(List tuples, TridentCollector collector) { + List documents = Lists.newArrayList(); + for (TridentTuple tuple : tuples) { + Document document = options.mapper.toDocument(tuple); + documents.add(document); + } + this.mongoClient.insert(documents); + } + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoStateFactory.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoStateFactory.java new file mode 100644 index 00000000000..d6cd3a5ce1d --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoStateFactory.java @@ -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. + */ +package org.apache.storm.mongodb.trident.state; + +import java.util.Map; + +import org.apache.storm.task.IMetricsContext; +import org.apache.storm.trident.state.State; +import org.apache.storm.trident.state.StateFactory; + +public class MongoStateFactory implements StateFactory { + + private MongoState.Options options; + + public MongoStateFactory(MongoState.Options options) { + this.options = options; + } + + @Override + public State makeState(Map conf, IMetricsContext metrics, + int partitionIndex, int numPartitions) { + MongoState state = new MongoState(conf, options); + state.prepare(); + return state; + } + +} diff --git a/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoStateUpdater.java b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoStateUpdater.java new file mode 100644 index 00000000000..3173f6c1184 --- /dev/null +++ b/external/storm-mongodb/src/main/java/org/apache/storm/mongodb/trident/state/MongoStateUpdater.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 org.apache.storm.mongodb.trident.state; + +import java.util.List; + +import org.apache.storm.trident.operation.TridentCollector; +import org.apache.storm.trident.state.BaseStateUpdater; +import org.apache.storm.trident.tuple.TridentTuple; + +public class MongoStateUpdater extends BaseStateUpdater { + + @Override + public void updateState(MongoState state, List tuples, + TridentCollector collector) { + state.updateState(tuples, collector); + } + +} diff --git a/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/InsertWordCount.java b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/InsertWordCount.java new file mode 100644 index 00000000000..c83bdbde56d --- /dev/null +++ b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/InsertWordCount.java @@ -0,0 +1,81 @@ +/** + * 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 org.apache.storm.mongodb.topology; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; +import org.apache.storm.mongodb.bolt.MongoInsertBolt; +import org.apache.storm.mongodb.common.mapper.MongoMapper; +import org.apache.storm.mongodb.common.mapper.SimpleMongoMapper; + +import java.util.HashMap; +import java.util.Map; + +public class InsertWordCount { + private static final String WORD_SPOUT = "WORD_SPOUT"; + private static final String COUNT_BOLT = "COUNT_BOLT"; + private static final String INSERT_BOLT = "INSERT_BOLT"; + + private static final String TEST_MONGODB_URL = "mongodb://127.0.0.1:27017/test"; + private static final String TEST_MONGODB_COLLECTION_NAME = "wordcount"; + + + public static void main(String[] args) throws Exception { + Config config = new Config(); + + String url = TEST_MONGODB_URL; + String collectionName = TEST_MONGODB_COLLECTION_NAME; + + if (args.length >= 2) { + url = args[0]; + collectionName = args[1]; + } + + WordSpout spout = new WordSpout(); + WordCounter bolt = new WordCounter(); + + MongoMapper mapper = new SimpleMongoMapper() + .withFields("word", "count"); + + MongoInsertBolt insertBolt = new MongoInsertBolt(url, collectionName, mapper); + + // wordSpout ==> countBolt ==> MongoInsertBolt + TopologyBuilder builder = new TopologyBuilder(); + + builder.setSpout(WORD_SPOUT, spout, 1); + builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT); + builder.setBolt(INSERT_BOLT, insertBolt, 1).fieldsGrouping(COUNT_BOLT, new Fields("word")); + + + if (args.length == 2) { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.createTopology()); + Thread.sleep(30000); + cluster.killTopology("test"); + cluster.shutdown(); + System.exit(0); + } else if (args.length == 3) { + StormSubmitter.submitTopology(args[2], config, builder.createTopology()); + } else{ + System.out.println("Usage: InsertWordCount [topology name]"); + } + } +} diff --git a/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/UpdateWordCount.java b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/UpdateWordCount.java new file mode 100644 index 00000000000..071708e6b37 --- /dev/null +++ b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/UpdateWordCount.java @@ -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. + */ +package org.apache.storm.mongodb.topology; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; +import org.apache.storm.mongodb.bolt.MongoInsertBolt; +import org.apache.storm.mongodb.bolt.MongoUpdateBolt; +import org.apache.storm.mongodb.common.QueryFilterCreator; +import org.apache.storm.mongodb.common.SimpleQueryFilterCreator; +import org.apache.storm.mongodb.common.mapper.MongoMapper; +import org.apache.storm.mongodb.common.mapper.SimpleMongoMapper; +import org.apache.storm.mongodb.common.mapper.SimpleMongoUpdateMapper; + +import java.util.HashMap; +import java.util.Map; + +public class UpdateWordCount { + private static final String WORD_SPOUT = "WORD_SPOUT"; + private static final String COUNT_BOLT = "COUNT_BOLT"; + private static final String UPDATE_BOLT = "UPDATE_BOLT"; + + private static final String TEST_MONGODB_URL = "mongodb://127.0.0.1:27017/test"; + private static final String TEST_MONGODB_COLLECTION_NAME = "wordcount"; + + + public static void main(String[] args) throws Exception { + Config config = new Config(); + + String url = TEST_MONGODB_URL; + String collectionName = TEST_MONGODB_COLLECTION_NAME; + + if (args.length >= 2) { + url = args[0]; + collectionName = args[1]; + } + + WordSpout spout = new WordSpout(); + WordCounter bolt = new WordCounter(); + + MongoMapper mapper = new SimpleMongoUpdateMapper() + .withFields("word", "count"); + + QueryFilterCreator updateQueryCreator = new SimpleQueryFilterCreator() + .withField("word"); + + MongoUpdateBolt updateBolt = new MongoUpdateBolt(url, collectionName, updateQueryCreator , mapper); + + //if a new document should be inserted if there are no matches to the query filter + //updateBolt.withUpsert(true); + + // wordSpout ==> countBolt ==> MongoUpdateBolt + TopologyBuilder builder = new TopologyBuilder(); + + builder.setSpout(WORD_SPOUT, spout, 1); + builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT); + builder.setBolt(UPDATE_BOLT, updateBolt, 1).fieldsGrouping(COUNT_BOLT, new Fields("word")); + + + if (args.length == 2) { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.createTopology()); + Thread.sleep(30000); + cluster.killTopology("test"); + cluster.shutdown(); + System.exit(0); + } else if (args.length == 3) { + StormSubmitter.submitTopology(args[2], config, builder.createTopology()); + } else{ + System.out.println("Usage: UpdateWordCount [topology name]"); + } + } +} diff --git a/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/WordCounter.java b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/WordCounter.java new file mode 100644 index 00000000000..481f959fb51 --- /dev/null +++ b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/WordCounter.java @@ -0,0 +1,67 @@ +/** + * 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 org.apache.storm.mongodb.topology; + +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.BasicOutputCollector; +import org.apache.storm.topology.IBasicBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import com.google.common.collect.Maps; + +import java.util.Map; + +import static org.apache.storm.utils.Utils.tuple; + +public class WordCounter implements IBasicBolt { + private Map wordCounter = Maps.newHashMap(); + + public void prepare(Map stormConf, TopologyContext context) { + + } + + public void execute(Tuple input, BasicOutputCollector collector) { + String word = input.getStringByField("word"); + int count; + if (wordCounter.containsKey(word)) { + count = wordCounter.get(word) + 1; + wordCounter.put(word, wordCounter.get(word) + 1); + } else { + count = 1; + } + + wordCounter.put(word, count); + collector.emit(new Values(word, String.valueOf(count))); + } + + public void cleanup() { + + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word", "count")); + } + + @Override + public Map getComponentConfiguration() { + return null; + } + +} diff --git a/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/WordSpout.java b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/WordSpout.java new file mode 100644 index 00000000000..284f2284192 --- /dev/null +++ b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/topology/WordSpout.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 org.apache.storm.mongodb.topology; + +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; + +import java.util.Map; +import java.util.Random; +import java.util.UUID; + +public class WordSpout implements IRichSpout { + boolean isDistributed; + SpoutOutputCollector collector; + public static final String[] words = new String[] { "apple", "orange", "pineapple", "banana", "watermelon" }; + + public WordSpout() { + this(true); + } + + public WordSpout(boolean isDistributed) { + this.isDistributed = isDistributed; + } + + public boolean isDistributed() { + return this.isDistributed; + } + + @SuppressWarnings("rawtypes") + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + this.collector = collector; + } + + public void close() { + + } + + public void nextTuple() { + final Random rand = new Random(); + final String word = words[rand.nextInt(words.length)]; + this.collector.emit(new Values(word), UUID.randomUUID()); + Thread.yield(); + } + + public void ack(Object msgId) { + + } + + public void fail(Object msgId) { + + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + + @Override + public void activate() { + } + + @Override + public void deactivate() { + } + + @Override + public Map getComponentConfiguration() { + return null; + } +} diff --git a/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/trident/WordCountTrident.java b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/trident/WordCountTrident.java new file mode 100644 index 00000000000..7a1886314a3 --- /dev/null +++ b/external/storm-mongodb/src/test/java/org/apache/storm/mongodb/trident/WordCountTrident.java @@ -0,0 +1,85 @@ +/** + * 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 org.apache.storm.mongodb.trident; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.mongodb.common.mapper.MongoMapper; +import org.apache.storm.mongodb.common.mapper.SimpleMongoMapper; +import org.apache.storm.mongodb.trident.state.MongoState; +import org.apache.storm.mongodb.trident.state.MongoStateFactory; +import org.apache.storm.mongodb.trident.state.MongoStateUpdater; +import org.apache.storm.trident.Stream; +import org.apache.storm.trident.TridentState; +import org.apache.storm.trident.TridentTopology; +import org.apache.storm.trident.state.StateFactory; +import org.apache.storm.trident.testing.FixedBatchSpout; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; + +public class WordCountTrident { + + public static StormTopology buildTopology(String url, String collectionName){ + Fields fields = new Fields("word", "count"); + FixedBatchSpout spout = new FixedBatchSpout(fields, 4, + new Values("storm", 1), + new Values("trident", 1), + new Values("needs", 1), + new Values("javadoc", 1) + ); + spout.setCycle(true); + + MongoMapper mapper = new SimpleMongoMapper() + .withFields("word", "count"); + + MongoState.Options options = new MongoState.Options() + .withUrl(url) + .withCollectionName(collectionName) + .withMapper(mapper); + + StateFactory factory = new MongoStateFactory(options); + + TridentTopology topology = new TridentTopology(); + Stream stream = topology.newStream("spout1", spout); + + stream.partitionPersist(factory, fields, new MongoStateUpdater(), new Fields()); + return topology.build(); + } + + public static void main(String[] args) throws Exception { + Config conf = new Config(); + conf.setMaxSpoutPending(5); + if (args.length == 2) { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("wordCounter", conf, buildTopology(args[0], args[1])); + Thread.sleep(60 * 1000); + cluster.killTopology("wordCounter"); + cluster.shutdown(); + System.exit(0); + } + else if(args.length == 3) { + conf.setNumWorkers(3); + StormSubmitter.submitTopology(args[2], conf, buildTopology(args[0], args[1])); + } else{ + System.out.println("Usage: WordCountTrident [topology name]"); + } + } + +} diff --git a/pom.xml b/pom.xml index facd824fd0e..213e5abcff4 100644 --- a/pom.xml +++ b/pom.xml @@ -277,6 +277,7 @@ external/storm-metrics external/storm-cassandra external/storm-mqtt + external/storm-mongodb examples/storm-starter diff --git a/storm-dist/binary/src/main/assembly/binary.xml b/storm-dist/binary/src/main/assembly/binary.xml index 6d40c19cb85..933228397b7 100644 --- a/storm-dist/binary/src/main/assembly/binary.xml +++ b/storm-dist/binary/src/main/assembly/binary.xml @@ -303,6 +303,20 @@ storm*jar + + ${project.basedir}/../../external/storm-mongodb/target + external/storm-mongodb + + storm*jar + + + + ${project.basedir}/../../external/storm-mongodb + external/storm-mongodb + + README.* + + From 53e1ab0c6efd79627bf411e153caac01efe281f0 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 16 Mar 2016 09:35:56 -0500 Subject: [PATCH 130/387] Added STORM-1549 to Changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index a959dbeefd6..5562ee115d1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1549: Add support for resetting tuple timeout from bolts via the OutputCollector * STORM-971: Metric for messages lost due to kafka retention * STORM-1608: Fix stateful topology acking behavior * STORM-1609: Netty Client is not best effort delivery on failed Connection From eeeb7b9c39167d3065f198aad8e91dfbb1b631f4 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Wed, 16 Mar 2016 17:51:14 -0700 Subject: [PATCH 131/387] Added STORM-1483 to CHANGLELOG. --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5562ee115d1..f7653157f72 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1483: add storm-mongodb connector * STORM-1549: Add support for resetting tuple timeout from bolts via the OutputCollector * STORM-971: Metric for messages lost due to kafka retention * STORM-1608: Fix stateful topology acking behavior From 8965a6c2459841c607fb9587897502581f1e3cdf Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 17 Mar 2016 15:53:15 +0900 Subject: [PATCH 132/387] STORM-1602 Blobstore UTs are failed on Windows * ensures objects of InputStream / OutputStream are closed after using * clojure: with-open * java: try-with-resource * skip checking symbolic link in LocalizerTest when on Windows * Windows seems not handle symbolic link in compressed file properly --- .../org/apache/storm/daemon/supervisor.clj | 6 +- .../org/apache/storm/blobstore/BlobStore.java | 5 + .../apache/storm/blobstore/BlobStoreTest.java | 171 +++++++++--------- .../apache/storm/localizer/LocalizerTest.java | 14 +- 4 files changed, 105 insertions(+), 91 deletions(-) diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj index 1287d775039..dc47988f4fb 100644 --- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj @@ -1168,8 +1168,10 @@ blob-store (Utils/getNimbusBlobStore conf master-code-dir nil)] (try (FileUtils/forceMkdir (File. tmproot)) - (.readBlobTo blob-store (master-stormcode-key storm-id) (FileOutputStream. (supervisor-stormcode-path tmproot)) nil) - (.readBlobTo blob-store (master-stormconf-key storm-id) (FileOutputStream. (supervisor-stormconf-path tmproot)) nil) + (with-open [fos-storm-code (FileOutputStream. (supervisor-stormcode-path tmproot)) + fos-storm-conf (FileOutputStream. (supervisor-stormconf-path tmproot))] + (.readBlobTo blob-store (master-stormcode-key storm-id) fos-storm-code nil) + (.readBlobTo blob-store (master-stormconf-key storm-id) fos-storm-conf nil)) (finally (.shutdown blob-store))) (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) diff --git a/storm-core/src/jvm/org/apache/storm/blobstore/BlobStore.java b/storm-core/src/jvm/org/apache/storm/blobstore/BlobStore.java index 09093a25f2a..14879b4f556 100644 --- a/storm-core/src/jvm/org/apache/storm/blobstore/BlobStore.java +++ b/storm-core/src/jvm/org/apache/storm/blobstore/BlobStore.java @@ -396,6 +396,11 @@ public int available() throws IOException { public long getFileLength() throws IOException { return part.getFileLength(); } + + @Override + public void close() throws IOException { + in.close(); + } } /** diff --git a/storm-core/test/jvm/org/apache/storm/blobstore/BlobStoreTest.java b/storm-core/test/jvm/org/apache/storm/blobstore/BlobStoreTest.java index 712537a825c..5f6f50aaa4f 100644 --- a/storm-core/test/jvm/org/apache/storm/blobstore/BlobStoreTest.java +++ b/storm-core/test/jvm/org/apache/storm/blobstore/BlobStoreTest.java @@ -181,30 +181,30 @@ public void testWithAuthentication(BlobStore store) throws Exception { Subject admin = getSubject("admin"); assertStoreHasExactly(store); SettableBlobMeta metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT); - AtomicOutputStream out = store.createBlob("test", metadata, admin); - assertStoreHasExactly(store, "test"); - out.write(1); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", metadata, admin)) { + assertStoreHasExactly(store, "test"); + out.write(1); + } store.deleteBlob("test", admin); //Test for Supervisor Admin Subject supervisor = getSubject("supervisor"); assertStoreHasExactly(store); metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT); - out = store.createBlob("test", metadata, supervisor); - assertStoreHasExactly(store, "test"); - out.write(1); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", metadata, supervisor)) { + assertStoreHasExactly(store, "test"); + out.write(1); + } store.deleteBlob("test", supervisor); //Test for Nimbus itself as a user Subject nimbus = getNimbusSubject(); assertStoreHasExactly(store); metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT); - out = store.createBlob("test", metadata, nimbus); - assertStoreHasExactly(store, "test"); - out.write(1); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", metadata, nimbus)) { + assertStoreHasExactly(store, "test"); + out.write(1); + } store.deleteBlob("test", nimbus); // Test with a dummy test_subject for cases where subject !=null (security turned on) @@ -214,9 +214,9 @@ public void testWithAuthentication(BlobStore store) throws Exception { // Tests for case when subject != null (security turned on) and // acls for the blob are set to WORLD_EVERYTHING metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING); - out = store.createBlob("test", metadata, who); - out.write(1); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", metadata, who)) { + out.write(1); + } assertStoreHasExactly(store, "test"); // Testing whether acls are set to WORLD_EVERYTHING assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)")); @@ -230,9 +230,9 @@ public void testWithAuthentication(BlobStore store) throws Exception { // acls are not set for the blob (DEFAULT) LOG.info("Creating test again"); metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT); - out = store.createBlob("test", metadata, who); - out.write(2); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", metadata, who)) { + out.write(2); + } assertStoreHasExactly(store, "test"); // Testing whether acls are set to WORLD_EVERYTHING. Here the acl should not contain WORLD_EVERYTHING because // the subject is neither null nor empty. The ACL should however contain USER_EVERYTHING as user needs to have @@ -241,28 +241,29 @@ public void testWithAuthentication(BlobStore store) throws Exception { readAssertEqualsWithAuth(store, who, "test", 2); LOG.info("Updating test"); - out = store.updateBlob("test", who); - out.write(3); - out.close(); + try (AtomicOutputStream out = store.updateBlob("test", who)) { + out.write(3); + } assertStoreHasExactly(store, "test"); readAssertEqualsWithAuth(store, who, "test", 3); LOG.info("Updating test again"); - out = store.updateBlob("test", who); - out.write(4); - out.flush(); - LOG.info("SLEEPING"); - Thread.sleep(2); - assertStoreHasExactly(store, "test"); - readAssertEqualsWithAuth(store, who, "test", 3); + try (AtomicOutputStream out = store.updateBlob("test", who)) { + out.write(4); + out.flush(); + LOG.info("SLEEPING"); + Thread.sleep(2); + assertStoreHasExactly(store, "test"); + readAssertEqualsWithAuth(store, who, "test", 3); + } // Test for subject with no principals and acls set to WORLD_EVERYTHING who = new Subject(); metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING); LOG.info("Creating test"); - out = store.createBlob("test-empty-subject-WE", metadata, who); - out.write(2); - out.close(); + try (AtomicOutputStream out = store.createBlob("test-empty-subject-WE", metadata, who)) { + out.write(2); + } assertStoreHasExactly(store, "test-empty-subject-WE", "test"); // Testing whether acls are set to WORLD_EVERYTHING assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)")); @@ -272,9 +273,10 @@ public void testWithAuthentication(BlobStore store) throws Exception { who = new Subject(); metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT); LOG.info("Creating other"); - out = store.createBlob("test-empty-subject-DEF", metadata, who); - out.write(2); - out.close(); + + try (AtomicOutputStream out = store.createBlob("test-empty-subject-DEF", metadata, who)) { + out.write(2); + } assertStoreHasExactly(store, "test-empty-subject-DEF", "test", "test-empty-subject-WE"); // Testing whether acls are set to WORLD_EVERYTHING assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)")); @@ -285,12 +287,6 @@ public void testWithAuthentication(BlobStore store) throws Exception { } else { fail("Error the blobstore is of unknowntype"); } - try { - out.close(); - } catch (IOException e) { - // This is likely to happen when we try to commit something that - // was cleaned up. This is expected and acceptable. - } } public void testBasic(BlobStore store) throws Exception { @@ -300,9 +296,9 @@ public void testBasic(BlobStore store) throws Exception { // acls for the blob are set to WORLD_EVERYTHING SettableBlobMeta metadata = new SettableBlobMeta(BlobStoreAclHandler .WORLD_EVERYTHING); - AtomicOutputStream out = store.createBlob("test", metadata, null); - out.write(1); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", metadata, null)) { + out.write(1); + } assertStoreHasExactly(store, "test"); // Testing whether acls are set to WORLD_EVERYTHING assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)")); @@ -316,37 +312,38 @@ public void testBasic(BlobStore store) throws Exception { // update blob interface metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING); LOG.info("Creating test again"); - out = store.createBlob("test", metadata, null); - out.write(2); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", metadata, null)) { + out.write(2); + } assertStoreHasExactly(store, "test"); if (store instanceof LocalFsBlobStore) { assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)")); } readAssertEquals(store, "test", 2); LOG.info("Updating test"); - out = store.updateBlob("test", null); - out.write(3); - out.close(); + try (AtomicOutputStream out = store.updateBlob("test", null)) { + out.write(3); + } assertStoreHasExactly(store, "test"); readAssertEquals(store, "test", 3); LOG.info("Updating test again"); - out = store.updateBlob("test", null); - out.write(4); - out.flush(); - LOG.info("SLEEPING"); - Thread.sleep(2); + try (AtomicOutputStream out = store.updateBlob("test", null)) { + out.write(4); + out.flush(); + LOG.info("SLEEPING"); + Thread.sleep(2); + } // Tests for case when subject == null (security turned off) and // acls for the blob are set to DEFAULT (Empty ACL List) only for LocalFsBlobstore if (store instanceof LocalFsBlobStore) { metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT); LOG.info("Creating test for empty acls when security is off"); - out = store.createBlob("test-empty-acls", metadata, null); - LOG.info("metadata {}", metadata); - out.write(2); - out.close(); + try (AtomicOutputStream out = store.createBlob("test-empty-acls", metadata, null)) { + LOG.info("metadata {}", metadata); + out.write(2); + } assertStoreHasExactly(store, "test-empty-acls", "test"); // Testing whether acls are set to WORLD_EVERYTHING, Here we are testing only for LocalFsBlobstore // as the HdfsBlobstore gets the subject information of the local system user and behaves as it is @@ -362,12 +359,6 @@ public void testBasic(BlobStore store) throws Exception { } else { fail("Error the blobstore is of unknowntype"); } - try { - out.close(); - } catch (IOException e) { - // This is likely to happen when we try to commit something that - // was cleaned up. This is expected and acceptable. - } } @@ -375,26 +366,26 @@ public void testMultiple(BlobStore store) throws Exception { assertStoreHasExactly(store); LOG.info("Creating test"); - AtomicOutputStream out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler - .WORLD_EVERYTHING), null); - out.write(1); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler + .WORLD_EVERYTHING), null)) { + out.write(1); + } assertStoreHasExactly(store, "test"); readAssertEquals(store, "test", 1); LOG.info("Creating other"); - out = store.createBlob("other", new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING), - null); - out.write(2); - out.close(); + try (AtomicOutputStream out = store.createBlob("other", new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING), + null)) { + out.write(2); + } assertStoreHasExactly(store, "test", "other"); readAssertEquals(store, "test", 1); readAssertEquals(store, "other", 2); LOG.info("Updating other"); - out = store.updateBlob("other", null); - out.write(5); - out.close(); + try (AtomicOutputStream out = store.updateBlob("other", null)) { + out.write(5); + } assertStoreHasExactly(store, "test", "other"); readAssertEquals(store, "test", 1); readAssertEquals(store, "other", 5); @@ -405,18 +396,18 @@ public void testMultiple(BlobStore store) throws Exception { readAssertEquals(store, "other", 5); LOG.info("Creating test again"); - out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING), - null); - out.write(2); - out.close(); + try (AtomicOutputStream out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING), + null)) { + out.write(2); + } assertStoreHasExactly(store, "test", "other"); readAssertEquals(store, "test", 2); readAssertEquals(store, "other", 5); LOG.info("Updating test"); - out = store.updateBlob("test", null); - out.write(3); - out.close(); + try (AtomicOutputStream out = store.updateBlob("test", null)) { + out.write(3); + } assertStoreHasExactly(store, "test", "other"); readAssertEquals(store, "test", 3); readAssertEquals(store, "other", 5); @@ -427,7 +418,9 @@ public void testMultiple(BlobStore store) throws Exception { readAssertEquals(store, "test", 3); LOG.info("Updating test again"); - out = store.updateBlob("test", null); + + // intended to not guarding with try-with-resource since otherwise test will fail + AtomicOutputStream out = store.updateBlob("test", null); out.write(4); out.flush(); LOG.info("SLEEPING"); @@ -451,10 +444,12 @@ public void testMultiple(BlobStore store) throws Exception { public void testGetFileLength() throws AuthorizationException, KeyNotFoundException, KeyAlreadyExistsException, IOException { LocalFsBlobStore store = initLocalFs(); - AtomicOutputStream out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler - .WORLD_EVERYTHING), null); - out.write(1); - out.close(); - assertEquals(1, store.getBlob("test", null).getFileLength()); + try (AtomicOutputStream out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler + .WORLD_EVERYTHING), null)) { + out.write(1); + } + try (InputStreamWithMeta blobInputStream = store.getBlob("test", null)) { + assertEquals(1, blobInputStream.getFileLength()); + } } } diff --git a/storm-core/test/jvm/org/apache/storm/localizer/LocalizerTest.java b/storm-core/test/jvm/org/apache/storm/localizer/LocalizerTest.java index 096c4b07ff5..45ba1085b50 100644 --- a/storm-core/test/jvm/org/apache/storm/localizer/LocalizerTest.java +++ b/storm-core/test/jvm/org/apache/storm/localizer/LocalizerTest.java @@ -110,7 +110,7 @@ public long getFileLength() { @Before public void setUp() throws Exception { - baseDir = new File("/tmp/blob-store-localizer-test-"+ UUID.randomUUID()); + baseDir = new File(System.getProperty("java.io.tmpdir") + "/blob-store-localizer-test-"+ UUID.randomUUID()); if (!baseDir.mkdir()) { throw new IOException("failed to create base directory"); } @@ -259,6 +259,11 @@ public void testArchivesJar() throws Exception { // archive passed in must contain symlink named tmptestsymlink if not a zip file public void testArchives(String archivePath, boolean supportSymlinks, int size) throws Exception { + if (isOnWindows()) { + // Windows should set this to false cause symlink in compressed file doesn't work properly. + supportSymlinks = false; + } + Map conf = new HashMap(); // set clean time really high so doesn't kick in conf.put(Config.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000); @@ -664,4 +669,11 @@ public void testUpdate() throws Exception { assertEquals("blob version not correct", 3, Utils.localVersionOfBlob(keyFile.toString())); assertTrue("blob file with version 3 not created", new File(keyFile + ".3").exists()); } + + private boolean isOnWindows() { + if (System.getenv("OS") != null) { + return System.getenv("OS").equals("Windows_NT"); + } + return false; + } } From 4fe225cd07b6905f7024969751271425059316d6 Mon Sep 17 00:00:00 2001 From: abhishekagarwal87 Date: Thu, 17 Mar 2016 20:03:50 +0530 Subject: [PATCH 133/387] STORM-1637: Correct the master version of storm in storm-mongodb module --- external/storm-mongodb/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/storm-mongodb/pom.xml b/external/storm-mongodb/pom.xml index 7653ac846f5..ce9f1bb8632 100644 --- a/external/storm-mongodb/pom.xml +++ b/external/storm-mongodb/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml From 00280de7321f762b175005cc964c688d3f256d0b Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 17 Mar 2016 15:32:29 -0400 Subject: [PATCH 134/387] add STORM-1614 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index a959dbeefd6..6697ed0170d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1614: backpressure init and cleanup changes * STORM-971: Metric for messages lost due to kafka retention * STORM-1608: Fix stateful topology acking behavior * STORM-1609: Netty Client is not best effort delivery on failed Connection From d0d53be94a967a07ed767ec1c79738649dd70722 Mon Sep 17 00:00:00 2001 From: Kyle Nusbaum Date: Tue, 8 Mar 2016 15:46:23 -0600 Subject: [PATCH 135/387] Initial changes. Signed-off-by: Kyle Nusbaum Ready for PR Signed-off-by: Kyle Nusbaum Addressing comments. Signed-off-by: Kyle Nusbaum Addressing Comments. Signed-off-by: Kyle Nusbaum adding code documentation explaining math of combining component resources. Signed-off-by: Kyle Nusbaum Addressing comments and adding a bit more documentation. Signed-off-by: Kyle Nusbaum --- .../clj/org/apache/storm/trident/testing.clj | 12 +- .../ComponentConfigurationDeclarer.java | 5 +- .../storm/topology/ResourceDeclarer.java | 28 +++++ .../jvm/org/apache/storm/trident/Stream.java | 31 ++++- .../apache/storm/trident/TridentState.java | 27 ++++- .../apache/storm/trident/TridentTopology.java | 91 +++++++++++++- .../org/apache/storm/trident/graph/Group.java | 22 +++- .../operation/DefaultResourceDeclarer.java | 66 +++++++++++ .../trident/operation/ITridentResource.java | 32 +++++ .../apache/storm/trident/planner/Node.java | 5 +- .../apache/storm/trident/integration_test.clj | 111 +++++++++++++++--- 11 files changed, 390 insertions(+), 40 deletions(-) create mode 100644 storm-core/src/jvm/org/apache/storm/topology/ResourceDeclarer.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/DefaultResourceDeclarer.java create mode 100644 storm-core/src/jvm/org/apache/storm/trident/operation/ITridentResource.java diff --git a/storm-core/src/clj/org/apache/storm/trident/testing.clj b/storm-core/src/clj/org/apache/storm/trident/testing.clj index 44e5ca9e2d2..aafb001481b 100644 --- a/storm-core/src/clj/org/apache/storm/trident/testing.clj +++ b/storm-core/src/clj/org/apache/storm/trident/testing.clj @@ -55,14 +55,14 @@ (.shutdown ~drpc) )) -(defn with-topology* [cluster topo body-fn] - (t/submit-local-topology (:nimbus cluster) "tester" {} (.build topo)) +(defn with-topology* [cluster storm-topo body-fn] + (t/submit-local-topology (:nimbus cluster) "tester" {} storm-topo) (body-fn) - (.killTopologyWithOpts (:nimbus cluster) "tester" (doto (KillOptions.) (.set_wait_secs 0))) - ) + (.killTopologyWithOpts (:nimbus cluster) "tester" (doto (KillOptions.) (.set_wait_secs 0)))) -(defmacro with-topology [[cluster topo] & body] - `(with-topology* ~cluster ~topo (fn [] ~@body))) +(defmacro with-topology [[cluster topo storm-topo] & body] + `(let [~storm-topo (.build ~topo)] + (with-topology* ~cluster ~storm-topo (fn [] ~@body)))) (defn bootstrap-imports [] (import 'org.apache.storm.LocalDRPC) diff --git a/storm-core/src/jvm/org/apache/storm/topology/ComponentConfigurationDeclarer.java b/storm-core/src/jvm/org/apache/storm/topology/ComponentConfigurationDeclarer.java index 328af5549d1..5dc726445de 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/ComponentConfigurationDeclarer.java +++ b/storm-core/src/jvm/org/apache/storm/topology/ComponentConfigurationDeclarer.java @@ -19,14 +19,11 @@ import java.util.Map; -public interface ComponentConfigurationDeclarer { +public interface ComponentConfigurationDeclarer extends ResourceDeclarer { T addConfigurations(Map conf); T addConfiguration(String config, Object value); T setDebug(boolean debug); T setMaxTaskParallelism(Number val); T setMaxSpoutPending(Number val); T setNumTasks(Number val); - T setMemoryLoad(Number onHeap); - T setMemoryLoad(Number onHeap, Number offHeap); - T setCPULoad(Number amount); } diff --git a/storm-core/src/jvm/org/apache/storm/topology/ResourceDeclarer.java b/storm-core/src/jvm/org/apache/storm/topology/ResourceDeclarer.java new file mode 100644 index 00000000000..4f648eb7838 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/topology/ResourceDeclarer.java @@ -0,0 +1,28 @@ +/** + * 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 org.apache.storm.topology; + +/** + * This is a new base interface that can be used by anything that wants to mirror + * RAS's basic API. Trident uses this to allow setting resources in the Stream API. + */ +public interface ResourceDeclarer { + T setMemoryLoad(Number onHeap); + T setMemoryLoad(Number onHeap, Number offHeap); + T setCPULoad(Number amount); +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/Stream.java b/storm-core/src/jvm/org/apache/storm/trident/Stream.java index d313678476a..4a51b56dc2a 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/Stream.java +++ b/storm-core/src/jvm/org/apache/storm/trident/Stream.java @@ -20,6 +20,7 @@ import org.apache.storm.generated.Grouping; import org.apache.storm.generated.NullStruct; import org.apache.storm.grouping.CustomStreamGrouping; +import org.apache.storm.topology.ResourceDeclarer; import org.apache.storm.trident.fluent.ChainedAggregatorDeclarer; import org.apache.storm.trident.fluent.GlobalAggregationScheme; import org.apache.storm.trident.fluent.GroupedStream; @@ -90,7 +91,7 @@ * */ // TODO: need to be able to replace existing fields with the function fields (like Cascading Fields.REPLACE) -public class Stream implements IAggregatableStream { +public class Stream implements IAggregatableStream, ResourceDeclarer { Node _node; TridentTopology _topology; String _name; @@ -123,6 +124,34 @@ public Stream parallelismHint(int hint) { return this; } + /** + * Sets the CPU Load resource for the current operation + */ + @Override + public Stream setCPULoad(Number load) { + _node.setCPULoad(load); + return this; + } + + /** + * Sets the Memory Load resources for the current operation. + * offHeap becomes default + */ + @Override + public Stream setMemoryLoad(Number onHeap) { + _node.setMemoryLoad(onHeap); + return this; + } + + /** + * Sets the Memory Load resources for the current operation. + */ + @Override + public Stream setMemoryLoad(Number onHeap, Number offHeap) { + _node.setMemoryLoad(onHeap, offHeap); + return this; + } + /** * Filters out fields from a stream, resulting in a Stream containing only the fields specified by `keepFields`. * diff --git a/storm-core/src/jvm/org/apache/storm/trident/TridentState.java b/storm-core/src/jvm/org/apache/storm/trident/TridentState.java index 7173254e8b0..18b60e05854 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/TridentState.java +++ b/storm-core/src/jvm/org/apache/storm/trident/TridentState.java @@ -17,24 +17,43 @@ */ package org.apache.storm.trident; +import org.apache.storm.topology.ResourceDeclarer; import org.apache.storm.trident.planner.Node; -public class TridentState { +public class TridentState implements ResourceDeclarer { TridentTopology _topology; Node _node; - + protected TridentState(TridentTopology topology, Node node) { _topology = topology; _node = node; } - + public Stream newValuesStream() { return new Stream(_topology, _node.name, _node); } - + public TridentState parallelismHint(int parallelism) { _node.parallelismHint = parallelism; return this; } + + @Override + public TridentState setCPULoad(Number load) { + _node.setCPULoad(load); + return this; + } + + @Override + public TridentState setMemoryLoad(Number onHeap) { + _node.setMemoryLoad(onHeap); + return this; + } + + @Override + public TridentState setMemoryLoad(Number onHeap, Number offHeap) { + _node.setMemoryLoad(onHeap, offHeap); + return this; + } } diff --git a/storm-core/src/jvm/org/apache/storm/trident/TridentTopology.java b/storm-core/src/jvm/org/apache/storm/trident/TridentTopology.java index eb50a10a009..e0a349b0e5c 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/TridentTopology.java +++ b/storm-core/src/jvm/org/apache/storm/trident/TridentTopology.java @@ -44,6 +44,7 @@ import org.apache.storm.trident.graph.GraphGrouper; import org.apache.storm.trident.graph.Group; import org.apache.storm.trident.operation.GroupedMultiReducer; +import org.apache.storm.trident.operation.ITridentResource; import org.apache.storm.trident.operation.MultiReducer; import org.apache.storm.trident.operation.impl.FilterExecutor; import org.apache.storm.trident.operation.impl.GroupedMultiReducerExecutor; @@ -394,11 +395,23 @@ public StormTopology build() { Map spoutIds = genSpoutIds(spoutNodes); Map boltIds = genBoltIds(mergedGroups); + Map defaults = Utils.readDefaultConfig(); + for(SpoutNode sn: spoutNodes) { Integer parallelism = parallelisms.get(grouper.nodeGroup(sn)); + + Map spoutRes = null; + spoutRes = mergeDefaultResources(sn.getResources(), defaults); + Number onHeap = spoutRes.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB); + Number offHeap = spoutRes.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB); + Number cpuLoad = spoutRes.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT); + if(sn.type == SpoutNode.SpoutType.DRPC) { + builder.setBatchPerTupleSpout(spoutIds.get(sn), sn.streamId, - (IRichSpout) sn.spout, parallelism, batchGroupMap.get(sn)); + (IRichSpout) sn.spout, parallelism, batchGroupMap.get(sn)) + .setMemoryLoad(onHeap, offHeap) + .setCPULoad(cpuLoad); } else { ITridentSpout s; if(sn.spout instanceof IBatchSpout) { @@ -409,16 +422,26 @@ public StormTopology build() { throw new RuntimeException("Regular rich spouts not supported yet... try wrapping in a RichSpoutBatchExecutor"); // TODO: handle regular rich spout without batches (need lots of updates to support this throughout) } - builder.setSpout(spoutIds.get(sn), sn.streamId, sn.txId, s, parallelism, batchGroupMap.get(sn)); + builder.setSpout(spoutIds.get(sn), sn.streamId, sn.txId, s, parallelism, batchGroupMap.get(sn)) + .setMemoryLoad(onHeap, offHeap) + .setCPULoad(cpuLoad); } } - + for(Group g: mergedGroups) { if(!isSpoutGroup(g)) { Integer p = parallelisms.get(g); Map streamToGroup = getOutputStreamBatchGroups(g, batchGroupMap); + Map groupRes = mergeDefaultResources(g.getResources(), defaults); + + Number onHeap = groupRes.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB); + Number offHeap = groupRes.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB); + Number cpuLoad = groupRes.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT); + BoltDeclarer d = builder.setBolt(boltIds.get(g), new SubtopologyBolt(graph, g.nodes, batchGroupMap), p, - committerBatches(g, batchGroupMap), streamToGroup); + committerBatches(g, batchGroupMap), streamToGroup) + .setMemoryLoad(onHeap, offHeap) + .setCPULoad(cpuLoad); Collection inputs = uniquedSubscriptions(externalGroupInputs(g)); for(PartitionNode n: inputs) { Node parent = TridentUtils.getParent(graph, n); @@ -431,6 +454,64 @@ public StormTopology build() { return builder.buildTopology(); } + + private static Map mergeDefaultResources(Map res, Map defaultConfig) { + Map ret = new HashMap(); + + Number onHeapDefault = (Number)defaultConfig.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB); + Number offHeapDefault = (Number)defaultConfig.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB); + Number cpuLoadDefault = (Number)defaultConfig.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT); + + if(res == null) { + ret.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, onHeapDefault); + ret.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, offHeapDefault); + ret.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, cpuLoadDefault); + return ret; + } + + Number onHeap = res.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB); + Number offHeap = res.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB); + Number cpuLoad = res.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT); + + /* We take the max of the default and whatever the user put in here. + Each node's resources can be the sum of several operations, so the simplest + thing to do is get the max. + + The situation we want to avoid is that the user sets low resources on one + node, and when that node is combined with a bunch of others, the sum is still + that low resource count. If any component isn't set, we want to use the default. + + Right now, this code does not check that. It just takes the max of the summed + up resource counts for simplicity's sake. We could perform some more complicated + logic to be more accurate, but the benefits are very small, and only apply to some + very odd corner cases. */ + if(onHeap == null) { + onHeap = onHeapDefault; + } + else { + onHeap = Math.max(onHeap.doubleValue(), onHeapDefault.doubleValue()); + } + + if(offHeap == null) { + offHeap = offHeapDefault; + } + else { + offHeap = Math.max(offHeap.doubleValue(), offHeapDefault.doubleValue()); + } + + if(cpuLoad == null) { + cpuLoad = cpuLoadDefault; + } + else { + cpuLoad = Math.max(cpuLoad.doubleValue(), cpuLoadDefault.doubleValue()); + } + + ret.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, onHeap); + ret.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, offHeap); + ret.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, cpuLoad); + + return ret; + } private static void completeDRPC(DefaultDirectedGraph graph, Map> colocate, UniqueIdGen gen) { List> connectedComponents = new ConnectivityInspector<>(graph).connectedSets(); @@ -464,7 +545,7 @@ private static Node getLastAddedNode(Collection g) { } return ret; } - + //returns null if it's not a drpc group private static SpoutNode getDRPCSpoutNode(Collection g) { for(Node n: g) { diff --git a/storm-core/src/jvm/org/apache/storm/trident/graph/Group.java b/storm-core/src/jvm/org/apache/storm/trident/graph/Group.java index ef1399baa0b..2c923043d96 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/graph/Group.java +++ b/storm-core/src/jvm/org/apache/storm/trident/graph/Group.java @@ -18,17 +18,20 @@ package org.apache.storm.trident.graph; import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.UUID; import org.jgrapht.DirectedGraph; +import org.apache.storm.trident.operation.ITridentResource; import org.apache.storm.trident.planner.Node; import org.apache.storm.trident.util.IndexedEdge; import org.apache.storm.trident.util.TridentUtils; -public class Group { +public class Group implements ITridentResource { public final Set nodes = new HashSet<>(); private final DirectedGraph graph; private final String id = UUID.randomUUID().toString(); @@ -64,6 +67,23 @@ public Set incomingNodes() { return ret; } + @Override + public Map getResources() { + Map ret = new HashMap<>(); + for(Node n: nodes) { + Map res = n.getResources(); + for(Map.Entry kv : res.entrySet()) { + String key = kv.getKey(); + Number val = kv.getValue(); + if(ret.containsKey(key)) { + val = new Double(val.doubleValue() + ret.get(key).doubleValue()); + } + ret.put(key, val); + } + } + return ret; + } + @Override public int hashCode() { return id.hashCode(); diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/DefaultResourceDeclarer.java b/storm-core/src/jvm/org/apache/storm/trident/operation/DefaultResourceDeclarer.java new file mode 100644 index 00000000000..d49011adaf9 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/DefaultResourceDeclarer.java @@ -0,0 +1,66 @@ +/** + * 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 org.apache.storm.trident.operation; + +import java.util.HashMap; +import java.util.Map; +import org.apache.storm.Config; +import org.apache.storm.utils.Utils; +import org.apache.storm.topology.ResourceDeclarer; + +/** + * @param T Must always be the type of the extending class. i.e. + * public class SubResourceDeclarer extends DefaultResourceDeclarer {...} + */ +public class DefaultResourceDeclarer implements ResourceDeclarer, ITridentResource { + + private Map resources = new HashMap<>(); + private Map conf = Utils.readStormConfig(); + + @Override + public T setMemoryLoad(Number onHeap) { + return setMemoryLoad(onHeap, Utils.getDouble(conf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB))); + } + + @Override + public T setMemoryLoad(Number onHeap, Number offHeap) { + if (onHeap != null) { + onHeap = onHeap.doubleValue(); + resources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, onHeap); + } + if (offHeap!=null) { + offHeap = offHeap.doubleValue(); + resources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, offHeap); + } + return (T)this; + } + + @Override + public T setCPULoad(Number amount) { + if(amount != null) { + amount = amount.doubleValue(); + resources.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, amount); + } + return (T)this; + } + + @Override + public Map getResources() { + return new HashMap(resources); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/operation/ITridentResource.java b/storm-core/src/jvm/org/apache/storm/trident/operation/ITridentResource.java new file mode 100644 index 00000000000..b3e10ef6f88 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/trident/operation/ITridentResource.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 org.apache.storm.trident.operation; + +import java.util.Map; + +/** + * This interface is implemented by various Trident classes in order to + * gather and propogate resources that have been set on them. + * @see ResourceDeclarer + */ +public interface ITridentResource { + /** + * @return a name of resource name -> amount of that resource. *Return should never be null!* + */ + Map getResources(); +} diff --git a/storm-core/src/jvm/org/apache/storm/trident/planner/Node.java b/storm-core/src/jvm/org/apache/storm/trident/planner/Node.java index 64d8a3bb83a..b2466e69a58 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/planner/Node.java +++ b/storm-core/src/jvm/org/apache/storm/trident/planner/Node.java @@ -17,6 +17,7 @@ */ package org.apache.storm.trident.planner; +import org.apache.storm.trident.operation.DefaultResourceDeclarer; import org.apache.storm.tuple.Fields; import java.io.Serializable; import java.util.UUID; @@ -25,7 +26,7 @@ import org.apache.commons.lang.builder.ToStringStyle; -public class Node implements Serializable { +public class Node extends DefaultResourceDeclarer implements Serializable { private static final AtomicInteger INDEX = new AtomicInteger(0); private String nodeId; @@ -62,6 +63,4 @@ public int hashCode() { public String toString() { return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE); } - - } diff --git a/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj index 4c52286d69b..7f81c4b9a08 100644 --- a/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj +++ b/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj @@ -19,10 +19,13 @@ (:import [org.apache.storm.trident.testing Split CountAsAggregator StringLength TrueFilter MemoryMapState$Factory]) (:import [org.apache.storm.trident.state StateSpec]) - (:import [org.apache.storm.trident.operation.impl CombinerAggStateUpdater]) - (:use [org.apache.storm.trident testing]) - (:use [org.apache.storm util])) - + (:import [org.apache.storm.trident.operation.impl CombinerAggStateUpdater] + [org.apache.storm.trident.operation BaseFunction] + [org.json.simple.parser JSONParser] + [org.apache.storm Config]) + (:use [org.apache.storm.trident testing] + [org.apache.storm log util config])) + (bootstrap-imports) (deftest test-memory-map-get-tuples @@ -38,13 +41,13 @@ (.groupBy (fields "word")) (.persistentAggregate (memory-map-state) (Count.) (fields "count")) (.parallelismHint 6) - )) + )) (-> topo (.newDRPCStream "all-tuples" drpc) (.broadcast) (.stateQuery word-counts (fields "args") (TupleCollectionGet.) (fields "word" "count")) (.project (fields "word" "count"))) - (with-topology [cluster topo] + (with-topology [cluster topo storm-topo] (feed feeder [["hello the man said"] ["the"]]) (is (= #{["hello" 1] ["said" 1] ["the" 2] ["man" 1]} (into #{} (exec-drpc drpc "all-tuples" "man")))) @@ -73,7 +76,7 @@ (.stateQuery word-counts (fields "word") (MapGet.) (fields "count")) (.aggregate (fields "count") (Sum.) (fields "sum")) (.project (fields "sum"))) - (with-topology [cluster topo] + (with-topology [cluster topo storm-topo] (feed feeder [["hello the man said"] ["the"]]) (is (= [[2]] (exec-drpc drpc "words" "the"))) (is (= [[1]] (exec-drpc drpc "words" "hello"))) @@ -83,7 +86,7 @@ (is (= [[8]] (exec-drpc drpc "words" "man where you the"))) ))))) -;; this test reproduces a bug where committer spouts freeze processing when +;; this test reproduces a bug where committer spouts freeze processing when ;; there's at least one repartitioning after the spout (deftest test-word-count-committer-spout (t/with-local-cluster [cluster] @@ -108,7 +111,7 @@ (.stateQuery word-counts (fields "word") (MapGet.) (fields "count")) (.aggregate (fields "count") (Sum.) (fields "sum")) (.project (fields "sum"))) - (with-topology [cluster topo] + (with-topology [cluster topo storm-topo] (feed feeder [["hello the man said"] ["the"]]) (is (= [[2]] (exec-drpc drpc "words" "the"))) (is (= [[1]] (exec-drpc drpc "words" "hello"))) @@ -135,13 +138,13 @@ (.aggregate (CountAsAggregator.) (fields "count")) (.parallelismHint 2) ;;this makes sure batchGlobal is working correctly (.project (fields "count"))) - (with-topology [cluster topo] + (with-topology [cluster topo storm-topo] (doseq [i (range 100)] (is (= [[1]] (exec-drpc drpc "numwords" "the")))) (is (= [[0]] (exec-drpc drpc "numwords" ""))) (is (= [[8]] (exec-drpc drpc "numwords" "1 2 3 4 5 6 7 8"))) ))))) - + (deftest test-split-merge (t/with-local-cluster [cluster] (with-drpc [drpc] @@ -158,7 +161,7 @@ (.project (fields "len")))) (.merge topo [s1 s2]) - (with-topology [cluster topo] + (with-topology [cluster topo storm-topo] (is (t/ms= [[7] ["the"] ["man"]] (exec-drpc drpc "splitter" "the man"))) (is (t/ms= [[5] ["hello"]] (exec-drpc drpc "splitter" "hello"))) ))))) @@ -180,11 +183,11 @@ (.aggregate (CountAsAggregator.) (fields "count")))) (.merge topo [s1 s2]) - (with-topology [cluster topo] + (with-topology [cluster topo storm-topo] (is (t/ms= [["the" 1] ["the" 1]] (exec-drpc drpc "tester" "the"))) (is (t/ms= [["aaaaa" 1] ["aaaaa" 1]] (exec-drpc drpc "tester" "aaaaa"))) ))))) - + (deftest test-multi-repartition (t/with-local-cluster [cluster] (with-drpc [drpc] @@ -196,7 +199,7 @@ (.shuffle) (.aggregate (CountAsAggregator.) (fields "count")) )) - (with-topology [cluster topo] + (with-topology [cluster topo storm-topo] (is (t/ms= [[2]] (exec-drpc drpc "tester" "the man"))) (is (t/ms= [[1]] (exec-drpc drpc "tester" "aaa"))) ))))) @@ -270,6 +273,82 @@ (.stateQuery word-counts (fields "word1") (MapGet.) (fields "count")))))) ))) + +(deftest test-set-component-resources + (t/with-local-cluster [cluster] + (with-drpc [drpc] + (letlocals + (bind topo (TridentTopology.)) + (bind feeder (feeder-spout ["sentence"])) + (bind add-bang (proxy [BaseFunction] [] + (execute [tuple collector] + (. collector emit (str (. tuple getString 0) "!"))))) + (bind word-counts + (.. topo + (newStream "words" feeder) + (parallelismHint 5) + (setCPULoad 20) + (setMemoryLoad 512 256) + (each (fields "sentence") (Split.) (fields "word")) + (setCPULoad 10) + (setMemoryLoad 512) + (each (fields "word") add-bang (fields "word!")) + (parallelismHint 10) + (setCPULoad 50) + (setMemoryLoad 1024) + (groupBy (fields "word!")) + (persistentAggregate (memory-map-state) (Count.) (fields "count")) + (setCPULoad 100) + (setMemoryLoad 2048))) + (with-topology [cluster topo storm-topo] + + (let [parse-fn (fn [[k v]] + [k (clojurify-structure (. (JSONParser.) parse (.. v get_common get_json_conf)))]) + json-confs (into {} (map parse-fn (. storm-topo get_bolts)))] + (testing "spout memory" + (is (= (-> (json-confs "spout-words") + (get TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB)) + 512.0)) + + (is (= (-> (json-confs "spout-words") + (get TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB)) + 256.0)) + + (is (= (-> (json-confs "$spoutcoord-spout-words") + (get TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB)) + 512.0)) + + (is (= (-> (json-confs "$spoutcoord-spout-words") + (get TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB)) + 256.0))) + + (testing "spout CPU" + (is (= (-> (json-confs "spout-words") + (get TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT)) + 20.0)) + + (is (= (-> (json-confs "$spoutcoord-spout-words") + (get TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT)) + 20.0))) + + (testing "bolt combinations" + (is (= (-> (json-confs "b-1") + (get TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB)) + (+ 1024.0 512.0))) + + (is (= (-> (json-confs "b-1") + (get TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT)) + 60.0))) + + (testing "aggregations after partition" + (is (= (-> (json-confs "b-0") + (get TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB)) + 2048.0)) + + (is (= (-> (json-confs "b-0") + (get TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT)) + 100.0))))))))) + ;; (deftest test-split-merge ;; (t/with-local-cluster [cluster] ;; (with-drpc [drpc] @@ -284,7 +363,7 @@ ;; (-> drpc-stream ;; (.each (fields "args") (StringLength.) (fields "len")) ;; (.project (fields "len")))) -;; +;; ;; (.merge topo [s1 s2]) ;; (with-topology [cluster topo] ;; (is (t/ms= [[7] ["the"] ["man"]] (exec-drpc drpc "splitter" "the man"))) From 885aaec4349c78016fbe6e3f8c8db3f7b8426704 Mon Sep 17 00:00:00 2001 From: Kyle Nusbaum Date: Thu, 17 Mar 2016 15:51:09 -0500 Subject: [PATCH 136/387] Adding STORM-1616 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bb139d355d2..861cafd0883 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 1.0.0 + * STORM-1616: Add RAS API for Trident * STORM-1483: add storm-mongodb connector * STORM-1614: backpressure init and cleanup changes * STORM-1549: Add support for resetting tuple timeout from bolts via the OutputCollector From d491c3ff225f6fa7a027bc04b4ebb575f4f128e8 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Sat, 19 Mar 2016 12:15:21 -0500 Subject: [PATCH 137/387] STORM-1617: Release Specific Documentation 0.9.x Conflicts: .gitignore docs/README.md docs/_config.yml docs/_includes/footer.html docs/_includes/head.html docs/_includes/header.html docs/_layouts/about.html docs/_layouts/default.html docs/_layouts/documentation.html docs/_layouts/page.html docs/_layouts/post.html docs/assets/css/bootstrap.css docs/assets/css/bootstrap.css.map docs/assets/js/bootstrap.min.js docs/images/logos/alibaba.jpg docs/images/logos/groupon.jpg docs/images/logos/parc.png docs/images/logos/webmd.jpg docs/images/topology.png Conflicts: .gitignore docs/README.md docs/STORM-UI-REST-API.md docs/_config.yml docs/_includes/footer.html docs/_includes/head.html docs/_includes/header.html docs/_layouts/about.html docs/_layouts/default.html docs/_layouts/documentation.html docs/_layouts/page.html docs/_layouts/post.html docs/assets/css/bootstrap.css docs/assets/css/bootstrap.css.map docs/assets/js/bootstrap.min.js docs/images/logos/alibaba.jpg docs/images/logos/groupon.jpg docs/images/logos/parc.png docs/images/logos/webmd.jpg --- .gitignore | 1 + docs/Acking-framework-implementation.md | 36 + docs/Clojure-DSL.md | 264 + docs/Command-line-client.md | 100 + docs/Common-patterns.md | 86 + docs/Concepts.md | 115 + docs/Configuration.md | 29 + docs/Contributing-to-Storm.md | 31 + docs/Creating-a-new-Storm-project.md | 25 + docs/DSLs-and-multilang-adapters.md | 9 + ...fining-a-non-jvm-language-dsl-for-storm.md | 36 + docs/Distributed-RPC.md | 197 + docs/Documentation.md | 50 + docs/FAQ.md | 121 + docs/Fault-tolerance.md | 28 + docs/Guaranteeing-message-processing.md | 179 + docs/Hooks.md | 7 + docs/Implementation-docs.md | 18 + docs/Installing-native-dependencies.md | 38 + docs/Kestrel-and-Storm.md | 198 + docs/Lifecycle-of-a-topology.md | 80 + docs/Local-mode.md | 27 + docs/Maven.md | 56 + docs/Message-passing-implementation.md | 28 + docs/Metrics.md | 34 + docs/Multilang-protocol.md | 221 + docs/Powered-By.md | 1028 +++ docs/Project-ideas.md | 6 + docs/README.md | 61 + docs/Rationale.md | 31 + ...ning-topologies-on-a-production-cluster.md | 75 + docs/SECURITY.md | 79 + docs/STORM-UI-REST-API.md | 678 ++ docs/Serialization-(prior-to-0.6.0).md | 50 + docs/Serialization.md | 60 + docs/Serializers.md | 4 + docs/Setting-up-a-Storm-cluster.md | 83 + docs/Setting-up-a-Storm-project-in-Eclipse.md | 1 + docs/Setting-up-development-environment.md | 39 + docs/Spout-implementations.md | 8 + ...age-protocol-(versions-0.7.0-and-below).md | 122 + docs/Structure-of-the-codebase.md | 140 + docs/Support-for-non-java-languages.md | 7 + docs/Transactional-topologies.md | 359 + docs/Trident-API-Overview.md | 311 + docs/Trident-spouts.md | 42 + docs/Trident-state.md | 330 + docs/Trident-tutorial.md | 253 + docs/Troubleshooting.md | 144 + docs/Tutorial.md | 310 + ...ing-the-parallelism-of-a-Storm-topology.md | 121 + docs/Using-non-JVM-languages-with-Storm.md | 52 + docs/_config.yml | 18 + docs/_includes/footer.html | 55 + docs/_includes/head.html | 34 + docs/_includes/header.html | 59 + docs/_layouts/about.html | 43 + docs/_layouts/default.html | 18 + docs/_layouts/documentation.html | 9 + docs/_layouts/page.html | 5 + docs/_layouts/post.html | 61 + docs/_plugins/releases.rb | 84 + docs/assets/css/bootstrap.css | 6800 +++++++++++++++++ docs/assets/css/bootstrap.css.map | 1 + docs/assets/css/font-awesome.min.css | 4 + docs/assets/css/main.scss | 48 + docs/assets/css/owl.carousel.css | 71 + docs/assets/css/owl.theme.css | 79 + docs/assets/css/style.css | 503 ++ docs/assets/js/bootstrap.min.js | 7 + docs/assets/js/jquery.min.js | 6 + docs/assets/js/owl.carousel.min.js | 47 + docs/assets/js/storm.js | 67 + docs/css/style.css | 553 ++ docs/favicon.ico | Bin 0 -> 1150 bytes docs/images/ack_tree.png | Bin 0 -> 31463 bytes docs/images/batched-stream.png | Bin 0 -> 66336 bytes docs/images/drpc-workflow.png | Bin 0 -> 66199 bytes docs/images/eclipse-project-properties.png | Bin 0 -> 80810 bytes docs/images/example-of-a-running-topology.png | Bin 0 -> 81430 bytes docs/images/footer-bg.png | Bin 0 -> 138 bytes docs/images/grouping.png | Bin 0 -> 39701 bytes docs/images/header-bg.png | Bin 0 -> 470 bytes docs/images/ld-library-path-eclipse-linux.png | Bin 0 -> 114597 bytes docs/images/loading.gif | Bin 0 -> 12150 bytes docs/images/logo.png | Bin 0 -> 26889 bytes docs/images/logos/aeris.jpg | Bin 0 -> 7420 bytes docs/images/logos/alibaba.jpg | Bin 0 -> 10317 bytes docs/images/logos/bai.jpg | Bin 0 -> 10026 bytes docs/images/logos/cerner.jpg | Bin 0 -> 7244 bytes docs/images/logos/flipboard.jpg | Bin 0 -> 8318 bytes docs/images/logos/fullcontact.jpg | Bin 0 -> 6172 bytes docs/images/logos/groupon.jpg | Bin 0 -> 9849 bytes docs/images/logos/health-market-science.jpg | Bin 0 -> 6509 bytes docs/images/logos/images.png | Bin 0 -> 7339 bytes docs/images/logos/infochimp.jpg | Bin 0 -> 5290 bytes docs/images/logos/klout.jpg | Bin 0 -> 7251 bytes docs/images/logos/loggly.jpg | Bin 0 -> 9258 bytes docs/images/logos/ooyala.jpg | Bin 0 -> 5675 bytes docs/images/logos/parc.png | Bin 0 -> 13720 bytes docs/images/logos/premise.jpg | Bin 0 -> 5391 bytes docs/images/logos/qiy.jpg | Bin 0 -> 7441 bytes docs/images/logos/quicklizard.jpg | Bin 0 -> 7382 bytes docs/images/logos/rocketfuel.jpg | Bin 0 -> 10007 bytes docs/images/logos/rubicon.jpg | Bin 0 -> 7120 bytes docs/images/logos/spider.jpg | Bin 0 -> 6265 bytes docs/images/logos/spotify.jpg | Bin 0 -> 6445 bytes docs/images/logos/taobao.jpg | Bin 0 -> 16814 bytes docs/images/logos/the-weather-channel.jpg | Bin 0 -> 13295 bytes docs/images/logos/twitter.jpg | Bin 0 -> 7139 bytes docs/images/logos/verisign.jpg | Bin 0 -> 5982 bytes docs/images/logos/webmd.jpg | Bin 0 -> 8226 bytes docs/images/logos/wego.jpg | Bin 0 -> 6836 bytes docs/images/logos/yahoo-japan.jpg | Bin 0 -> 10350 bytes docs/images/logos/yahoo.png | Bin 0 -> 13067 bytes docs/images/logos/yelp.jpg | Bin 0 -> 7220 bytes ...ships-worker-processes-executors-tasks.png | Bin 0 -> 54804 bytes docs/images/spout-vs-state.png | Bin 0 -> 24804 bytes docs/images/storm-cluster.png | Bin 0 -> 34604 bytes docs/images/storm-flow.png | Bin 0 -> 59688 bytes docs/images/topology-tasks.png | Bin 0 -> 45960 bytes docs/images/transactional-batches.png | Bin 0 -> 23293 bytes docs/images/transactional-commit-flow.png | Bin 0 -> 17725 bytes docs/images/transactional-design-2.png | Bin 0 -> 13537 bytes docs/images/transactional-spout-structure.png | Bin 0 -> 25067 bytes docs/images/trident-to-storm1.png | Bin 0 -> 67173 bytes docs/images/trident-to-storm2.png | Bin 0 -> 68943 bytes docs/images/tuple-dag.png | Bin 0 -> 18849 bytes docs/images/tuple_tree.png | Bin 0 -> 58186 bytes docs/index.md | 69 + 130 files changed, 15049 insertions(+) create mode 100644 docs/Acking-framework-implementation.md create mode 100644 docs/Clojure-DSL.md create mode 100644 docs/Command-line-client.md create mode 100644 docs/Common-patterns.md create mode 100644 docs/Concepts.md create mode 100644 docs/Configuration.md create mode 100644 docs/Contributing-to-Storm.md create mode 100644 docs/Creating-a-new-Storm-project.md create mode 100644 docs/DSLs-and-multilang-adapters.md create mode 100644 docs/Defining-a-non-jvm-language-dsl-for-storm.md create mode 100644 docs/Distributed-RPC.md create mode 100644 docs/Documentation.md create mode 100644 docs/FAQ.md create mode 100644 docs/Fault-tolerance.md create mode 100644 docs/Guaranteeing-message-processing.md create mode 100644 docs/Hooks.md create mode 100644 docs/Implementation-docs.md create mode 100644 docs/Installing-native-dependencies.md create mode 100644 docs/Kestrel-and-Storm.md create mode 100644 docs/Lifecycle-of-a-topology.md create mode 100644 docs/Local-mode.md create mode 100644 docs/Maven.md create mode 100644 docs/Message-passing-implementation.md create mode 100644 docs/Metrics.md create mode 100644 docs/Multilang-protocol.md create mode 100644 docs/Powered-By.md create mode 100644 docs/Project-ideas.md create mode 100644 docs/README.md create mode 100644 docs/Rationale.md create mode 100644 docs/Running-topologies-on-a-production-cluster.md create mode 100644 docs/SECURITY.md create mode 100644 docs/STORM-UI-REST-API.md create mode 100644 docs/Serialization-(prior-to-0.6.0).md create mode 100644 docs/Serialization.md create mode 100644 docs/Serializers.md create mode 100644 docs/Setting-up-a-Storm-cluster.md create mode 100644 docs/Setting-up-a-Storm-project-in-Eclipse.md create mode 100644 docs/Setting-up-development-environment.md create mode 100644 docs/Spout-implementations.md create mode 100644 docs/Storm-multi-language-protocol-(versions-0.7.0-and-below).md create mode 100644 docs/Structure-of-the-codebase.md create mode 100644 docs/Support-for-non-java-languages.md create mode 100644 docs/Transactional-topologies.md create mode 100644 docs/Trident-API-Overview.md create mode 100644 docs/Trident-spouts.md create mode 100644 docs/Trident-state.md create mode 100644 docs/Trident-tutorial.md create mode 100644 docs/Troubleshooting.md create mode 100644 docs/Tutorial.md create mode 100644 docs/Understanding-the-parallelism-of-a-Storm-topology.md create mode 100644 docs/Using-non-JVM-languages-with-Storm.md create mode 100644 docs/_config.yml create mode 100644 docs/_includes/footer.html create mode 100644 docs/_includes/head.html create mode 100644 docs/_includes/header.html create mode 100644 docs/_layouts/about.html create mode 100644 docs/_layouts/default.html create mode 100644 docs/_layouts/documentation.html create mode 100644 docs/_layouts/page.html create mode 100644 docs/_layouts/post.html create mode 100644 docs/_plugins/releases.rb create mode 100644 docs/assets/css/bootstrap.css create mode 100644 docs/assets/css/bootstrap.css.map create mode 100644 docs/assets/css/font-awesome.min.css create mode 100644 docs/assets/css/main.scss create mode 100644 docs/assets/css/owl.carousel.css create mode 100644 docs/assets/css/owl.theme.css create mode 100644 docs/assets/css/style.css create mode 100644 docs/assets/js/bootstrap.min.js create mode 100644 docs/assets/js/jquery.min.js create mode 100644 docs/assets/js/owl.carousel.min.js create mode 100644 docs/assets/js/storm.js create mode 100644 docs/css/style.css create mode 100644 docs/favicon.ico create mode 100644 docs/images/ack_tree.png create mode 100644 docs/images/batched-stream.png create mode 100644 docs/images/drpc-workflow.png create mode 100644 docs/images/eclipse-project-properties.png create mode 100644 docs/images/example-of-a-running-topology.png create mode 100644 docs/images/footer-bg.png create mode 100644 docs/images/grouping.png create mode 100644 docs/images/header-bg.png create mode 100644 docs/images/ld-library-path-eclipse-linux.png create mode 100644 docs/images/loading.gif create mode 100644 docs/images/logo.png create mode 100644 docs/images/logos/aeris.jpg create mode 100644 docs/images/logos/alibaba.jpg create mode 100644 docs/images/logos/bai.jpg create mode 100644 docs/images/logos/cerner.jpg create mode 100644 docs/images/logos/flipboard.jpg create mode 100644 docs/images/logos/fullcontact.jpg create mode 100644 docs/images/logos/groupon.jpg create mode 100644 docs/images/logos/health-market-science.jpg create mode 100644 docs/images/logos/images.png create mode 100644 docs/images/logos/infochimp.jpg create mode 100644 docs/images/logos/klout.jpg create mode 100644 docs/images/logos/loggly.jpg create mode 100644 docs/images/logos/ooyala.jpg create mode 100644 docs/images/logos/parc.png create mode 100644 docs/images/logos/premise.jpg create mode 100644 docs/images/logos/qiy.jpg create mode 100644 docs/images/logos/quicklizard.jpg create mode 100644 docs/images/logos/rocketfuel.jpg create mode 100644 docs/images/logos/rubicon.jpg create mode 100644 docs/images/logos/spider.jpg create mode 100644 docs/images/logos/spotify.jpg create mode 100644 docs/images/logos/taobao.jpg create mode 100644 docs/images/logos/the-weather-channel.jpg create mode 100644 docs/images/logos/twitter.jpg create mode 100644 docs/images/logos/verisign.jpg create mode 100644 docs/images/logos/webmd.jpg create mode 100644 docs/images/logos/wego.jpg create mode 100644 docs/images/logos/yahoo-japan.jpg create mode 100755 docs/images/logos/yahoo.png create mode 100644 docs/images/logos/yelp.jpg create mode 100644 docs/images/relationships-worker-processes-executors-tasks.png create mode 100644 docs/images/spout-vs-state.png create mode 100644 docs/images/storm-cluster.png create mode 100644 docs/images/storm-flow.png create mode 100644 docs/images/topology-tasks.png create mode 100644 docs/images/transactional-batches.png create mode 100644 docs/images/transactional-commit-flow.png create mode 100644 docs/images/transactional-design-2.png create mode 100644 docs/images/transactional-spout-structure.png create mode 100644 docs/images/trident-to-storm1.png create mode 100644 docs/images/trident-to-storm2.png create mode 100644 docs/images/tuple-dag.png create mode 100644 docs/images/tuple_tree.png create mode 100644 docs/index.md diff --git a/.gitignore b/.gitignore index 54bd2893097..6ec109bb2b2 100644 --- a/.gitignore +++ b/.gitignore @@ -40,3 +40,4 @@ metastore_db .classpath logs build +/docs/javadocs diff --git a/docs/Acking-framework-implementation.md b/docs/Acking-framework-implementation.md new file mode 100644 index 00000000000..5ca5d93df0d --- /dev/null +++ b/docs/Acking-framework-implementation.md @@ -0,0 +1,36 @@ +--- +layout: documentation +--- +[Storm's acker](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L28) tracks completion of each tupletree with a checksum hash: each time a tuple is sent, its value is XORed into the checksum, and each time a tuple is acked its value is XORed in again. If all tuples have been successfully acked, the checksum will be zero (the odds that the checksum will be zero otherwise are vanishingly small). + +You can read a bit more about the [reliability mechanism](Guaranteeing-message-processing.html#what-is-storms-reliability-api) elsewhere on the wiki -- this explains the internal details. + +### acker `execute()` + +The acker is actually a regular bolt, with its [execute method](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L36) defined withing `mk-acker-bolt`. When a new tupletree is born, the spout sends the XORed edge-ids of each tuple recipient, which the acker records in its `pending` ledger. Every time an executor acks a tuple, the acker receives a partial checksum that is the XOR of the tuple's own edge-id (clearing it from the ledger) and the edge-id of each downstream tuple the executor emitted (thus entering them into the ledger). + +This is accomplished as follows. + +On a tick tuple, just advance pending tupletree checksums towards death and return. Otherwise, update or create the record for this tupletree: + +* on init: initialize with the given checksum value, and record the spout's id for later. +* on ack: xor the partial checksum into the existing checksum value +* on fail: just mark it as failed + +Next, [put the record](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L50)), into the RotatingMap (thus resetting is countdown to expiry) and take action: + +* if the total checksum is zero, the tupletree is complete: remove it from the pending collection and notify the spout of success +* if the tupletree has failed, it is also complete: remove it from the pending collection and notify the spout of failure + +Finally, pass on an ack of our own. + +### Pending tuples and the `RotatingMap` + +The acker stores pending tuples in a [`RotatingMap`](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/utils/RotatingMap.java#L19), a simple device used in several places within Storm to efficiently time-expire a process. + +The RotatingMap behaves as a HashMap, and offers the same O(1) access guarantees. + +Internally, it holds several HashMaps ('buckets') of its own, each holding a cohort of records that will expire at the same time. Let's call the longest-lived bucket death row, and the most recent the nursery. Whenever a value is `.put()` to the RotatingMap, it is relocated to the nursery -- and removed from any other bucket it might have been in (effectively resetting its death clock). + +Whenever its owner calls `.rotate()`, the RotatingMap advances each cohort one step further towards expiration. (Typically, Storm objects call rotate on every receipt of a system tick stream tuple.) If there are any key-value pairs in the former death row bucket, the RotatingMap invokes a callback (given in the constructor) for each key-value pair, letting its owner take appropriate action (eg, failing a tuple. + diff --git a/docs/Clojure-DSL.md b/docs/Clojure-DSL.md new file mode 100644 index 00000000000..b3109fafd97 --- /dev/null +++ b/docs/Clojure-DSL.md @@ -0,0 +1,264 @@ +--- +layout: documentation +--- +Storm comes with a Clojure DSL for defining spouts, bolts, and topologies. The Clojure DSL has access to everything the Java API exposes, so if you're a Clojure user you can code Storm topologies without touching Java at all. The Clojure DSL is defined in the source in the [backtype.storm.clojure](https://github.com/apache/incubator-storm/blob/0.5.3/src/clj/backtype/storm/clojure.clj) namespace. + +This page outlines all the pieces of the Clojure DSL, including: + +1. Defining topologies +2. `defbolt` +3. `defspout` +4. Running topologies in local mode or on a cluster +5. Testing topologies + +### Defining topologies + +To define a topology, use the `topology` function. `topology` takes in two arguments: a map of "spout specs" and a map of "bolt specs". Each spout and bolt spec wires the code for the component into the topology by specifying things like inputs and parallelism. + +Let's take a look at an example topology definition [from the storm-starter project](https://github.com/nathanmarz/storm-starter/blob/master/src/clj/storm/starter/clj/word_count.clj): + +```clojure +(topology + {"1" (spout-spec sentence-spout) + "2" (spout-spec (sentence-spout-parameterized + ["the cat jumped over the door" + "greetings from a faraway land"]) + :p 2)} + {"3" (bolt-spec {"1" :shuffle "2" :shuffle} + split-sentence + :p 5) + "4" (bolt-spec {"3" ["word"]} + word-count + :p 6)}) +``` + +The maps of spout and bolt specs are maps from the component id to the corresponding spec. The component ids must be unique across the maps. Just like defining topologies in Java, component ids are used when declaring inputs for bolts in the topology. + +#### spout-spec + +`spout-spec` takes as arguments the spout implementation (an object that implements [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html)) and optional keyword arguments. The only option that exists currently is the `:p` option, which specifies the parallelism for the spout. If you omit `:p`, the spout will execute as a single task. + +#### bolt-spec + +`bolt-spec` takes as arguments the input declaration for the bolt, the bolt implementation (an object that implements [IRichBolt](javadocs/backtype/storm/topology/IRichBolt.html)), and optional keyword arguments. + +The input declaration is a map from stream ids to stream groupings. A stream id can have one of two forms: + +1. `[==component id== ==stream id==]`: Subscribes to a specific stream on a component +2. `==component id==`: Subscribes to the default stream on a component + +A stream grouping can be one of the following: + +1. `:shuffle`: subscribes with a shuffle grouping +2. Vector of field names, like `["id" "name"]`: subscribes with a fields grouping on the specified fields +3. `:global`: subscribes with a global grouping +4. `:all`: subscribes with an all grouping +5. `:direct`: subscribes with a direct grouping + +See [Concepts](Concepts.html) for more info on stream groupings. Here's an example input declaration showcasing the various ways to declare inputs: + +```clojure +{["2" "1"] :shuffle + "3" ["field1" "field2"] + ["4" "2"] :global} +``` + +This input declaration subscribes to three streams total. It subscribes to stream "1" on component "2" with a shuffle grouping, subscribes to the default stream on component "3" with a fields grouping on the fields "field1" and "field2", and subscribes to stream "2" on component "4" with a global grouping. + +Like `spout-spec`, the only current supported keyword argument for `bolt-spec` is `:p` which specifies the parallelism for the bolt. + +#### shell-bolt-spec + +`shell-bolt-spec` is used for defining bolts that are implemented in a non-JVM language. It takes as arguments the input declaration, the command line program to run, the name of the file implementing the bolt, an output specification, and then the same keyword arguments that `bolt-spec` accepts. + +Here's an example `shell-bolt-spec`: + +```clojure +(shell-bolt-spec {"1" :shuffle "2" ["id"]} + "python" + "mybolt.py" + ["outfield1" "outfield2"] + :p 25) +``` + +The syntax of output declarations is described in more detail in the `defbolt` section below. See [Using non JVM languages with Storm](Using-non-JVM-languages-with-Storm.html) for more details on how multilang works within Storm. + +### defbolt + +`defbolt` is used for defining bolts in Clojure. Bolts have the constraint that they must be serializable, and this is why you can't just reify `IRichBolt` to implement a bolt (closures aren't serializable). `defbolt` works around this restriction and provides a nicer syntax for defining bolts than just implementing a Java interface. + +At its fullest expressiveness, `defbolt` supports parameterized bolts and maintaining state in a closure around the bolt implementation. It also provides shortcuts for defining bolts that don't need this extra functionality. The signature for `defbolt` looks like the following: + +(defbolt _name_ _output-declaration_ *_option-map_ & _impl_) + +Omitting the option map is equivalent to having an option map of `{:prepare false}`. + +#### Simple bolts + +Let's start with the simplest form of `defbolt`. Here's an example bolt that splits a tuple containing a sentence into a tuple for each word: + +```clojure +(defbolt split-sentence ["word"] [tuple collector] + (let [words (.split (.getString tuple 0) " ")] + (doseq [w words] + (emit-bolt! collector [w] :anchor tuple)) + (ack! collector tuple) + )) +``` + +Since the option map is omitted, this is a non-prepared bolt. The DSL simply expects an implementation for the `execute` method of `IRichBolt`. The implementation takes two parameters, the tuple and the `OutputCollector`, and is followed by the body of the `execute` function. The DSL automatically type-hints the parameters for you so you don't need to worry about reflection if you use Java interop. + +This implementation binds `split-sentence` to an actual `IRichBolt` object that you can use in topologies, like so: + +```clojure +(bolt-spec {"1" :shuffle} + split-sentence + :p 5) +``` + + +#### Parameterized bolts + +Many times you want to parameterize your bolts with other arguments. For example, let's say you wanted to have a bolt that appends a suffix to every input string it receives, and you want that suffix to be set at runtime. You do this with `defbolt` by including a `:params` option in the option map, like so: + +```clojure +(defbolt suffix-appender ["word"] {:params [suffix]} + [tuple collector] + (emit-bolt! collector [(str (.getString tuple 0) suffix)] :anchor tuple) + ) +``` + +Unlike the previous example, `suffix-appender` will be bound to a function that returns an `IRichBolt` rather than be an `IRichBolt` object directly. This is caused by specifying `:params` in its option map. So to use `suffix-appender` in a topology, you would do something like: + +```clojure +(bolt-spec {"1" :shuffle} + (suffix-appender "-suffix") + :p 10) +``` + +#### Prepared bolts + +To do more complex bolts, such as ones that do joins and streaming aggregations, the bolt needs to store state. You can do this by creating a prepared bolt which is specified by including `{:prepare true}` in the option map. Consider, for example, this bolt that implements word counting: + +```clojure +(defbolt word-count ["word" "count"] {:prepare true} + [conf context collector] + (let [counts (atom {})] + (bolt + (execute [tuple] + (let [word (.getString tuple 0)] + (swap! counts (partial merge-with +) {word 1}) + (emit-bolt! collector [word (@counts word)] :anchor tuple) + (ack! collector tuple) + ))))) +``` + +The implementation for a prepared bolt is a function that takes as input the topology config, `TopologyContext`, and `OutputCollector`, and returns an implementation of the `IBolt` interface. This design allows you to have a closure around the implementation of `execute` and `cleanup`. + +In this example, the word counts are stored in the closure in a map called `counts`. The `bolt` macro is used to create the `IBolt` implementation. The `bolt` macro is a more concise way to implement the interface than reifying, and it automatically type-hints all of the method parameters. This bolt implements the execute method which updates the count in the map and emits the new word count. + +Note that the `execute` method in prepared bolts only takes as input the tuple since the `OutputCollector` is already in the closure of the function (for simple bolts the collector is a second parameter to the `execute` function). + +Prepared bolts can be parameterized just like simple bolts. + +#### Output declarations + +The Clojure DSL has a concise syntax for declaring the outputs of a bolt. The most general way to declare the outputs is as a map from stream id a stream spec. For example: + +```clojure +{"1" ["field1" "field2"] + "2" (direct-stream ["f1" "f2" "f3"]) + "3" ["f1"]} +``` + +The stream id is a string, while the stream spec is either a vector of fields or a vector of fields wrapped by `direct-stream`. `direct stream` marks the stream as a direct stream (See [Concepts](Concepts.html) and [Direct groupings]() for more details on direct streams). + +If the bolt only has one output stream, you can define the default stream of the bolt by using a vector instead of a map for the output declaration. For example: + +```clojure +["word" "count"] +``` +This declares the output of the bolt as the fields ["word" "count"] on the default stream id. + +#### Emitting, acking, and failing + +Rather than use the Java methods on `OutputCollector` directly, the DSL provides a nicer set of functions for using `OutputCollector`: `emit-bolt!`, `emit-direct-bolt!`, `ack!`, and `fail!`. + +1. `emit-bolt!`: takes as parameters the `OutputCollector`, the values to emit (a Clojure sequence), and keyword arguments for `:anchor` and `:stream`. `:anchor` can be a single tuple or a list of tuples, and `:stream` is the id of the stream to emit to. Omitting the keyword arguments emits an unanchored tuple to the default stream. +2. `emit-direct-bolt!`: takes as parameters the `OutputCollector`, the task id to send the tuple to, the values to emit, and keyword arguments for `:anchor` and `:stream`. This function can only emit to streams declared as direct streams. +2. `ack!`: takes as parameters the `OutputCollector` and the tuple to ack. +3. `fail!`: takes as parameters the `OutputCollector` and the tuple to fail. + +See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more info on acking and anchoring. + +### defspout + +`defspout` is used for defining spouts in Clojure. Like bolts, spouts must be serializable so you can't just reify `IRichSpout` to do spout implementations in Clojure. `defspout` works around this restriction and provides a nicer syntax for defining spouts than just implementing a Java interface. + +The signature for `defspout` looks like the following: + +(defspout _name_ _output-declaration_ *_option-map_ & _impl_) + +If you leave out the option map, it defaults to {:prepare true}. The output declaration for `defspout` has the same syntax as `defbolt`. + +Here's an example `defspout` implementation from [storm-starter](https://github.com/nathanmarz/storm-starter/blob/master/src/clj/storm/starter/clj/word_count.clj): + +```clojure +(defspout sentence-spout ["sentence"] + [conf context collector] + (let [sentences ["a little brown dog" + "the man petted the dog" + "four score and seven years ago" + "an apple a day keeps the doctor away"]] + (spout + (nextTuple [] + (Thread/sleep 100) + (emit-spout! collector [(rand-nth sentences)]) + ) + (ack [id] + ;; You only need to define this method for reliable spouts + ;; (such as one that reads off of a queue like Kestrel) + ;; This is an unreliable spout, so it does nothing here + )))) +``` + +The implementation takes in as input the topology config, `TopologyContext`, and `SpoutOutputCollector`. The implementation returns an `ISpout` object. Here, the `nextTuple` function emits a random sentence from `sentences`. + +This spout isn't reliable, so the `ack` and `fail` methods will never be called. A reliable spout will add a message id when emitting tuples, and then `ack` or `fail` will be called when the tuple is completed or failed respectively. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more info on how reliability works within Storm. + +`emit-spout!` takes in as parameters the `SpoutOutputCollector` and the new tuple to be emitted, and accepts as keyword arguments `:stream` and `:id`. `:stream` specifies the stream to emit to, and `:id` specifies a message id for the tuple (used in the `ack` and `fail` callbacks). Omitting these arguments emits an unanchored tuple to the default output stream. + +There is also a `emit-direct-spout!` function that emits a tuple to a direct stream and takes an additional argument as the second parameter of the task id to send the tuple to. + +Spouts can be parameterized just like bolts, in which case the symbol is bound to a function returning `IRichSpout` instead of the `IRichSpout` itself. You can also declare an unprepared spout which only defines the `nextTuple` method. Here is an example of an unprepared spout that emits random sentences parameterized at runtime: + +```clojure +(defspout sentence-spout-parameterized ["word"] {:params [sentences] :prepare false} + [collector] + (Thread/sleep 500) + (emit-spout! collector [(rand-nth sentences)])) +``` + +The following example illustrates how to use this spout in a `spout-spec`: + +```clojure +(spout-spec (sentence-spout-parameterized + ["the cat jumped over the door" + "greetings from a faraway land"]) + :p 2) +``` + +### Running topologies in local mode or on a cluster + +That's all there is to the Clojure DSL. To submit topologies in remote mode or local mode, just use the `StormSubmitter` or `LocalCluster` classes just like you would from Java. + +To create topology configs, it's easiest to use the `backtype.storm.config` namespace which defines constants for all of the possible configs. The constants are the same as the static constants in the `Config` class, except with dashes instead of underscores. For example, here's a topology config that sets the number of workers to 15 and configures the topology in debug mode: + +```clojure +{TOPOLOGY-DEBUG true + TOPOLOGY-WORKERS 15} +``` + +### Testing topologies + +[This blog post](http://www.pixelmachine.org/2011/12/17/Testing-Storm-Topologies.html) and its [follow-up](http://www.pixelmachine.org/2011/12/21/Testing-Storm-Topologies-Part-2.html) give a good overview of Storm's powerful built-in facilities for testing topologies in Clojure. diff --git a/docs/Command-line-client.md b/docs/Command-line-client.md new file mode 100644 index 00000000000..0e645d74821 --- /dev/null +++ b/docs/Command-line-client.md @@ -0,0 +1,100 @@ +--- +layout: documentation +--- +This page describes all the commands that are possible with the "storm" command line client. To learn how to set up your "storm" client to talk to a remote cluster, follow the instructions in [Setting up development environment](Setting-up-a-development-environment.html). + +These commands are: + +1. jar +1. kill +1. activate +1. deactivate +1. rebalance +1. repl +1. classpath +1. localconfvalue +1. remoteconfvalue +1. nimbus +1. supervisor +1. ui +1. drpc + +### jar + +Syntax: `storm jar topology-jar-path class ...` + +Runs the main method of `class` with the specified arguments. The storm jars and configs in `~/.storm` are put on the classpath. The process is configured so that [StormSubmitter](javadocs/backtype/storm/StormSubmitter.html) will upload the jar at `topology-jar-path` when the topology is submitted. + +### kill + +Syntax: `storm kill topology-name [-w wait-time-secs]` + +Kills the topology with the name `topology-name`. Storm will first deactivate the topology's spouts for the duration of the topology's message timeout to allow all messages currently being processed to finish processing. Storm will then shutdown the workers and clean up their state. You can override the length of time Storm waits between deactivation and shutdown with the -w flag. + +### activate + +Syntax: `storm activate topology-name` + +Activates the specified topology's spouts. + +### deactivate + +Syntax: `storm deactivate topology-name` + +Deactivates the specified topology's spouts. + +### rebalance + +Syntax: `storm rebalance topology-name [-w wait-time-secs]` + +Sometimes you may wish to spread out where the workers for a topology are running. For example, let's say you have a 10 node cluster running 4 workers per node, and then let's say you add another 10 nodes to the cluster. You may wish to have Storm spread out the workers for the running topology so that each node runs 2 workers. One way to do this is to kill the topology and resubmit it, but Storm provides a "rebalance" command that provides an easier way to do this. + +Rebalance will first deactivate the topology for the duration of the message timeout (overridable with the -w flag) and then redistribute the workers evenly around the cluster. The topology will then return to its previous state of activation (so a deactivated topology will still be deactivated and an activated topology will go back to being activated). + +### repl + +Syntax: `storm repl` + +Opens up a Clojure REPL with the storm jars and configuration on the classpath. Useful for debugging. + +### classpath + +Syntax: `storm classpath` + +Prints the classpath used by the storm client when running commands. + +### localconfvalue + +Syntax: `storm localconfvalue conf-name` + +Prints out the value for `conf-name` in the local Storm configs. The local Storm configs are the ones in `~/.storm/storm.yaml` merged in with the configs in `defaults.yaml`. + +### remoteconfvalue + +Syntax: `storm remoteconfvalue conf-name` + +Prints out the value for `conf-name` in the cluster's Storm configs. The cluster's Storm configs are the ones in `$STORM-PATH/conf/storm.yaml` merged in with the configs in `defaults.yaml`. This command must be run on a cluster machine. + +### nimbus + +Syntax: `storm nimbus` + +Launches the nimbus daemon. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html) for more information. + +### supervisor + +Syntax: `storm supervisor` + +Launches the supervisor daemon. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html) for more information. + +### ui + +Syntax: `storm ui` + +Launches the UI daemon. The UI provides a web interface for a Storm cluster and shows detailed stats about running topologies. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html) for more information. + +### drpc + +Syntax: `storm drpc` + +Launches a DRPC daemon. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Distributed RPC](Distributed-RPC.html) for more information. diff --git a/docs/Common-patterns.md b/docs/Common-patterns.md new file mode 100644 index 00000000000..3f8c97971b6 --- /dev/null +++ b/docs/Common-patterns.md @@ -0,0 +1,86 @@ +--- +layout: documentation +--- + +This page lists a variety of common patterns in Storm topologies. + +1. Streaming joins +2. Batching +3. BasicBolt +4. In-memory caching + fields grouping combo +5. Streaming top N +6. TimeCacheMap for efficiently keeping a cache of things that have been recently updated +7. CoordinatedBolt and KeyedFairBolt for Distributed RPC + +### Joins + +A streaming join combines two or more data streams together based on some common field. Whereas a normal database join has finite input and clear semantics for a join, a streaming join has infinite input and unclear semantics for what a join should be. + +The join type you need will vary per application. Some applications join all tuples for two streams over a finite window of time, whereas other applications expect exactly one tuple for each side of the join for each join field. Other applications may do the join completely differently. The common pattern among all these join types is partitioning multiple input streams in the same way. This is easily accomplished in Storm by using a fields grouping on the same fields for many input streams to the joiner bolt. For example: + +```java +builder.setBolt("join", new MyJoiner(), parallelism) + .fieldsGrouping("1", new Fields("joinfield1", "joinfield2")) + .fieldsGrouping("2", new Fields("joinfield1", "joinfield2")) + .fieldsGrouping("3", new Fields("joinfield1", "joinfield2")); +``` + +The different streams don't have to have the same field names, of course. + + +### Batching + +Oftentimes for efficiency reasons or otherwise, you want to process a group of tuples in batch rather than individually. For example, you may want to batch updates to a database or do a streaming aggregation of some sort. + +If you want reliability in your data processing, the right way to do this is to hold on to tuples in an instance variable while the bolt waits to do the batching. Once you do the batch operation, you then ack all the tuples you were holding onto. + +If the bolt emits tuples, then you may want to use multi-anchoring to ensure reliability. It all depends on the specific application. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more details on how reliability works. + +### BasicBolt +Many bolts follow a similar pattern of reading an input tuple, emitting zero or more tuples based on that input tuple, and then acking that input tuple immediately at the end of the execute method. Bolts that match this pattern are things like functions and filters. This is such a common pattern that Storm exposes an interface called [IBasicBolt](javadocs/backtype/storm/topology/IBasicBolt.html) that automates this pattern for you. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more information. + +### In-memory caching + fields grouping combo + +It's common to keep caches in-memory in Storm bolts. Caching becomes particularly powerful when you combine it with a fields grouping. For example, suppose you have a bolt that expands short URLs (like bit.ly, t.co, etc.) into long URLs. You can increase performance by keeping an LRU cache of short URL to long URL expansions to avoid doing the same HTTP requests over and over. Suppose component "urls" emits short URLS, and component "expand" expands short URLs into long URLs and keeps a cache internally. Consider the difference between the two following snippets of code: + +```java +builder.setBolt("expand", new ExpandUrl(), parallelism) + .shuffleGrouping(1); +``` + +```java +builder.setBolt("expand", new ExpandUrl(), parallelism) + .fieldsGrouping("urls", new Fields("url")); +``` + +The second approach will have vastly more effective caches, since the same URL will always go to the same task. This avoids having duplication across any of the caches in the tasks and makes it much more likely that a short URL will hit the cache. + +### Streaming top N + +A common continuous computation done on Storm is a "streaming top N" of some sort. Suppose you have a bolt that emits tuples of the form ["value", "count"] and you want a bolt that emits the top N tuples based on count. The simplest way to do this is to have a bolt that does a global grouping on the stream and maintains a list in memory of the top N items. + +This approach obviously doesn't scale to large streams since the entire stream has to go through one task. A better way to do the computation is to do many top N's in parallel across partitions of the stream, and then merge those top N's together to get the global top N. The pattern looks like this: + +```java +builder.setBolt("rank", new RankObjects(), parallellism) + .fieldsGrouping("objects", new Fields("value")); +builder.setBolt("merge", new MergeObjects()) + .globalGrouping("rank"); +``` + +This pattern works because of the fields grouping done by the first bolt which gives the partitioning you need for this to be semantically correct. You can see an example of this pattern in storm-starter [here](https://github.com/nathanmarz/storm-starter/blob/master/src/jvm/storm/starter/RollingTopWords.java). + + +### TimeCacheMap for efficiently keeping a cache of things that have been recently updated + +You sometimes want to keep a cache in memory of items that have been recently "active" and have items that have been inactive for some time be automatically expires. [TimeCacheMap](javadocs/backtype/storm/utils/TimeCacheMap.html) is an efficient data structure for doing this and provides hooks so you can insert callbacks whenever an item is expired. + +### CoordinatedBolt and KeyedFairBolt for Distributed RPC + +When building distributed RPC applications on top of Storm, there are two common patterns that are usually needed. These are encapsulated by [CoordinatedBolt](javadocs/backtype/storm/task/CoordinatedBolt.html) and [KeyedFairBolt](javadocs/backtype/storm/task/KeyedFairBolt.html) which are part of the "standard library" that ships with the Storm codebase. + +`CoordinatedBolt` wraps the bolt containing your logic and figures out when your bolt has received all the tuples for any given request. It makes heavy use of direct streams to do this. + +`KeyedFairBolt` also wraps the bolt containing your logic and makes sure your topology processes multiple DRPC invocations at the same time, instead of doing them serially one at a time. + +See [Distributed RPC](Distributed-RPC.html) for more details. diff --git a/docs/Concepts.md b/docs/Concepts.md new file mode 100644 index 00000000000..33779f2bf8f --- /dev/null +++ b/docs/Concepts.md @@ -0,0 +1,115 @@ +--- +layout: documentation +--- + +This page lists the main concepts of Storm and links to resources where you can find more information. The concepts discussed are: + +1. Topologies +2. Streams +3. Spouts +4. Bolts +5. Stream groupings +6. Reliability +7. Tasks +8. Workers + +### Topologies + +The logic for a realtime application is packaged into a Storm topology. A Storm topology is analogous to a MapReduce job. One key difference is that a MapReduce job eventually finishes, whereas a topology runs forever (or until you kill it, of course). A topology is a graph of spouts and bolts that are connected with stream groupings. These concepts are described below. + +**Resources:** + +* [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html): use this class to construct topologies in Java +* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html) +* [Local mode](Local-mode.html): Read this to learn how to develop and test topologies in local mode. + +### Streams + +The stream is the core abstraction in Storm. A stream is an unbounded sequence of tuples that is processed and created in parallel in a distributed fashion. Streams are defined with a schema that names the fields in the stream's tuples. By default, tuples can contain integers, longs, shorts, bytes, strings, doubles, floats, booleans, and byte arrays. You can also define your own serializers so that custom types can be used natively within tuples. + +Every stream is given an id when declared. Since single-stream spouts and bolts are so common, [OutputFieldsDeclarer](javadocs/backtype/storm/topology/OutputFieldsDeclarer.html) has convenience methods for declaring a single stream without specifying an id. In this case, the stream is given the default id of "default". + + +**Resources:** + +* [Tuple](javadocs/backtype/storm/tuple/Tuple.html): streams are composed of tuples +* [OutputFieldsDeclarer](javadocs/backtype/storm/topology/OutputFieldsDeclarer.html): used to declare streams and their schemas +* [Serialization](Serialization.html): Information about Storm's dynamic typing of tuples and declaring custom serializations +* [ISerialization](javadocs/backtype/storm/serialization/ISerialization.html): custom serializers must implement this interface +* [CONFIG.TOPOLOGY_SERIALIZATIONS](javadocs/backtype/storm/Config.html#TOPOLOGY_SERIALIZATIONS): custom serializers can be registered using this configuration + +### Spouts + +A spout is a source of streams in a topology. Generally spouts will read tuples from an external source and emit them into the topology (e.g. a Kestrel queue or the Twitter API). Spouts can either be __reliable__ or __unreliable__. A reliable spout is capable of replaying a tuple if it failed to be processed by Storm, whereas an unreliable spout forgets about the tuple as soon as it is emitted. + +Spouts can emit more than one stream. To do so, declare multiple streams using the `declareStream` method of [OutputFieldsDeclarer](javadocs/backtype/storm/topology/OutputFieldsDeclarer.html) and specify the stream to emit to when using the `emit` method on [SpoutOutputCollector](javadocs/backtype/storm/spout/SpoutOutputCollector.html). + +The main method on spouts is `nextTuple`. `nextTuple` either emits a new tuple into the topology or simply returns if there are no new tuples to emit. It is imperative that `nextTuple` does not block for any spout implementation, because Storm calls all the spout methods on the same thread. + +The other main methods on spouts are `ack` and `fail`. These are called when Storm detects that a tuple emitted from the spout either successfully completed through the topology or failed to be completed. `ack` and `fail` are only called for reliable spouts. See [the Javadoc](javadocs/backtype/storm/spout/ISpout.html) for more information. + +**Resources:** + +* [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html): this is the interface that spouts must implement. +* [Guaranteeing message processing](Guaranteeing-message-processing.html) + +### Bolts + +All processing in topologies is done in bolts. Bolts can do anything from filtering, functions, aggregations, joins, talking to databases, and more. + +Bolts can do simple stream transformations. Doing complex stream transformations often requires multiple steps and thus multiple bolts. For example, transforming a stream of tweets into a stream of trending images requires at least two steps: a bolt to do a rolling count of retweets for each image, and one or more bolts to stream out the top X images (you can do this particular stream transformation in a more scalable way with three bolts than with two). + +Bolts can emit more than one stream. To do so, declare multiple streams using the `declareStream` method of [OutputFieldsDeclarer](javadocs/backtype/storm/topology/OutputFieldsDeclarer.html) and specify the stream to emit to when using the `emit` method on [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html). + +When you declare a bolt's input streams, you always subscribe to specific streams of another component. If you want to subscribe to all the streams of another component, you have to subscribe to each one individually. [InputDeclarer](javadocs/backtype/storm/topology/InputDeclarer.html) has syntactic sugar for subscribing to streams declared on the default stream id. Saying `declarer.shuffleGrouping("1")` subscribes to the default stream on component "1" and is equivalent to `declarer.shuffleGrouping("1", DEFAULT_STREAM_ID)`. + +The main method in bolts is the `execute` method which takes in as input a new tuple. Bolts emit new tuples using the [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html) object. Bolts must call the `ack` method on the `OutputCollector` for every tuple they process so that Storm knows when tuples are completed (and can eventually determine that its safe to ack the original spout tuples). For the common case of processing an input tuple, emitting 0 or more tuples based on that tuple, and then acking the input tuple, Storm provides an [IBasicBolt](javadocs/backtype/storm/topology/IBasicBolt.html) interface which does the acking automatically. + +Its perfectly fine to launch new threads in bolts that do processing asynchronously. [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html) is thread-safe and can be called at any time. + +**Resources:** + +* [IRichBolt](javadocs/backtype/storm/topology/IRichBolt.html): this is general interface for bolts. +* [IBasicBolt](javadocs/backtype/storm/topology/IBasicBolt.html): this is a convenience interface for defining bolts that do filtering or simple functions. +* [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html): bolts emit tuples to their output streams using an instance of this class +* [Guaranteeing message processing](Guaranteeing-message-processing.html) + +### Stream groupings + +Part of defining a topology is specifying for each bolt which streams it should receive as input. A stream grouping defines how that stream should be partitioned among the bolt's tasks. + +There are seven built-in stream groupings in Storm, and you can implement a custom stream grouping by implementing the [CustomStreamGrouping](javadocs/backtype/storm/grouping/CustomStreamGrouping.html) interface: + +1. **Shuffle grouping**: Tuples are randomly distributed across the bolt's tasks in a way such that each bolt is guaranteed to get an equal number of tuples. +2. **Fields grouping**: The stream is partitioned by the fields specified in the grouping. For example, if the stream is grouped by the "user-id" field, tuples with the same "user-id" will always go to the same task, but tuples with different "user-id"'s may go to different tasks. +3. **All grouping**: The stream is replicated across all the bolt's tasks. Use this grouping with care. +4. **Global grouping**: The entire stream goes to a single one of the bolt's tasks. Specifically, it goes to the task with the lowest id. +5. **None grouping**: This grouping specifies that you don't care how the stream is grouped. Currently, none groupings are equivalent to shuffle groupings. Eventually though, Storm will push down bolts with none groupings to execute in the same thread as the bolt or spout they subscribe from (when possible). +6. **Direct grouping**: This is a special kind of grouping. A stream grouped this way means that the __producer__ of the tuple decides which task of the consumer will receive this tuple. Direct groupings can only be declared on streams that have been declared as direct streams. Tuples emitted to a direct stream must be emitted using one of the [emitDirect](javadocs/backtype/storm/task/OutputCollector.html#emitDirect(int, int, java.util.List) methods. A bolt can get the task ids of its consumers by either using the provided [TopologyContext](javadocs/backtype/storm/task/TopologyContext.html) or by keeping track of the output of the `emit` method in [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html) (which returns the task ids that the tuple was sent to). +7. **Local or shuffle grouping**: If the target bolt has one or more tasks in the same worker process, tuples will be shuffled to just those in-process tasks. Otherwise, this acts like a normal shuffle grouping. + +**Resources:** + +* [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html): use this class to define topologies +* [InputDeclarer](javadocs/backtype/storm/topology/InputDeclarer.html): this object is returned whenever `setBolt` is called on `TopologyBuilder` and is used for declaring a bolt's input streams and how those streams should be grouped +* [CoordinatedBolt](javadocs/backtype/storm/task/CoordinatedBolt.html): this bolt is useful for distributed RPC topologies and makes heavy use of direct streams and direct groupings + +### Reliability + +Storm guarantees that every spout tuple will be fully processed by the topology. It does this by tracking the tree of tuples triggered by every spout tuple and determining when that tree of tuples has been successfully completed. Every topology has a "message timeout" associated with it. If Storm fails to detect that a spout tuple has been completed within that timeout, then it fails the tuple and replays it later. + +To take advantage of Storm's reliability capabilities, you must tell Storm when new edges in a tuple tree are being created and tell Storm whenever you've finished processing an individual tuple. These are done using the [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html) object that bolts use to emit tuples. Anchoring is done in the `emit` method, and you declare that you're finished with a tuple using the `ack` method. + +This is all explained in much more detail in [Guaranteeing message processing](Guaranteeing-message-processing.html). + +### Tasks + +Each spout or bolt executes as many tasks across the cluster. Each task corresponds to one thread of execution, and stream groupings define how to send tuples from one set of tasks to another set of tasks. You set the parallelism for each spout or bolt in the `setSpout` and `setBolt` methods of [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html). + +### Workers + +Topologies execute across one or more worker processes. Each worker process is a physical JVM and executes a subset of all the tasks for the topology. For example, if the combined parallelism of the topology is 300 and 50 workers are allocated, then each worker will execute 6 tasks (as threads within the worker). Storm tries to spread the tasks evenly across all the workers. + +**Resources:** + +* [Config.TOPOLOGY_WORKERS](javadocs/backtype/storm/Config.html#TOPOLOGY_WORKERS): this config sets the number of workers to allocate for executing the topology diff --git a/docs/Configuration.md b/docs/Configuration.md new file mode 100644 index 00000000000..8e8ca776916 --- /dev/null +++ b/docs/Configuration.md @@ -0,0 +1,29 @@ +--- +layout: documentation +--- +Storm has a variety of configurations for tweaking the behavior of nimbus, supervisors, and running topologies. Some configurations are system configurations and cannot be modified on a topology by topology basis, whereas other configurations can be modified per topology. + +Every configuration has a default value defined in [defaults.yaml](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml) in the Storm codebase. You can override these configurations by defining a storm.yaml in the classpath of Nimbus and the supervisors. Finally, you can define a topology-specific configuration that you submit along with your topology when using [StormSubmitter](javadocs/backtype/storm/StormSubmitter.html). However, the topology-specific configuration can only override configs prefixed with "TOPOLOGY". + +Storm 0.7.0 and onwards lets you override configuration on a per-bolt/per-spout basis. The only configurations that can be overriden this way are: + +1. "topology.debug" +2. "topology.max.spout.pending" +3. "topology.max.task.parallelism" +4. "topology.kryo.register": This works a little bit differently than the other ones, since the serializations will be available to all components in the topology. More details on [Serialization](Serialization.html). + +The Java API lets you specify component specific configurations in two ways: + +1. *Internally:* Override `getComponentConfiguration` in any spout or bolt and return the component-specific configuration map. +2. *Externally:* `setSpout` and `setBolt` in `TopologyBuilder` return an object with methods `addConfiguration` and `addConfigurations` that can be used to override the configurations for the component. + +The preference order for configuration values is defaults.yaml < storm.yaml < topology specific configuration < internal component specific configuration < external component specific configuration. + + +**Resources:** + +* [Config](javadocs/backtype/storm/Config.html): a listing of all configurations as well as a helper class for creating topology specific configurations +* [defaults.yaml](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml): the default values for all configurations +* [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html): explains how to create and configure a Storm cluster +* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html): lists useful configurations when running topologies on a cluster +* [Local mode](Local-mode.html): lists useful configurations when using local mode diff --git a/docs/Contributing-to-Storm.md b/docs/Contributing-to-Storm.md new file mode 100644 index 00000000000..dff23fb51cf --- /dev/null +++ b/docs/Contributing-to-Storm.md @@ -0,0 +1,31 @@ +--- +layout: documentation +--- + +### Getting started with contributing + +Some of the issues on the [issue tracker](https://issues.apache.org/jira/browse/STORM) are marked with the "Newbie" label. If you're interesting in contributing to Storm but don't know where to begin, these are good issues to start with. These issues are a great way to get your feet wet with learning the codebase because they require learning about only an isolated portion of the codebase and are a relatively small amount of work. + +### Learning the codebase + +The [Implementation docs](Implementation-docs.html) section of the wiki gives detailed walkthroughs of the codebase. Reading through these docs is highly recommended to understand the codebase. + +### Contribution process + +Contributions to the Storm codebase should be sent as GitHub pull requests. If there's any problems to the pull request we can iterate on it using GitHub's commenting features. + +For small patches, feel free to submit pull requests directly for them. For larger contributions, please use the following process. The idea behind this process is to prevent any wasted work and catch design issues early on: + +1. Open an issue on the [issue tracker](https://issues.apache.org/jira/browse/STORM) if one doesn't exist already +2. Comment on the issue with your plan for implementing the issue. Explain what pieces of the codebase you're going to touch and how everything is going to fit together. +3. Storm committers will iterate with you on the design to make sure you're on the right track +4. Implement your issue, submit a pull request, and iterate from there. + +### Modules built on top of Storm + +Modules built on top of Storm (like spouts, bolts, etc) that aren't appropriate for Storm core can be done as your own project or as part of [@stormprocessor](https://github.com/stormprocessor). To be part of @stormprocessor put your project on your own Github and then send an email to the mailing list proposing to make it part of @stormprocessor. Then the community can discuss whether it's useful enough to be part of @stormprocessor. Then you'll be added to the @stormprocessor organization and can maintain your project there. The advantage of hosting your module in @stormprocessor is that it will be easier for potential users to find your project. + +### Contributing documentation + +Documentation contributions are very welcome! The best way to send contributions is as emails through the mailing list. + diff --git a/docs/Creating-a-new-Storm-project.md b/docs/Creating-a-new-Storm-project.md new file mode 100644 index 00000000000..feb49b8d0d5 --- /dev/null +++ b/docs/Creating-a-new-Storm-project.md @@ -0,0 +1,25 @@ +--- +layout: documentation +--- +This page outlines how to set up a Storm project for development. The steps are: + +1. Add Storm jars to classpath +2. If using multilang, add multilang dir to classpath + +Follow along to see how to set up the [storm-starter](http://github.com/nathanmarz/storm-starter) project in Eclipse. + +### Add Storm jars to classpath + +You'll need the Storm jars on your classpath to develop Storm topologies. Using [Maven](Maven.html) is highly recommended. [Here's an example](https://github.com/nathanmarz/storm-starter/blob/master/m2-pom.xml) of how to setup your pom.xml for a Storm project. If you don't want to use Maven, you can include the jars from the Storm release on your classpath. + +[storm-starter](http://github.com/nathanmarz/storm-starter) uses [Leiningen](http://github.com/technomancy/leiningen) for build and dependency resolution. You can install leiningen by downloading [this script](https://raw.github.com/technomancy/leiningen/stable/bin/lein), placing it on your path, and making it executable. To retrieve the dependencies for Storm, simply run `lein deps` in the project root. + +To set up the classpath in Eclipse, create a new Java project, include `src/jvm/` as a source path, and make sure all the jars in `lib/` and `lib/dev/` are in the `Referenced Libraries` section of the project. + +### If using multilang, add multilang dir to classpath + +If you implement spouts or bolts in languages other than Java, then those implementations should be under the `multilang/resources/` directory of the project. For Storm to find these files in local mode, the `resources/` dir needs to be on the classpath. You can do this in Eclipse by adding `multilang/` as a source folder. You may also need to add multilang/resources as a source directory. + +For more information on writing topologies in other languages, see [Using non-JVM languages with Storm](Using-non-JVM-languages-with-Storm.html). + +To test that everything is working in Eclipse, you should now be able to `Run` the `WordCountTopology.java` file. You will see messages being emitted at the console for 10 seconds. diff --git a/docs/DSLs-and-multilang-adapters.md b/docs/DSLs-and-multilang-adapters.md new file mode 100644 index 00000000000..31bd453f75b --- /dev/null +++ b/docs/DSLs-and-multilang-adapters.md @@ -0,0 +1,9 @@ +--- +layout: documentation +--- +* [Scala DSL](https://github.com/velvia/ScalaStorm) +* [JRuby DSL](https://github.com/colinsurprenant/redstorm) +* [Clojure DSL](Clojure-DSL.html) +* [Storm/Esper integration](https://github.com/tomdz/storm-esper): Streaming SQL on top of Storm +* [io-storm](https://github.com/gphat/io-storm): Perl multilang adapter +* [storm-php](https://github.com/lazyshot/storm-php): PHP multilang adapter diff --git a/docs/Defining-a-non-jvm-language-dsl-for-storm.md b/docs/Defining-a-non-jvm-language-dsl-for-storm.md new file mode 100644 index 00000000000..f52f4abe7cd --- /dev/null +++ b/docs/Defining-a-non-jvm-language-dsl-for-storm.md @@ -0,0 +1,36 @@ +--- +layout: documentation +--- +The right place to start to learn how to make a non-JVM DSL for Storm is [storm-core/src/storm.thrift](https://github.com/apache/incubator-storm/blob/master/storm-core/src/storm.thrift). Since Storm topologies are just Thrift structures, and Nimbus is a Thrift daemon, you can create and submit topologies in any language. + +When you create the Thrift structs for spouts and bolts, the code for the spout or bolt is specified in the ComponentObject struct: + +``` +union ComponentObject { + 1: binary serialized_java; + 2: ShellComponent shell; + 3: JavaObject java_object; +} +``` + +For a Python DSL, you would want to make use of "2" and "3". ShellComponent lets you specify a script to run that component (e.g., your python code). And JavaObject lets you specify native java spouts and bolts for the component (and Storm will use reflection to create that spout or bolt). + +There's a "storm shell" command that will help with submitting a topology. Its usage is like this: + +``` +storm shell resources/ python topology.py arg1 arg2 +``` + +storm shell will then package resources/ into a jar, upload the jar to Nimbus, and call your topology.py script like this: + +``` +python topology.py arg1 arg2 {nimbus-host} {nimbus-port} {uploaded-jar-location} +``` + +Then you can connect to Nimbus using the Thrift API and submit the topology, passing {uploaded-jar-location} into the submitTopology method. For reference, here's the submitTopology definition: + +```java +void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite); +``` + +Finally, one of the key things to do in a non-JVM DSL is make it easy to define the entire topology in one file (the bolts, spouts, and the definition of the topology). diff --git a/docs/Distributed-RPC.md b/docs/Distributed-RPC.md new file mode 100644 index 00000000000..fc75ee4fb3d --- /dev/null +++ b/docs/Distributed-RPC.md @@ -0,0 +1,197 @@ +--- +layout: documentation +--- +The idea behind distributed RPC (DRPC) is to parallelize the computation of really intense functions on the fly using Storm. The Storm topology takes in as input a stream of function arguments, and it emits an output stream of the results for each of those function calls. + +DRPC is not so much a feature of Storm as it is a pattern expressed from Storm's primitives of streams, spouts, bolts, and topologies. DRPC could have been packaged as a separate library from Storm, but it's so useful that it's bundled with Storm. + +### High level overview + +Distributed RPC is coordinated by a "DRPC server" (Storm comes packaged with an implementation of this). The DRPC server coordinates receiving an RPC request, sending the request to the Storm topology, receiving the results from the Storm topology, and sending the results back to the waiting client. From a client's perspective, a distributed RPC call looks just like a regular RPC call. For example, here's how a client would compute the results for the "reach" function with the argument "http://twitter.com": + +```java +DRPCClient client = new DRPCClient("drpc-host", 3772); +String result = client.execute("reach", "http://twitter.com"); +``` + +The distributed RPC workflow looks like this: + +![Tasks in a topology](images/drpc-workflow.png) + +A client sends the DRPC server the name of the function to execute and the arguments to that function. The topology implementing that function uses a `DRPCSpout` to receive a function invocation stream from the DRPC server. Each function invocation is tagged with a unique id by the DRPC server. The topology then computes the result and at the end of the topology a bolt called `ReturnResults` connects to the DRPC server and gives it the result for the function invocation id. The DRPC server then uses the id to match up that result with which client is waiting, unblocks the waiting client, and sends it the result. + +### LinearDRPCTopologyBuilder + +Storm comes with a topology builder called [LinearDRPCTopologyBuilder](javadocs/backtype/storm/drpc/LinearDRPCTopologyBuilder.html) that automates almost all the steps involved for doing DRPC. These include: + +1. Setting up the spout +2. Returning the results to the DRPC server +3. Providing functionality to bolts for doing finite aggregations over groups of tuples + +Let's look at a simple example. Here's the implementation of a DRPC topology that returns its input argument with a "!" appended: + +```java +public static class ExclaimBolt extends BaseBasicBolt { + public void execute(Tuple tuple, BasicOutputCollector collector) { + String input = tuple.getString(1); + collector.emit(new Values(tuple.getValue(0), input + "!")); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "result")); + } +} + +public static void main(String[] args) throws Exception { + LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("exclamation"); + builder.addBolt(new ExclaimBolt(), 3); + // ... +} +``` + +As you can see, there's very little to it. When creating the `LinearDRPCTopologyBuilder`, you tell it the name of the DRPC function for the topology. A single DRPC server can coordinate many functions, and the function name distinguishes the functions from one another. The first bolt you declare will take in as input 2-tuples, where the first field is the request id and the second field is the arguments for that request. `LinearDRPCTopologyBuilder` expects the last bolt to emit an output stream containing 2-tuples of the form [id, result]. Finally, all intermediate tuples must contain the request id as the first field. + +In this example, `ExclaimBolt` simply appends a "!" to the second field of the tuple. `LinearDRPCTopologyBuilder` handles the rest of the coordination of connecting to the DRPC server and sending results back. + +### Local mode DRPC + +DRPC can be run in local mode. Here's how to run the above example in local mode: + +```java +LocalDRPC drpc = new LocalDRPC(); +LocalCluster cluster = new LocalCluster(); + +cluster.submitTopology("drpc-demo", conf, builder.createLocalTopology(drpc)); + +System.out.println("Results for 'hello':" + drpc.execute("exclamation", "hello")); + +cluster.shutdown(); +drpc.shutdown(); +``` + +First you create a `LocalDRPC` object. This object simulates a DRPC server in process, just like how `LocalCluster` simulates a Storm cluster in process. Then you create the `LocalCluster` to run the topology in local mode. `LinearDRPCTopologyBuilder` has separate methods for creating local topologies and remote topologies. In local mode the `LocalDRPC` object does not bind to any ports so the topology needs to know about the object to communicate with it. This is why `createLocalTopology` takes in the `LocalDRPC` object as input. + +After launching the topology, you can do DRPC invocations using the `execute` method on `LocalDRPC`. + +### Remote mode DRPC + +Using DRPC on an actual cluster is also straightforward. There's three steps: + +1. Launch DRPC server(s) +2. Configure the locations of the DRPC servers +3. Submit DRPC topologies to Storm cluster + +Launching a DRPC server can be done with the `storm` script and is just like launching Nimbus or the UI: + +``` +bin/storm drpc +``` + +Next, you need to configure your Storm cluster to know the locations of the DRPC server(s). This is how `DRPCSpout` knows from where to read function invocations. This can be done through the `storm.yaml` file or the topology configurations. Configuring this through the `storm.yaml` looks something like this: + +```yaml +drpc.servers: + - "drpc1.foo.com" + - "drpc2.foo.com" +``` + +Finally, you launch DRPC topologies using `StormSubmitter` just like you launch any other topology. To run the above example in remote mode, you do something like this: + +```java +StormSubmitter.submitTopology("exclamation-drpc", conf, builder.createRemoteTopology()); +``` + +`createRemoteTopology` is used to create topologies suitable for Storm clusters. + +### A more complex example + +The exclamation DRPC example was a toy example for illustrating the concepts of DRPC. Let's look at a more complex example which really needs the parallelism a Storm cluster provides for computing the DRPC function. The example we'll look at is computing the reach of a URL on Twitter. + +The reach of a URL is the number of unique people exposed to a URL on Twitter. To compute reach, you need to: + +1. Get all the people who tweeted the URL +2. Get all the followers of all those people +3. Unique the set of followers +4. Count the unique set of followers + +A single reach computation can involve thousands of database calls and tens of millions of follower records during the computation. It's a really, really intense computation. As you're about to see, implementing this function on top of Storm is dead simple. On a single machine, reach can take minutes to compute; on a Storm cluster, you can compute reach for even the hardest URLs in a couple seconds. + +A sample reach topology is defined in storm-starter [here](https://github.com/nathanmarz/storm-starter/blob/master/src/jvm/storm/starter/ReachTopology.java). Here's how you define the reach topology: + +```java +LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("reach"); +builder.addBolt(new GetTweeters(), 3); +builder.addBolt(new GetFollowers(), 12) + .shuffleGrouping(); +builder.addBolt(new PartialUniquer(), 6) + .fieldsGrouping(new Fields("id", "follower")); +builder.addBolt(new CountAggregator(), 2) + .fieldsGrouping(new Fields("id")); +``` + +The topology executes as four steps: + +1. `GetTweeters` gets the users who tweeted the URL. It transforms an input stream of `[id, url]` into an output stream of `[id, tweeter]`. Each `url` tuple will map to many `tweeter` tuples. +2. `GetFollowers` gets the followers for the tweeters. It transforms an input stream of `[id, tweeter]` into an output stream of `[id, follower]`. Across all the tasks, there may of course be duplication of follower tuples when someone follows multiple people who tweeted the same URL. +3. `PartialUniquer` groups the followers stream by the follower id. This has the effect of the same follower going to the same task. So each task of `PartialUniquer` will receive mutually independent sets of followers. Once `PartialUniquer` receives all the follower tuples directed at it for the request id, it emits the unique count of its subset of followers. +4. Finally, `CountAggregator` receives the partial counts from each of the `PartialUniquer` tasks and sums them up to complete the reach computation. + +Let's take a look at the `PartialUniquer` bolt: + +```java +public class PartialUniquer extends BaseBatchBolt { + BatchOutputCollector _collector; + Object _id; + Set _followers = new HashSet(); + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + _followers.add(tuple.getString(1)); + } + + @Override + public void finishBatch() { + _collector.emit(new Values(_id, _followers.size())); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "partial-count")); + } +} +``` + +`PartialUniquer` implements `IBatchBolt` by extending `BaseBatchBolt`. A batch bolt provides a first class API to processing a batch of tuples as a concrete unit. A new instance of the batch bolt is created for each request id, and Storm takes care of cleaning up the instances when appropriate. + +When `PartialUniquer` receives a follower tuple in the `execute` method, it adds it to the set for the request id in an internal `HashSet`. + +Batch bolts provide the `finishBatch` method which is called after all the tuples for this batch targeted at this task have been processed. In the callback, `PartialUniquer` emits a single tuple containing the unique count for its subset of follower ids. + +Under the hood, `CoordinatedBolt` is used to detect when a given bolt has received all of the tuples for any given request id. `CoordinatedBolt` makes use of direct streams to manage this coordination. + +The rest of the topology should be self-explanatory. As you can see, every single step of the reach computation is done in parallel, and defining the DRPC topology was extremely simple. + +### Non-linear DRPC topologies + +`LinearDRPCTopologyBuilder` only handles "linear" DRPC topologies, where the computation is expressed as a sequence of steps (like reach). It's not hard to imagine functions that would require a more complicated topology with branching and merging of the bolts. For now, to do this you'll need to drop down into using `CoordinatedBolt` directly. Be sure to talk about your use case for non-linear DRPC topologies on the mailing list to inform the construction of more general abstractions for DRPC topologies. + +### How LinearDRPCTopologyBuilder works + +* DRPCSpout emits [args, return-info]. return-info is the host and port of the DRPC server as well as the id generated by the DRPC server +* constructs a topology comprising of: + * DRPCSpout + * PrepareRequest (generates a request id and creates a stream for the return info and a stream for the args) + * CoordinatedBolt wrappers and direct groupings + * JoinResult (joins the result with the return info) + * ReturnResult (connects to the DRPC server and returns the result) +* LinearDRPCTopologyBuilder is a good example of a higher level abstraction built on top of Storm's primitives + +### Advanced +* KeyedFairBolt for weaving the processing of multiple requests at the same time +* How to use `CoordinatedBolt` directly diff --git a/docs/Documentation.md b/docs/Documentation.md new file mode 100644 index 00000000000..8da874c3266 --- /dev/null +++ b/docs/Documentation.md @@ -0,0 +1,50 @@ +--- +layout: documentation +--- +### Basics of Storm + +* [Javadoc](javadocs/index.html) +* [Concepts](Concepts.html) +* [Configuration](Configuration.html) +* [Guaranteeing message processing](Guaranteeing-message-processing.html) +* [Fault-tolerance](Fault-tolerance.html) +* [Command line client](Command-line-client.html) +* [Understanding the parallelism of a Storm topology](Understanding-the-parallelism-of-a-Storm-topology.html) +* [FAQ](FAQ.html) + +### Trident + +Trident is an alternative interface to Storm. It provides exactly-once processing, "transactional" datastore persistence, and a set of common stream analytics operations. + +* [Trident Tutorial](Trident-tutorial.html) -- basic concepts and walkthrough +* [Trident API Overview](Trident-API-Overview.html) -- operations for transforming and orchestrating data +* [Trident State](Trident-state.html) -- exactly-once processing and fast, persistent aggregation +* [Trident spouts](Trident-spouts.html) -- transactional and non-transactional data intake + +### Setup and deploying + +* [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html) +* [Local mode](Local-mode.html) +* [Troubleshooting](Troubleshooting.html) +* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html) +* [Building Storm](Maven.html) with Maven + +### Intermediate + +* [Serialization](Serialization.html) +* [Common patterns](Common-patterns.html) +* [Clojure DSL](Clojure-DSL.html) +* [Using non-JVM languages with Storm](Using-non-JVM-languages-with-Storm.html) +* [Distributed RPC](Distributed-RPC.html) +* [Transactional topologies](Transactional-topologies.html) +* [Kestrel and Storm](Kestrel-and-Storm.html) +* [Direct groupings](Direct-groupings.html) +* [Hooks](Hooks.html) +* [Metrics](Metrics.html) +* [Lifecycle of a trident tuple]() + +### Advanced + +* [Defining a non-JVM language DSL for Storm](Defining-a-non-jvm-language-dsl-for-storm.html) +* [Multilang protocol](Multilang-protocol.html) (how to provide support for another language) +* [Implementation docs](Implementation-docs.html) diff --git a/docs/FAQ.md b/docs/FAQ.md new file mode 100644 index 00000000000..8ff7a6fcb56 --- /dev/null +++ b/docs/FAQ.md @@ -0,0 +1,121 @@ +--- +layout: documentation +--- + +## Best Practices + +### What rules of thumb can you give me for configuring Storm+Trident? + +* number of workers a multiple of number of machines; parallelism a multiple of number of workers; number of kafka partitions a multiple of number of spout parallelism +* Use one worker per topology per machine +* Start with fewer, larger aggregators, one per machine with workers on it +* Use the isolation scheduler +* Use one acker per worker -- 0.9 makes that the default, but earlier versions do not. +* enable GC logging; you should see very few major GCs if things are in reasonable shape. +* set the trident batch millis to about 50% of your typical end-to-end latency. +* Start with a max spout pending that is for sure too small -- one for trident, or the number of executors for storm -- and increase it until you stop seeing changes in the flow. You'll probably end up with something near `2*(throughput in recs/sec)*(end-to-end latency)` (2x the Little's law capacity). + +### What are some of the best ways to get a worker to mysteriously and bafflingly die? + +* Do you have write access to the log directory +* Are you blowing out your heap? +* Are all the right libraries installed on all of the workers? +* Is the zookeeper hostname still set to localhost? +* Did you supply a correct, unique hostname -- one that resolves back to the machine -- to each worker, and put it in the storm conf file? +* Have you opened firewall/securitygroup permissions _bidirectionally_ among a) all the workers, b) the storm master, c) zookeeper? Also, from the workers to any kafka/kestrel/database/etc that your topology accesses? Use netcat to poke the appropriate ports and be sure. + +### Halp! I cannot see: + +* **my logs** Logs by default go to $STORM_HOME/logs. Check that you have write permissions to that directory. They are configured in the logback/cluster.xml (0.9) and log4j/*.properties in earlier versions. +* **final JVM settings** Add the `-XX+PrintFlagsFinal` commandline option in the childopts (see the conf file) +* **final Java system properties** Add `Properties props = System.getProperties(); props.list(System.out);` near where you build your topology. + +### How many Workers should I use? + +The total number of workers is set by the supervisors -- there's some number of JVM slots each supervisor will superintend. The thing you set on the topology is how many worker slots it will try to claim. + +There's no great reason to use more than one worker per topology per machine. + +With one topology running on three 8-core nodes, and parallelism hint 24, each bolt gets 8 executors per machine, i.e. one for each core. There are three big benefits to running three workers (with 8 assigned executors each) compare to running say 24 workers (one assigned executor each). + +First, data that is repartitioned (shuffles or group-bys) to executors in the same worker will not have to hit the transfer buffer. Instead, tuples are deposited directly from send to receive buffer. That's a big win. By contrast, if the destination executor were on the same machine in a different worker, it would have to go send -> worker transfer -> local socket -> worker recv -> exec recv buffer. It doesn't hit the network card, but it's not as big a win as when executors are in the same worker. + +Second, you're typically better off with three aggregators having very large backing cache than having twenty-four aggregators having small backing caches. This reduces the effect of skew, and improves LRU efficiency. + +Lastly, fewer workers reduces control flow chatter. + +## Topology + +### Can a Trident topology have Multiple Streams? + +> Can a Trident Topology work like a workflow with conditional paths (if-else)? e.g. A Spout (S1) connects to a bolt (B0) which based on certain values in the incoming tuple routes them to either bolt (B1) or bolt (B2) but not both. + +A Trident "each" operator returns a Stream object, which you can store in a variable. You can then run multiple eaches on the same Stream to split it, e.g.: + + Stream s = topology.each(...).groupBy(...).aggregate(...) + Stream branch1 = s.each(..., FilterA) + Stream branch2 = s.each(..., FilterB) + +You can join streams with join, merge or multiReduce. + +At time of writing, you can't emit to multiple output streams from Trident -- see [STORM-68](https://issues.apache.org/jira/browse/STORM-68) + +## Spouts + +### What is a coordinator, and why are there several? + +A trident-spout is actually run within a storm _bolt_. The storm-spout of a trident topology is the MasterBatchCoordinator -- it coordinates trident batches and is the same no matter what spouts you use. A batch is born when the MBC dispenses a seed tuple to each of the spout-coordinators. The spout-coordinator bolts know how your particular spouts should cooperate -- so in the kafka case, it's what helps figure out what partition and offset range each spout should pull from. + +### What can I store into the spout's metadata record? + +You should only store static data, and as little of it as possible, into the metadata record (note: maybe you _can_ store more interesting things; you shouldn't, though) + +### How often is the 'emitPartitionBatchNew' function called? + +Since the MBC is the actual spout, all the tuples in a batch are just members of its tupletree. That means storm's "max spout pending" config effectively defines the number of concurrent batches trident runs. The MBC emits a new batch if it has fewer than max-spending tuples pending and if at least one [trident batch interval](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml#L115)'s worth of seconds has passed since the last batch. + +### If nothing was emitted does Trident slow down the calls? + +Yes, there's a pluggable "spout wait strategy"; the default is to sleep for a [configurable amount of time](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml#L110) + +### OK, then what is the trident batch interval for? + +You know how computers of the 486 era had a [turbo button](http://en.wikipedia.org/wiki/Turbo_button) on them? It's like that. + +Actually, it has two practical uses. One is to throttle spouts that poll a remote source without throttling processing. For example, we have a spout that looks in a given S3 bucket for a new batch-uploaded file to read, linebreak and emit. We don't want it hitting S3 more than every few seconds: files don't show up more than once every few minutes, and a batch takes a few seconds to process. + +The other is to limit overpressure on the internal queues during startup or under a heavy burst load -- if the spouts spring to life and suddenly jam ten batches' worth of records into the system, you could have a mass of less-urgent tuples from batch 7 clog up the transfer buffer and prevent the $commit tuple from batch 3 to get through (or even just the regular old tuples from batch 3). What we do is set the trident batch interval to about half the typical end-to-end processing latency -- if it takes 600ms to process a batch, it's OK to only kick off a batch every 300ms. + +Note that this is a cap, not an additional delay -- with a period of 300ms, if your batch takes 258ms Trident will only delay an additional 42ms. + +### How do you set the batch size? + +Trident doesn't place its own limits on the batch count. In the case of the Kafka spout, the max fetch bytes size divided by the average record size defines an effective records per subbatch partition. + +### How do I resize a batch? + +The trident batch is a somewhat overloaded facility. Together with the number of partitions, the batch size is constrained by or serves to define + +1. the unit of transactional safety (tuples at risk vs time) +2. per partition, an effective windowing mechanism for windowed stream analytics +3. per partition, the number of simultaneous queries that will be made by a partitionQuery, partitionPersist, etc; +4. per partition, the number of records convenient for the spout to dispatch at the same time; + +You can't change the overall batch size once generated, but you can change the number of partitions -- do a shuffle and then change the parallelism hint + +## Time Series + +### How do I aggregate events by time? + +If have records with an immutable timestamp, and you would like to count, average or otherwise aggregate them into discrete time buckets, Trident is an excellent and scalable solution. + +Write an `Each` function that turns the timestamp into a time bucket: if the bucket size was "by hour", then the timestamp `2013-08-08 12:34:56` would be mapped to the `2013-08-08 12:00:00` time bucket, and so would everything else in the twelve o'clock hour. Then group on that timebucket and use a grouped persistentAggregate. The persistentAggregate uses a local cacheMap backed by a data store. Groups with many records require very few reads from the data store, and use efficient bulk reads and writes; as long as your data feed is relatively prompt Trident will make very efficient use of memory and network. Even if a server drops off line for a day, then delivers that full day's worth of data in a rush, the old results will be calmly retrieved and updated -- and without interfering with calculating the current results. + +### How can I know that all records for a time bucket have been received? + +You cannot know that all events are collected -- this is an epistemological challenge, not a distributed systems challenge. You can: + +* Set a time limit using domain knowledge +* Introduce a _punctuation_: a record known to come after all records in the given time bucket. Trident uses this scheme to know when a batch is complete. If you for instance receive records from a set of sensors, each in order for that sensor, then once all sensors have sent you a 3:02:xx or later timestamp lets you know you can commit. +* When possible, make your process incremental: each value that comes in makes the answer more an more true. A Trident ReducerAggregator is an operator that takes a prior result and a set of new records and returns a new result. This lets the result be cached and serialized to a datastore; if a server drops off line for a day and then comes back with a full day's worth of data in a rush, the old results will be calmly retrieved and updated. +* Lambda architecture: Record all events into an archival store (S3, HBase, HDFS) on receipt. in the fast layer, once the time window is clear, process the bucket to get an actionable answer, and ignore everything older than the time window. Periodically run a global aggregation to calculate a "correct" answer. diff --git a/docs/Fault-tolerance.md b/docs/Fault-tolerance.md new file mode 100644 index 00000000000..9a7a349f5b2 --- /dev/null +++ b/docs/Fault-tolerance.md @@ -0,0 +1,28 @@ +--- +layout: documentation +--- +This page explains the design details of Storm that make it a fault-tolerant system. + +## What happens when a worker dies? + +When a worker dies, the supervisor will restart it. If it continuously fails on startup and is unable to heartbeat to Nimbus, Nimbus will reassign the worker to another machine. + +## What happens when a node dies? + +The tasks assigned to that machine will time-out and Nimbus will reassign those tasks to other machines. + +## What happens when Nimbus or Supervisor daemons die? + +The Nimbus and Supervisor daemons are designed to be fail-fast (process self-destructs whenever any unexpected situation is encountered) and stateless (all state is kept in Zookeeper or on disk). As described in [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html), the Nimbus and Supervisor daemons must be run under supervision using a tool like daemontools or monit. So if the Nimbus or Supervisor daemons die, they restart like nothing happened. + +Most notably, no worker processes are affected by the death of Nimbus or the Supervisors. This is in contrast to Hadoop, where if the JobTracker dies, all the running jobs are lost. + +## Is Nimbus a single point of failure? + +If you lose the Nimbus node, the workers will still continue to function. Additionally, supervisors will continue to restart workers if they die. However, without Nimbus, workers won't be reassigned to other machines when necessary (like if you lose a worker machine). + +So the answer is that Nimbus is "sort of" a SPOF. In practice, it's not a big deal since nothing catastrophic happens when the Nimbus daemon dies. There are plans to make Nimbus highly available in the future. + +## How does Storm guarantee data processing? + +Storm provides mechanisms to guarantee data processing even if nodes die or messages are lost. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for the details. diff --git a/docs/Guaranteeing-message-processing.md b/docs/Guaranteeing-message-processing.md new file mode 100644 index 00000000000..91d43849905 --- /dev/null +++ b/docs/Guaranteeing-message-processing.md @@ -0,0 +1,179 @@ +--- +layout: documentation +--- +Storm guarantees that each message coming off a spout will be fully processed. This page describes how Storm accomplishes this guarantee and what you have to do as a user to benefit from Storm's reliability capabilities. + +### What does it mean for a message to be "fully processed"? + +A tuple coming off a spout can trigger thousands of tuples to be created based on it. Consider, for example, the streaming word count topology: + +```java +TopologyBuilder builder = new TopologyBuilder(); +builder.setSpout("sentences", new KestrelSpout("kestrel.backtype.com", + 22133, + "sentence_queue", + new StringScheme())); +builder.setBolt("split", new SplitSentence(), 10) + .shuffleGrouping("sentences"); +builder.setBolt("count", new WordCount(), 20) + .fieldsGrouping("split", new Fields("word")); +``` + +This topology reads sentences off of a Kestrel queue, splits the sentences into its constituent words, and then emits for each word the number of times it has seen that word before. A tuple coming off the spout triggers many tuples being created based on it: a tuple for each word in the sentence and a tuple for the updated count for each word. The tree of messages looks something like this: + +![Tuple tree](images/tuple_tree.png) + +Storm considers a tuple coming off a spout "fully processed" when the tuple tree has been exhausted and every message in the tree has been processed. A tuple is considered failed when its tree of messages fails to be fully processed within a specified timeout. This timeout can be configured on a topology-specific basis using the [Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS](javadocs/backtype/storm/Config.html#TOPOLOGY_MESSAGE_TIMEOUT_SECS) configuration and defaults to 30 seconds. + +### What happens if a message is fully processed or fails to be fully processed? + +To understand this question, let's take a look at the lifecycle of a tuple coming off of a spout. For reference, here is the interface that spouts implement (see the [Javadoc](javadocs/backtype/storm/spout/ISpout.html) for more information): + +```java +public interface ISpout extends Serializable { + void open(Map conf, TopologyContext context, SpoutOutputCollector collector); + void close(); + void nextTuple(); + void ack(Object msgId); + void fail(Object msgId); +} +``` + +First, Storm requests a tuple from the `Spout` by calling the `nextTuple` method on the `Spout`. The `Spout` uses the `SpoutOutputCollector` provided in the `open` method to emit a tuple to one of its output streams. When emitting a tuple, the `Spout` provides a "message id" that will be used to identify the tuple later. For example, the `KestrelSpout` reads a message off of the kestrel queue and emits as the "message id" the id provided by Kestrel for the message. Emitting a message to the `SpoutOutputCollector` looks like this: + +```java +_collector.emit(new Values("field1", "field2", 3) , msgId); +``` + +Next, the tuple gets sent to consuming bolts and Storm takes care of tracking the tree of messages that is created. If Storm detects that a tuple is fully processed, Storm will call the `ack` method on the originating `Spout` task with the message id that the `Spout` provided to Storm. Likewise, if the tuple times-out Storm will call the `fail` method on the `Spout`. Note that a tuple will be acked or failed by the exact same `Spout` task that created it. So if a `Spout` is executing as many tasks across the cluster, a tuple won't be acked or failed by a different task than the one that created it. + +Let's use `KestrelSpout` again to see what a `Spout` needs to do to guarantee message processing. When `KestrelSpout` takes a message off the Kestrel queue, it "opens" the message. This means the message is not actually taken off the queue yet, but instead placed in a "pending" state waiting for acknowledgement that the message is completed. While in the pending state, a message will not be sent to other consumers of the queue. Additionally, if a client disconnects all pending messages for that client are put back on the queue. When a message is opened, Kestrel provides the client with the data for the message as well as a unique id for the message. The `KestrelSpout` uses that exact id as the "message id" for the tuple when emitting the tuple to the `SpoutOutputCollector`. Sometime later on, when `ack` or `fail` are called on the `KestrelSpout`, the `KestrelSpout` sends an ack or fail message to Kestrel with the message id to take the message off the queue or have it put back on. + +### What is Storm's reliability API? + +There's two things you have to do as a user to benefit from Storm's reliability capabilities. First, you need to tell Storm whenever you're creating a new link in the tree of tuples. Second, you need to tell Storm when you have finished processing an individual tuple. By doing both these things, Storm can detect when the tree of tuples is fully processed and can ack or fail the spout tuple appropriately. Storm's API provides a concise way of doing both of these tasks. + +Specifying a link in the tuple tree is called _anchoring_. Anchoring is done at the same time you emit a new tuple. Let's use the following bolt as an example. This bolt splits a tuple containing a sentence into a tuple for each word: + +```java +public class SplitSentence extends BaseRichBolt { + OutputCollector _collector; + + public void prepare(Map conf, TopologyContext context, OutputCollector collector) { + _collector = collector; + } + + public void execute(Tuple tuple) { + String sentence = tuple.getString(0); + for(String word: sentence.split(" ")) { + _collector.emit(tuple, new Values(word)); + } + _collector.ack(tuple); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + } +``` + +Each word tuple is _anchored_ by specifying the input tuple as the first argument to `emit`. Since the word tuple is anchored, the spout tuple at the root of the tree will be replayed later on if the word tuple failed to be processed downstream. In contrast, let's look at what happens if the word tuple is emitted like this: + +```java +_collector.emit(new Values(word)); +``` + +Emitting the word tuple this way causes it to be _unanchored_. If the tuple fails be processed downstream, the root tuple will not be replayed. Depending on the fault-tolerance guarantees you need in your topology, sometimes it's appropriate to emit an unanchored tuple. + +An output tuple can be anchored to more than one input tuple. This is useful when doing streaming joins or aggregations. A multi-anchored tuple failing to be processed will cause multiple tuples to be replayed from the spouts. Multi-anchoring is done by specifying a list of tuples rather than just a single tuple. For example: + +```java +List anchors = new ArrayList(); +anchors.add(tuple1); +anchors.add(tuple2); +_collector.emit(anchors, new Values(1, 2, 3)); +``` + +Multi-anchoring adds the output tuple into multiple tuple trees. Note that it's also possible for multi-anchoring to break the tree structure and create tuple DAGs, like so: + +![Tuple DAG](images/tuple-dag.png) + +Storm's implementation works for DAGs as well as trees (pre-release it only worked for trees, and the name "tuple tree" stuck). + +Anchoring is how you specify the tuple tree -- the next and final piece to Storm's reliability API is specifying when you've finished processing an individual tuple in the tuple tree. This is done by using the `ack` and `fail` methods on the `OutputCollector`. If you look back at the `SplitSentence` example, you can see that the input tuple is acked after all the word tuples are emitted. + +You can use the `fail` method on the `OutputCollector` to immediately fail the spout tuple at the root of the tuple tree. For example, your application may choose to catch an exception from a database client and explicitly fail the input tuple. By failing the tuple explicitly, the spout tuple can be replayed faster than if you waited for the tuple to time-out. + +Every tuple you process must be acked or failed. Storm uses memory to track each tuple, so if you don't ack/fail every tuple, the task will eventually run out of memory. + +A lot of bolts follow a common pattern of reading an input tuple, emitting tuples based on it, and then acking the tuple at the end of the `execute` method. These bolts fall into the categories of filters and simple functions. Storm has an interface called `BasicBolt` that encapsulates this pattern for you. The `SplitSentence` example can be written as a `BasicBolt` like follows: + +```java +public class SplitSentence extends BaseBasicBolt { + public void execute(Tuple tuple, BasicOutputCollector collector) { + String sentence = tuple.getString(0); + for(String word: sentence.split(" ")) { + collector.emit(new Values(word)); + } + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + } +``` + +This implementation is simpler than the implementation from before and is semantically identical. Tuples emitted to `BasicOutputCollector` are automatically anchored to the input tuple, and the input tuple is acked for you automatically when the execute method completes. + +In contrast, bolts that do aggregations or joins may delay acking a tuple until after it has computed a result based on a bunch of tuples. Aggregations and joins will commonly multi-anchor their output tuples as well. These things fall outside the simpler pattern of `IBasicBolt`. + +### How do I make my applications work correctly given that tuples can be replayed? + +As always in software design, the answer is "it depends." Storm 0.7.0 introduced the "transactional topologies" feature, which enables you to get fully fault-tolerant exactly-once messaging semantics for most computations. Read more about transactional topologies [here](Transactional-topologies.html). + + +### How does Storm implement reliability in an efficient way? + +A Storm topology has a set of special "acker" tasks that track the DAG of tuples for every spout tuple. When an acker sees that a DAG is complete, it sends a message to the spout task that created the spout tuple to ack the message. You can set the number of acker tasks for a topology in the topology configuration using [Config.TOPOLOGY_ACKERS](javadocs/backtype/storm/Config.html#TOPOLOGY_ACKERS). Storm defaults TOPOLOGY_ACKERS to one task -- you will need to increase this number for topologies processing large amounts of messages. + +The best way to understand Storm's reliability implementation is to look at the lifecycle of tuples and tuple DAGs. When a tuple is created in a topology, whether in a spout or a bolt, it is given a random 64 bit id. These ids are used by ackers to track the tuple DAG for every spout tuple. + +Every tuple knows the ids of all the spout tuples for which it exists in their tuple trees. When you emit a new tuple in a bolt, the spout tuple ids from the tuple's anchors are copied into the new tuple. When a tuple is acked, it sends a message to the appropriate acker tasks with information about how the tuple tree changed. In particular it tells the acker "I am now completed within the tree for this spout tuple, and here are the new tuples in the tree that were anchored to me". + +For example, if tuples "D" and "E" were created based on tuple "C", here's how the tuple tree changes when "C" is acked: + +![What happens on an ack](images/ack_tree.png) + +Since "C" is removed from the tree at the same time that "D" and "E" are added to it, the tree can never be prematurely completed. + +There are a few more details to how Storm tracks tuple trees. As mentioned already, you can have an arbitrary number of acker tasks in a topology. This leads to the following question: when a tuple is acked in the topology, how does it know to which acker task to send that information? + +Storm uses mod hashing to map a spout tuple id to an acker task. Since every tuple carries with it the spout tuple ids of all the trees they exist within, they know which acker tasks to communicate with. + +Another detail of Storm is how the acker tasks track which spout tasks are responsible for each spout tuple they're tracking. When a spout task emits a new tuple, it simply sends a message to the appropriate acker telling it that its task id is responsible for that spout tuple. Then when an acker sees a tree has been completed, it knows to which task id to send the completion message. + +Acker tasks do not track the tree of tuples explicitly. For large tuple trees with tens of thousands of nodes (or more), tracking all the tuple trees could overwhelm the memory used by the ackers. Instead, the ackers take a different strategy that only requires a fixed amount of space per spout tuple (about 20 bytes). This tracking algorithm is the key to how Storm works and is one of its major breakthroughs. + +An acker task stores a map from a spout tuple id to a pair of values. The first value is the task id that created the spout tuple which is used later on to send completion messages. The second value is a 64 bit number called the "ack val". The ack val is a representation of the state of the entire tuple tree, no matter how big or how small. It is simply the xor of all tuple ids that have been created and/or acked in the tree. + +When an acker task sees that an "ack val" has become 0, then it knows that the tuple tree is completed. Since tuple ids are random 64 bit numbers, the chances of an "ack val" accidentally becoming 0 is extremely small. If you work the math, at 10K acks per second, it will take 50,000,000 years until a mistake is made. And even then, it will only cause data loss if that tuple happens to fail in the topology. + +Now that you understand the reliability algorithm, let's go over all the failure cases and see how in each case Storm avoids data loss: + +- **A tuple isn't acked because the task died**: In this case the spout tuple ids at the root of the trees for the failed tuple will time out and be replayed. +- **Acker task dies**: In this case all the spout tuples the acker was tracking will time out and be replayed. +- **Spout task dies**: In this case the source that the spout talks to is responsible for replaying the messages. For example, queues like Kestrel and RabbitMQ will place all pending messages back on the queue when a client disconnects. + +As you have seen, Storm's reliability mechanisms are completely distributed, scalable, and fault-tolerant. + +### Tuning reliability + +Acker tasks are lightweight, so you don't need very many of them in a topology. You can track their performance through the Storm UI (component id "__acker"). If the throughput doesn't look right, you'll need to add more acker tasks. + +If reliability isn't important to you -- that is, you don't care about losing tuples in failure situations -- then you can improve performance by not tracking the tuple tree for spout tuples. Not tracking a tuple tree halves the number of messages transferred since normally there's an ack message for every tuple in the tuple tree. Additionally, it requires fewer ids to be kept in each downstream tuple, reducing bandwidth usage. + +There are three ways to remove reliability. The first is to set Config.TOPOLOGY_ACKERS to 0. In this case, Storm will call the `ack` method on the spout immediately after the spout emits a tuple. The tuple tree won't be tracked. + +The second way is to remove reliability on a message by message basis. You can turn off tracking for an individual spout tuple by omitting a message id in the `SpoutOutputCollector.emit` method. + +Finally, if you don't care if a particular subset of the tuples downstream in the topology fail to be processed, you can emit them as unanchored tuples. Since they're not anchored to any spout tuples, they won't cause any spout tuples to fail if they aren't acked. diff --git a/docs/Hooks.md b/docs/Hooks.md new file mode 100644 index 00000000000..bbe87a9b24a --- /dev/null +++ b/docs/Hooks.md @@ -0,0 +1,7 @@ +--- +layout: documentation +--- +Storm provides hooks with which you can insert custom code to run on any number of events within Storm. You create a hook by extending the [BaseTaskHook](javadocs/backtype/storm/hooks/BaseTaskHook.html) class and overriding the appropriate method for the event you want to catch. There are two ways to register your hook: + +1. In the open method of your spout or prepare method of your bolt using the [TopologyContext#addTaskHook](javadocs/backtype/storm/task/TopologyContext.html) method. +2. Through the Storm configuration using the ["topology.auto.task.hooks"](javadocs/backtype/storm/Config.html#TOPOLOGY_AUTO_TASK_HOOKS) config. These hooks are automatically registered in every spout or bolt, and are useful for doing things like integrating with a custom monitoring system. diff --git a/docs/Implementation-docs.md b/docs/Implementation-docs.md new file mode 100644 index 00000000000..f01083a8b86 --- /dev/null +++ b/docs/Implementation-docs.md @@ -0,0 +1,18 @@ +--- +layout: documentation +--- +This section of the wiki is dedicated to explaining how Storm is implemented. You should have a good grasp of how to use Storm before reading these sections. + +- [Structure of the codebase](Structure-of-the-codebase.html) +- [Lifecycle of a topology](Lifecycle-of-a-topology.html) +- [Message passing implementation](Message-passing-implementation.html) +- [Acking framework implementation](Acking-framework-implementation.html) +- [Metrics](Metrics.html) +- How transactional topologies work + - subtopology for TransactionalSpout + - how state is stored in ZK + - subtleties around what to do when emitting batches out of order +- Unit testing + - time simulation + - complete-topology + - tracker clusters diff --git a/docs/Installing-native-dependencies.md b/docs/Installing-native-dependencies.md new file mode 100644 index 00000000000..1937d4bffcf --- /dev/null +++ b/docs/Installing-native-dependencies.md @@ -0,0 +1,38 @@ +--- +layout: documentation +--- +The native dependencies are only needed on actual Storm clusters. When running Storm in local mode, Storm uses a pure Java messaging system so that you don't need to install native dependencies on your development machine. + +Installing ZeroMQ and JZMQ is usually straightforward. Sometimes, however, people run into issues with autoconf and get strange errors. If you run into any issues, please email the [Storm mailing list](http://groups.google.com/group/storm-user) or come get help in the #storm-user room on freenode. + +Storm has been tested with ZeroMQ 2.1.7, and this is the recommended ZeroMQ release that you install. You can download a ZeroMQ release [here](http://download.zeromq.org/). Installing ZeroMQ should look something like this: + +``` +wget http://download.zeromq.org/zeromq-2.1.7.tar.gz +tar -xzf zeromq-2.1.7.tar.gz +cd zeromq-2.1.7 +./configure +make +sudo make install +``` + +JZMQ is the Java bindings for ZeroMQ. JZMQ doesn't have any releases (we're working with them on that), so there is risk of a regression if you always install from the master branch. To prevent a regression from happening, you should instead install from [this fork](http://github.com/nathanmarz/jzmq) which is tested to work with Storm. Installing JZMQ should look something like this: + +``` +#install jzmq +git clone https://github.com/nathanmarz/jzmq.git +cd jzmq +./autogen.sh +./configure +make +sudo make install +``` + +To get the JZMQ build to work, you may need to do one or all of the following: + +1. Set JAVA_HOME environment variable appropriately +2. Install Java dev package (more info [here](http://codeslinger.posterous.com/getting-zeromq-and-jzmq-running-on-mac-os-x) for Mac OSX users) +3. Upgrade autoconf on your machine +4. Follow the instructions in [this blog post](http://blog.pmorelli.com/getting-zeromq-and-jzmq-running-on-mac-os-x) + +If you run into any errors when running `./configure`, [this thread](http://stackoverflow.com/questions/3522248/how-do-i-compile-jzmq-for-zeromq-on-osx) may provide a solution. diff --git a/docs/Kestrel-and-Storm.md b/docs/Kestrel-and-Storm.md new file mode 100644 index 00000000000..e16b0d91ef4 --- /dev/null +++ b/docs/Kestrel-and-Storm.md @@ -0,0 +1,198 @@ +--- +layout: documentation +--- +This page explains how to use to Storm to consume items from a Kestrel cluster. + +## Preliminaries +### Storm +This tutorial uses examples from the [storm-kestrel](https://github.com/nathanmarz/storm-kestrel) project and the [storm-starter](https://github.com/nathanmarz/storm-starter) project. It's recommended that you clone those projects and follow along with the examples. Read [Setting up development environment](https://github.com/apache/incubator-storm/wiki/Setting-up-development-environment) and [Creating a new Storm project](https://github.com/apache/incubator-storm/wiki/Creating-a-new-Storm-project) to get your machine set up. +### Kestrel +It assumes you are able to run locally a Kestrel server as described [here](https://github.com/nathanmarz/storm-kestrel). + +## Kestrel Server and Queue +A single kestrel server has a set of queues. A Kestrel queue is a very simple message queue that runs on the JVM and uses the memcache protocol (with some extensions) to talk to clients. For details, look at the implementation of the [KestrelThriftClient](https://github.com/nathanmarz/storm-kestrel/blob/master/src/jvm/backtype/storm/spout/KestrelThriftClient.java) class provided in [storm-kestrel](https://github.com/nathanmarz/storm-kestrel) project. + +Each queue is strictly ordered following the FIFO (first in, first out) principle. To keep up with performance items are cached in system memory; though, only the first 128MB is kept in memory. When stopping the server, the queue state is stored in a journal file. + +Further, details can be found [here](https://github.com/nathanmarz/kestrel/blob/master/docs/guide.md). + +Kestrel is: +* fast +* small +* durable +* reliable + +For instance, Twitter uses Kestrel as the backbone of its messaging infrastructure as described [here] (http://bhavin.directi.com/notes-on-kestrel-the-open-source-twitter-queue/). + +## Add items to Kestrel +At first, we need to have a program that can add items to a Kestrel queue. The following method takes benefit of the KestrelClient implementation in [storm-kestrel](https://github.com/nathanmarz/storm-kestrel). It adds sentences into a Kestrel queue randomly chosen out of an array that holds five possible sentences. + +``` + private static void queueSentenceItems(KestrelClient kestrelClient, String queueName) + throws ParseError, IOException { + + String[] sentences = new String[] { + "the cow jumped over the moon", + "an apple a day keeps the doctor away", + "four score and seven years ago", + "snow white and the seven dwarfs", + "i am at two with nature"}; + + Random _rand = new Random(); + + for(int i=1; i<=10; i++){ + + String sentence = sentences[_rand.nextInt(sentences.length)]; + + String val = "ID " + i + " " + sentence; + + boolean queueSucess = kestrelClient.queue(queueName, val); + + System.out.println("queueSucess=" +queueSucess+ " [" + val +"]"); + } + } +``` + +## Remove items from Kestrel + +This method dequeues items from a queue without removing them. +``` + private static void dequeueItems(KestrelClient kestrelClient, String queueName) throws IOException, ParseError + { + for(int i=1; i<=12; i++){ + + Item item = kestrelClient.dequeue(queueName); + + if(item==null){ + System.out.println("The queue (" + queueName + ") contains no items."); + } + else + { + byte[] data = item._data; + + String receivedVal = new String(data); + + System.out.println("receivedItem=" + receivedVal); + } + } +``` + +This method dequeues items from a queue and then removes them. +``` + private static void dequeueAndRemoveItems(KestrelClient kestrelClient, String queueName) + throws IOException, ParseError + { + for(int i=1; i<=12; i++){ + + Item item = kestrelClient.dequeue(queueName); + + + if(item==null){ + System.out.println("The queue (" + queueName + ") contains no items."); + } + else + { + int itemID = item._id; + + + byte[] data = item._data; + + String receivedVal = new String(data); + + kestrelClient.ack(queueName, itemID); + + System.out.println("receivedItem=" + receivedVal); + } + } + } +``` + +## Add Items continuously to Kestrel + +This is our final program to run in order to add continuously sentence items to a queue called **sentence_queue** of a locally running Kestrel server. + +In order to stop it type a closing bracket char ']' in console and hit 'Enter'. + +``` + import java.io.IOException; + import java.io.InputStream; + import java.util.Random; + + import backtype.storm.spout.KestrelClient; + import backtype.storm.spout.KestrelClient.Item; + import backtype.storm.spout.KestrelClient.ParseError; + + public class AddSentenceItemsToKestrel { + + /** + * @param args + */ + public static void main(String[] args) { + + InputStream is = System.in; + + char closing_bracket = ']'; + + int val = closing_bracket; + + boolean aux = true; + + try { + + KestrelClient kestrelClient = null; + String queueName = "sentence_queue"; + + while(aux){ + + kestrelClient = new KestrelClient("localhost",22133); + + queueSentenceItems(kestrelClient, queueName); + + kestrelClient.close(); + + Thread.sleep(1000); + + if(is.available()>0){ + if(val==is.read()) + aux=false; + } + } + } catch (IOException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + catch (ParseError e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + + System.out.println("end"); + + } + } +``` +## Using KestrelSpout + +This topology reads sentences off of a Kestrel queue using KestrelSpout, splits the sentences into its constituent words (Bolt: SplitSentence), and then emits for each word the number of times it has seen that word before (Bolt: WordCount). How data is processed is described in detail in [Guaranteeing message processing](Guaranteeing-message-processing.html). + +``` + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout("sentences", new KestrelSpout("localhost",22133,"sentence_queue",new StringScheme())); + builder.setBolt("split", new SplitSentence(), 10) + .shuffleGrouping("sentences"); + builder.setBolt("count", new WordCount(), 20) + .fieldsGrouping("split", new Fields("word")); +``` + +## Execution + +At first, start your local kestrel server in production or development mode. + +Than, wait about 5 seconds in order to avoid a ConnectException. + +Now execute the program to add items to the queue and launch the Storm topology. The order in which you launch the programs is of no importance. + +If you run the topology with TOPOLOGY_DEBUG you should see tuples being emitted in the topology. diff --git a/docs/Lifecycle-of-a-topology.md b/docs/Lifecycle-of-a-topology.md new file mode 100644 index 00000000000..4919be8e920 --- /dev/null +++ b/docs/Lifecycle-of-a-topology.md @@ -0,0 +1,80 @@ +--- +layout: documentation +--- +(**NOTE**: this page is based on the 0.7.1 code; many things have changed since then, including a split between tasks and executors, and a reorganization of the code under `storm-core/src` rather than `src/`.) + +This page explains in detail the lifecycle of a topology from running the "storm jar" command to uploading the topology to Nimbus to the supervisors starting/stopping workers to workers and tasks setting themselves up. It also explains how Nimbus monitors topologies and how topologies are shutdown when they are killed. + +First a couple of important notes about topologies: + +1. The actual topology that runs is different than the topology the user specifies. The actual topology has implicit streams and an implicit "acker" bolt added to manage the acking framework (used to guarantee data processing). The implicit topology is created via the [system-topology!](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/common.clj#L188) function. +2. `system-topology!` is used in two places: + - when Nimbus is creating tasks for the topology [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L316) + - in the worker so it knows where it needs to route messages to [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L90) + +## Starting a topology + +- "storm jar" command executes your class with the specified arguments. The only special thing that "storm jar" does is set the "storm.jar" environment variable for use by `StormSubmitter` later. [code](https://github.com/apache/incubator-storm/blob/0.7.1/bin/storm#L101) +- When your code uses `StormSubmitter.submitTopology`, `StormSubmitter` takes the following actions: + - First, `StormSubmitter` uploads the jar if it hasn't been uploaded before. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/jvm/backtype/storm/StormSubmitter.java#L83) + - Jar uploading is done via Nimbus's Thrift interface [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/storm.thrift#L200) + - `beginFileUpload` returns a path in Nimbus's inbox + - 15 kilobytes are uploaded at a time through `uploadChunk` + - `finishFileUpload` is called when it's finished uploading + - Here is Nimbus's implementation of those Thrift methods: [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L694) + - Second, `StormSubmitter` calls `submitTopology` on the Nimbus thrift interface [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/jvm/backtype/storm/StormSubmitter.java#L60) + - The topology config is serialized using JSON (JSON is used so that writing DSL's in any language is as easy as possible) + - Notice that the Thrift `submitTopology` call takes in the Nimbus inbox path where the jar was uploaded + +- Nimbus receives the topology submission. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L639) +- Nimbus normalizes the topology configuration. The main purpose of normalization is to ensure that every single task will have the same serialization registrations, which is critical for getting serialization working correctly. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L557) +- Nimbus sets up the static state for the topology [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L661) + - Jars and configs are kept on local filesystem because they're too big for Zookeeper. The jar and configs are copied into the path {nimbus local dir}/stormdist/{topology id} + - `setup-storm-static` writes task -> component mapping into ZK + - `setup-heartbeats` creates a ZK "directory" in which tasks can heartbeat +- Nimbus calls `mk-assignment` to assign tasks to machines [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L458) + - Assignment record definition is here: [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/common.clj#L25) + - Assignment contains: + - `master-code-dir`: used by supervisors to download the correct jars/configs for the topology from Nimbus + - `task->node+port`: Map from a task id to the worker that task should be running on. (A worker is identified by a node/port pair) + - `node->host`: A map from node id to hostname. This is used so workers know which machines to connect to to communicate with other workers. Node ids are used to identify supervisors so that multiple supervisors can be run on one machine. One place this is done is with Mesos integration. + - `task->start-time-secs`: Contains a map from task id to the timestamp at which Nimbus launched that task. This is used by Nimbus when monitoring topologies, as tasks are given a longer timeout to heartbeat when they're first launched (the launch timeout is configured by "nimbus.task.launch.secs" config) +- Once topologies are assigned, they're initially in a deactivated mode. `start-storm` writes data into Zookeeper so that the cluster knows the topology is active and can start emitting tuples from spouts. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L504) + +- TODO cluster state diagram (show all nodes and what's kept everywhere) + +- Supervisor runs two functions in the background: + - `synchronize-supervisor`: This is called whenever assignments in Zookeeper change and also every 10 seconds. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L241) + - Downloads code from Nimbus for topologies assigned to this machine for which it doesn't have the code yet. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L258) + - Writes into local filesystem what this node is supposed to be running. It writes a map from port -> LocalAssignment. LocalAssignment contains a topology id as well as the list of task ids for that worker. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L13) + - `sync-processes`: Reads from the LFS what `synchronize-supervisor` wrote and compares that to what's actually running on the machine. It then starts/stops worker processes as necessary to synchronize. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L177) + +- Worker processes start up through the `mk-worker` function [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L67) + - Worker connects to other workers and starts a thread to monitor for changes. So if a worker gets reassigned, the worker will automatically reconnect to the other worker's new location. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L123) + - Monitors whether a topology is active or not and stores that state in the `storm-active-atom` variable. This variable is used by tasks to determine whether or not to call `nextTuple` on the spouts. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L155) + - The worker launches the actual tasks as threads within it [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L178) +- Tasks are set up through the `mk-task` function [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L160) + - Tasks set up routing function which takes in a stream and an output tuple and returns a list of task ids to send the tuple to [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L207) (there's also a 3-arity version used for direct streams) + - Tasks set up the spout-specific or bolt-specific code with [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L241) + +## Topology Monitoring + +- Nimbus monitors the topology during its lifetime + - Schedules recurring task on the timer thread to check the topologies [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L623) + - Nimbus's behavior is represented as a finite state machine [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L98) + - The "monitor" event is called on a topology every "nimbus.monitor.freq.secs", which calls `reassign-topology` through `reassign-transition` [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L497) + - `reassign-topology` calls `mk-assignments`, the same function used to assign the topology the first time. `mk-assignments` is also capable of incrementally updating a topology + - `mk-assignments` checks heartbeats and reassigns workers as necessary + - Any reassignments change the state in ZK, which will trigger supervisors to synchronize and start/stop workers + +## Killing a topology + +- "storm kill" command runs this code which just calls the Nimbus Thrift interface to kill the topology: [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/command/kill_topology.clj) +- Nimbus receives the kill command [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L671) +- Nimbus applies the "kill" transition to the topology [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L676) +- The kill transition function changes the status of the topology to "killed" and schedules the "remove" event to run "wait time seconds" in the future. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L63) + - The wait time defaults to the topology message timeout but can be overridden with the -w flag in the "storm kill" command + - This causes the topology to be deactivated for the wait time before its actually shut down. This gives the topology a chance to finish processing what it's currently processing before shutting down the workers + - Changing the status during the kill transition ensures that the kill protocol is fault-tolerant to Nimbus crashing. On startup, if the status of the topology is "killed", Nimbus schedules the remove event to run "wait time seconds" in the future [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L111) +- Removing a topology cleans out the assignment and static information from ZK [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L116) +- A separate cleanup thread runs the `do-cleanup` function which will clean up the heartbeat dir and the jars/configs stored locally. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L577) diff --git a/docs/Local-mode.md b/docs/Local-mode.md new file mode 100644 index 00000000000..1f98e369245 --- /dev/null +++ b/docs/Local-mode.md @@ -0,0 +1,27 @@ +--- +layout: documentation +--- +Local mode simulates a Storm cluster in process and is useful for developing and testing topologies. Running topologies in local mode is similar to running topologies [on a cluster](Running-topologies-on-a-production-cluster.html). + +To create an in-process cluster, simply use the `LocalCluster` class. For example: + +```java +import backtype.storm.LocalCluster; + +LocalCluster cluster = new LocalCluster(); +``` + +You can then submit topologies using the `submitTopology` method on the `LocalCluster` object. Just like the corresponding method on [StormSubmitter](javadocs/backtype/storm/StormSubmitter.html), `submitTopology` takes a name, a topology configuration, and the topology object. You can then kill a topology using the `killTopology` method which takes the topology name as an argument. + +To shutdown a local cluster, simple call: + +```java +cluster.shutdown(); +``` + +### Common configurations for local mode + +You can see a full list of configurations [here](javadocs/backtype/storm/Config.html). + +1. **Config.TOPOLOGY_MAX_TASK_PARALLELISM**: This config puts a ceiling on the number of threads spawned for a single component. Oftentimes production topologies have a lot of parallelism (hundreds of threads) which places unreasonable load when trying to test the topology in local mode. This config lets you easy control that parallelism. +2. **Config.TOPOLOGY_DEBUG**: When this is set to true, Storm will log a message every time a tuple is emitted from any spout or bolt. This is extremely useful for debugging. diff --git a/docs/Maven.md b/docs/Maven.md new file mode 100644 index 00000000000..85828da2bfe --- /dev/null +++ b/docs/Maven.md @@ -0,0 +1,56 @@ +--- +layout: documentation +--- +To develop topologies, you'll need the Storm jars on your classpath. You should either include the unpacked jars in the classpath for your project or use Maven to include Storm as a development dependency. Storm is hosted on Clojars (a Maven repository). To include Storm in your project as a development dependency, add the following to your pom.xml: + +```xml + + clojars.org + http://clojars.org/repo + +``` + +```xml + + storm + storm + 0.7.2 + test + +``` + +[Here's an example](https://github.com/nathanmarz/storm-starter/blob/master/m2-pom.xml) of a pom.xml for a Storm project. + +If Maven isn't your thing, check out [leiningen](https://github.com/technomancy/leiningen). Leiningen is a build tool for Clojure, but it can be used for pure Java projects as well. Leiningen makes builds and dependency management using Maven dead-simple. Here's an example project.clj for a pure-Java Storm project: + +```clojure +(defproject storm-starter "0.0.1-SNAPSHOT" + :java-source-path "src/jvm" + :javac-options {:debug "true" :fork "true"} + :jvm-opts ["-Djava.library.path=/usr/local/lib:/opt/local/lib:/usr/lib"] + :dependencies [] + :dev-dependencies [ + [storm "0.7.2"] + ]) +``` + +You can fetch dependencies using `lein deps`, build the project with `lein compile`, and make a jar suitable for submitting to a cluster with `lein uberjar`. + +### Using Storm as a library + +If you want to use Storm as a library (e.g., use the Distributed RPC client) and have the Storm dependency jars be distributed with your application, there's a separate Maven dependency called "storm/storm-lib". The only difference between this dependency and the usual "storm/storm" is that storm-lib does not have any logging configured. + +### Developing Storm + +You will want to + + bash ./bin/install_zmq.sh # install the jzmq dependency + lein sub install + +Build javadocs with + + bash ./bin/javadoc.sh + +### Building a Storm Release + +Use the file `bin/build_release.sh` to make a zipfile like the ones you would download (and like what the bin files require in order to run daemons). diff --git a/docs/Message-passing-implementation.md b/docs/Message-passing-implementation.md new file mode 100644 index 00000000000..f22a5aaf3c4 --- /dev/null +++ b/docs/Message-passing-implementation.md @@ -0,0 +1,28 @@ +--- +layout: documentation +--- +(Note: this walkthrough is out of date as of 0.8.0. 0.8.0 revamped the message passing infrastructure to be based on the Disruptor) + +This page walks through how emitting and transferring tuples works in Storm. + +- Worker is responsible for message transfer + - `refresh-connections` is called every "task.refresh.poll.secs" or whenever assignment in ZK changes. It manages connections to other workers and maintains a mapping from task -> worker [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L123) + - Provides a "transfer function" that is used by tasks to send tuples to other tasks. The transfer function takes in a task id and a tuple, and it serializes the tuple and puts it onto a "transfer queue". There is a single transfer queue for each worker. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L56) + - The serializer is thread-safe [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/jvm/backtype/storm/serialization/KryoTupleSerializer.java#L26) + - The worker has a single thread which drains the transfer queue and sends the messages to other workers [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L185) + - Message sending happens through this protocol: [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/protocol.clj) + - The implementation for distributed mode uses ZeroMQ [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/zmq.clj) + - The implementation for local mode uses in memory Java queues (so that it's easy to use Storm locally without needing to get ZeroMQ installed) [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/local.clj) +- Receiving messages in tasks works differently in local mode and distributed mode + - In local mode, the tuple is sent directly to an in-memory queue for the receiving task [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/messaging/local.clj#L21) + - In distributed mode, each worker listens on a single TCP port for incoming messages and then routes those messages in-memory to tasks. The TCP port is called a "virtual port", because it receives [task id, message] and then routes it to the actual task. [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/worker.clj#L204) + - The virtual port implementation is here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/zilch/virtual_port.clj) + - Tasks listen on an in-memory ZeroMQ port for messages from the virtual port [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L201) + - Bolts listen here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L489) + - Spouts listen here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L382) +- Tasks are responsible for message routing. A tuple is emitted either to a direct stream (where the task id is specified) or a regular stream. In direct streams, the message is only sent if that bolt subscribes to that direct stream. In regular streams, the stream grouping functions are used to determine the task ids to send the tuple to. + - Tasks have a routing map from {stream id} -> {component id} -> {stream grouping function} [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L198) + - The "tasks-fn" returns the task ids to send the tuples to for either regular stream emit or direct stream emit [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L207) + - After getting the output task ids, bolts and spouts use the transfer-fn provided by the worker to actually transfer the tuples + - Bolt transfer code here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L429) + - Spout transfer code here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L329) diff --git a/docs/Metrics.md b/docs/Metrics.md new file mode 100644 index 00000000000..f43f8c765ab --- /dev/null +++ b/docs/Metrics.md @@ -0,0 +1,34 @@ +--- +layout: documentation +--- +Storm exposes a metrics interface to report summary statistics across the full topology. +It's used internally to track the numbers you see in the Nimbus UI console: counts of executes and acks; average process latency per bolt; worker heap usage; and so forth. + +### Metric Types + +Metrics have to implement just one method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values back to zero. + +Storm gives you these metric types: + +* [AssignableMetric]() -- set the metric to the explicit value you supply. Useful if it's an external value or in the case that you are already calculating the summary statistic yourself. +* [CombinedMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/CombinedMetric.java) -- generic interface for metrics that can be updated associatively. +* [CountMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java) -- a running total of the supplied values. Call `incr()` to increment by one, `incrBy(n)` to add/subtract the given number. + - [MultiCountMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java) -- a hashmap of count metrics. +* [ReducedMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/ReducedMetric.java) + - [MeanReducer](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java) -- track a running average of values given to its `reduce()` method. (It accepts `Double`, `Integer` or `Long` values, and maintains the internal average as a `Double`.) Despite his reputation, the MeanReducer is actually a pretty nice guy in person. + - [MultiReducedMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java) -- a hashmap of reduced metrics. + + +### Metric Consumer + + +### Build your own metric + + + +### Builtin Metrics + +The [builtin metrics](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj) instrument Storm itself. + +[builtin_metrics.clj](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ack-spout-msg`](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/executor.clj#358) in `clj/b/s/daemon/daemon/executor.clj` + diff --git a/docs/Multilang-protocol.md b/docs/Multilang-protocol.md new file mode 100644 index 00000000000..a3cb22c20c7 --- /dev/null +++ b/docs/Multilang-protocol.md @@ -0,0 +1,221 @@ +--- +layout: documentation +--- +This page explains the multilang protocol as of Storm 0.7.1. Versions prior to 0.7.1 used a somewhat different protocol, documented [here](Storm-multi-language-protocol-(versions-0.7.0-and-below\).html). + +# Storm Multi-Language Protocol + +## Shell Components + +Support for multiple languages is implemented via the ShellBolt, +ShellSpout, and ShellProcess classes. These classes implement the +IBolt and ISpout interfaces and the protocol for executing a script or +program via the shell using Java's ProcessBuilder class. + +## Output fields + +Output fields are part of the Thrift definition of the topology. This means that when you multilang in Java, you need to create a bolt that extends ShellBolt, implements IRichBolt, and declare the fields in `declareOutputFields` (similarly for ShellSpout). + +You can learn more about this on [Concepts](Concepts.html) + +## Protocol Preamble + +A simple protocol is implemented via the STDIN and STDOUT of the +executed script or program. All data exchanged with the process is +encoded in JSON, making support possible for pretty much any language. + +# Packaging Your Stuff + +To run a shell component on a cluster, the scripts that are shelled +out to must be in the `resources/` directory within the jar submitted +to the master. + +However, during development or testing on a local machine, the resources +directory just needs to be on the classpath. + +## The Protocol + +Notes: + +* Both ends of this protocol use a line-reading mechanism, so be sure to +trim off newlines from the input and to append them to your output. +* All JSON inputs and outputs are terminated by a single line containing "end". Note that this delimiter is not itself JSON encoded. +* The bullet points below are written from the perspective of the script writer's +STDIN and STDOUT. + +### Initial Handshake + +The initial handshake is the same for both types of shell components: + +* STDIN: Setup info. This is a JSON object with the Storm configuration, Topology context, and a PID directory, like this: + +``` +{ + "conf": { + "topology.message.timeout.secs": 3, + // etc + }, + "context": { + "task->component": { + "1": "example-spout", + "2": "__acker", + "3": "example-bolt" + }, + "taskid": 3 + }, + "pidDir": "..." +} +``` + +Your script should create an empty file named with its PID in this directory. e.g. +the PID is 1234, so an empty file named 1234 is created in the directory. This +file lets the supervisor know the PID so it can shutdown the process later on. + +* STDOUT: Your PID, in a JSON object, like `{"pid": 1234}`. The shell component will log the PID to its log. + +What happens next depends on the type of component: + +### Spouts + +Shell spouts are synchronous. The rest happens in a while(true) loop: + +* STDIN: Either a next, ack, or fail command. + +"next" is the equivalent of ISpout's `nextTuple`. It looks like: + +``` +{"command": "next"} +``` + +"ack" looks like: + +``` +{"command": "ack", "id": "1231231"} +``` + +"fail" looks like: + +``` +{"command": "fail", "id": "1231231"} +``` + +* STDOUT: The results of your spout for the previous command. This can + be a sequence of emits and logs. + +An emit looks like: + +``` +{ + "command": "emit", + // The id for the tuple. Leave this out for an unreliable emit. The id can + // be a string or a number. + "id": "1231231", + // The id of the stream this tuple was emitted to. Leave this empty to emit to default stream. + "stream": "1", + // If doing an emit direct, indicate the task to send the tuple to + "task": 9, + // All the values in this tuple + "tuple": ["field1", 2, 3] +} +``` + +If not doing an emit direct, you will immediately receive the task ids to which the tuple was emitted on STDIN as a JSON array. + +A "log" will log a message in the worker log. It looks like: + +``` +{ + "command": "log", + // the message to log + "msg": "hello world!" +} +``` + +* STDOUT: a "sync" command ends the sequence of emits and logs. It looks like: + +``` +{"command": "sync"} +``` + +After you sync, ShellSpout will not read your output until it sends another next, ack, or fail command. + +Note that, similarly to ISpout, all of the spouts in the worker will be locked up after a next, ack, or fail, until you sync. Also like ISpout, if you have no tuples to emit for a next, you should sleep for a small amount of time before syncing. ShellSpout will not automatically sleep for you. + + +### Bolts + +The shell bolt protocol is asynchronous. You will receive tuples on STDIN as soon as they are available, and you may emit, ack, and fail, and log at any time by writing to STDOUT, as follows: + +* STDIN: A tuple! This is a JSON encoded structure like this: + +``` +{ + // The tuple's id - this is a string to support languages lacking 64-bit precision + "id": "-6955786537413359385", + // The id of the component that created this tuple + "comp": "1", + // The id of the stream this tuple was emitted to + "stream": "1", + // The id of the task that created this tuple + "task": 9, + // All the values in this tuple + "tuple": ["snow white and the seven dwarfs", "field2", 3] +} +``` + +* STDOUT: An ack, fail, emit, or log. Emits look like: + +``` +{ + "command": "emit", + // The ids of the tuples this output tuples should be anchored to + "anchors": ["1231231", "-234234234"], + // The id of the stream this tuple was emitted to. Leave this empty to emit to default stream. + "stream": "1", + // If doing an emit direct, indicate the task to send the tuple to + "task": 9, + // All the values in this tuple + "tuple": ["field1", 2, 3] +} +``` + +If not doing an emit direct, you will receive the task ids to which +the tuple was emitted on STDIN as a JSON array. Note that, due to the +asynchronous nature of the shell bolt protocol, when you read after +emitting, you may not receive the task ids. You may instead read the +task ids for a previous emit or a new tuple to process. You will +receive the task id lists in the same order as their corresponding +emits, however. + +An ack looks like: + +``` +{ + "command": "ack", + // the id of the tuple to ack + "id": "123123" +} +``` + +A fail looks like: + +``` +{ + "command": "fail", + // the id of the tuple to fail + "id": "123123" +} +``` + +A "log" will log a message in the worker log. It looks like: + +``` +{ + "command": "log", + // the message to log + "msg": "hello world!" +} +``` + +* Note that, as of version 0.7.1, there is no longer any need for a + shell bolt to 'sync'. diff --git a/docs/Powered-By.md b/docs/Powered-By.md new file mode 100644 index 00000000000..7fcc0345b67 --- /dev/null +++ b/docs/Powered-By.md @@ -0,0 +1,1028 @@ +--- +layout: documentation +--- +Want to be added to this page? Send an email [here](mailto:nathan.marz@gmail.com). + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
+Groupon + +

+At Groupon we use Storm to build real-time data integration systems. Storm helps us analyze, clean, normalize, and resolve large amounts of non-unique data points with low latency and high throughput. +

+
The Weather Channel +

At Weather Channel we use several Storm topologies to ingest and persist weather data. Each topology is responsible for fetching one dataset from an internal or external network (the Internet), reshaping the records for use by our company, and persisting the records to relational databases. It is particularly useful to have an automatic mechanism for repeating attempts to download and manipulate the data when there is a hiccup.

+
+FullContact + +

+At FullContact we currently use Storm as the backbone of the system which synchronizes our Cloud Address Book with third party services such as Google Contacts and Salesforce. We also use it to provide real-time support for our contact graph analysis and federated contact search systems. +

+
+Twitter + +

+Storm powers a wide variety of Twitter systems, ranging in applications from discovery, realtime analytics, personalization, search, revenue optimization, and many more. Storm integrates with the rest of Twitter's infrastructure, including database systems (Cassandra, Memcached, etc), the messaging infrastructure, Mesos, and the monitoring/alerting systems. Storm's isolation scheduler makes it easy to use the same cluster both for production applications and in-development applications, and it provides a sane way to do capacity planning. +

+
+Yahoo! + +

+Yahoo! is developing a next generation platform that enables the convergence of big-data and low-latency processing. While Hadoop is our primary technology for batch processing, Storm empowers stream/micro-batch processing of user events, content feeds, and application logs. +

+
+Yahoo! JAPAN + +

+Yahoo! JAPAN is a leading web portal in Japan. Storm applications are processing various streaming data such as logs or social data. We use Storm to feed contents, monitor systems, detect trending topics, and crawl on websites. +

+
+WebMD + +

+We use Storm to power our Medscape Medpulse mobile application which allow medical professionals to follow important medical trends with Medscape's curated Today on Twitter feed and selection of blogs. Storm topology is capturing and processing tweets with twitter streaming API, enhance tweets with metadata and images, do real time NLP and execute several business rules. Storm also monitors selection of blogs in order to give our customers real-time updates. We also use Storm for internal data pipelines to do ETL and for our internal marketing platform where time and freshness are essential. +

+

+We use storm to power our search indexing process. We continue to discover new use cases for storm and it became one of the core component in our technology stack. +

+
+Spotify + +

+Spotify serves streaming music to over 10 million subscribers and 40 million active users. Storm powers a wide range of real-time features at Spotify, including music recommendation, monitoring, analytics, and ads targeting. Together with Kafka, memcached, Cassandra, and netty-zmtp based messaging, Storm enables us to build low-latency fault-tolerant distributed systems with ease. +

+
+Infochimps + +

+Infochimps uses Storm as part of its Big Data Enterprise Cloud. Specifically, it uses Storm as the basis for one of three of its cloud data services - namely, Data Delivery Services (DDS), which uses Storm to provide a fault-tolerant and linearly scalable enterprise data collection, transport, and complex in-stream processing cloud service. +

+ +

+In much the same way that Hadoop provides batch ETL and large-scale batch analytical processing, the Data Delivery Service provides real-time ETL and large-scale real-time analytical processing — the perfect complement to Hadoop (or in some cases, what you needed instead of Hadoop). +

+ +

+DDS uses both Storm and Kafka along with a host of additional technologies to provide an enterprise-class real-time stream processing solution with features including: +

+ +
    +
  • +Integration connections to any variety of data sources in a way that is robust yet as non-invasive +
  • +
  • +Optimizations for highly scalable, reliable data import and distributed ETL (extract, transform, load), fulfilling data transport needs +
  • +
  • +Developer tools for rapid development of decorators, which perform the real-time stream processing +
  • +
  • +Guaranteed delivery framework and data failover snapshots to send processed data to analytics systems, databases, file systems, and applications with extreme reliability +
  • +
  • +Rapid solution development and deployment, along with our expert Big Data methodology and best practices +
  • +
+ +

Infochimps has extensive experience in deploying its DDS to power large-scale clickstream web data flows, massive Twitter stream processes, Foursquare event processing, customer purchase data, product pricing data, and more. +

+
+Health Market Science + +

+Health Market Science (HMS) provides data management as a service for the healthcare industry. Storm is at the core of the HMS big data platform functioning as the data ingestion mechanism, which orchestrates the data flow across multiple persistence mechanisms that allow HMS to deliver Master Data Management (MDM) and analytics capabilities for wide range of healthcare needs: compliance, integrity, data quality, and operational decision support. +

+
+Cerner + +

+Cerner is a leader in health care information technology. We have been using Storm since its release to process massive amounts of clinical data in real-time. Storm integrates well in our architecture, allowing us to quickly provide clinicians with the data they need to make medical decisions. +

+
+Aeris Communications + +

+Aeris Communications has the only cellular network that was designed and built exclusively for machines. Our ability to provide scalable, reliable real-time analytics - powered by Storm - for machine to machine (M2M) communication offers immense value to our customers. We are using Storm in production since Q1 of 2013. +

+
+Flipboard + +

+Flipboard is the worldʼs first social magazine, a single place to keep up with everything you care about and collect it in ways that let reflect you. Inspired by the beauty and ease of print media, Flipboard is designed so you can easily flip through news from around the world or stories from right at home, helping people find the one thing that can inform, entertain or even inspire them every day. +

+

+We are using Storm across a wide range of our services from content search, to realtime analytics, to generating custom magazine feeds. We then integrate Storm across our infrastructure within systems like ElasticSearch, HBase, Hadoop and HDFS to create a highly scalable data platform. +

+
+Rubicon Project + +

+Storm is being used in production mode at the Rubicon Project to analyze the results of auctions of ad impressions on its RTB exchange as they occur. It is currently processing around 650 million auction results in three data centers daily (with 3 separate Storm clusters). One simple application is identifying new creatives (ads) in real time for ad quality purposes. A more sophisticated application is an "Inventory Valuation Service" that uses DRPC to return appraisals of new impressions before the auction takes place. The appraisals are used for various optimization problems, such as deciding whether to auction an impression or skip it when close to maximum capacity. +

+
+Ooyala + +

+Ooyala powers personalized multi-screen video experiences for some of the world's largest networks, brands and media companies. We provide all the technology and tools our customers need to manage, distribute and monetize digital video content at a global scale. +

+ +

+At the core of our technology is an analytics engine that processes over two billion analytics events each day, derived from nearly 200 million viewers worldwide who watch video on an Ooyala-powered player. +

+ +

+Ooyala will be deploying Storm in production to give our customers real-time streaming analytics on consumer viewing behavior and digital content trends. Storm enables us to rapidly mine one of the world's largest online video data sets to deliver up-to-the-minute business intelligence ranging from real-time viewing patterns to personalized content recommendations to dynamic programming guides and dozens of other insights for maximizing revenue with online video. +

+
+Taobao + +

+We make statistics of logs and extract useful information from the statistics in almost real-time with Storm. Logs are read from Kafka-like persistent message queues into spouts, then processed and emitted over the topologies to compute desired results, which are then stored into distributed databases to be used elsewhere. Input log count varies from 2 millions to 1.5 billion every day, whose size is up to 2 terabytes among the projects. The main challenge here is not only real-time processing of big data set; storing and persisting result is also a challenge and needs careful design and implementation. +

+
+Alibaba + +

+Alibaba is the leading B2B e-commerce website in the world. We use storm to process the application log and the data change in database to supply realtime stats for data apps. +

+
+iQIYI + +

+iQIYI is China`s largest online video platform. We are using Storm in our video advertising system, video recommendation system, log analysis system and many other scenarios. Now we have several standalone Storm clusters, and we also have Storm clusters on Mesos and on Yarn. Kafka-Storm integration and Storm–HBase integration are quite common in our production environment. We have great interests in the new development about integration of Storm with other applications, like HBase, HDFS and Kafka. +

+
+Baidu + +

+Baidu offers top searching technology services for websites, audio files and images, my group using Storm to process the searching logs to supply realtime stats for accounting pv, ar-time and so on. +This project helps Ops to determine and monitor services status and can do great things in the future. +

+
+Yelp + +

+Yelp is using Storm with Pyleus to build a platform for developers to consume and process high throughput streams of data in real time. We have ongoing projects to use Storm and Pyleus for overhauling our internal application metrics pipeline, building an automated Python profile analysis system, and for general ETL operations. As its support for non-JVM components matures, we hope to make Storm the standard way of processing streaming data at Yelp. +

+
+Klout + +

+Klout helps everyone discover and be recognized for their influence by analyzing engagement with their content across social networks. Our analysis powers a daily Klout Score on a scale from 1-100 that shows how much influence social media users have and on what topics. We are using Storm to develop a realtime scoring and moments generation pipeline. Leveraging Storm's intuitive Trident abstraction we are able to create complex topologies which stream data from our network collectors via Kafka, processed and written out to HDFS. +

+
+Loggly + +

+Loggly is the world's most popular cloud-based log management. Our cloud-based log management service helps DevOps and technical teams make sense of the the massive quantity of logs that are being produced by a growing number of cloud-centric applications – in order to solve operational problems faster. Storm is the heart of our ingestion pipeline where it filters, parses and analyses billions of log events all-day, every day and in real-time. +

+
+premise.is + +

+We're building a platform for alternative, bottom-up, high-granularity econometric data capture, particularly targeting opaque developing economies (i.e., Argentina might lie about their inflation statistics, but their black market certainly doesn't). Basically we get to funnel hedge fund money into improving global economic transparency. +

+

+We've been using Storm in production since January 2012 as a streaming, time-indexed web crawl + extraction + machine learning-based semantic markup flow (about 60 physical nodes comparable to m1.large; generating a modest 25GB/hr incremental). We wanted to have an end-to-end push-based system where new inputs get percolated through the topology in realtime and appear on the website, with no batch jobs required in between steps. Storm has been really integral to realizing this goal. +

+
+Wego + +

About Wego, we are one of the world’s most comprehensive travel metasearch engines, operating in 42 markets worldwide and used by millions of travelers to save time, pay less and travel more. We compare and display real-time flights, hotel pricing and availability from hundreds of leading travel sites from all around the world on one simple screen.

+ +

At the heart of our products, Storm helps us to stream real-time meta-search data from our partners to end-users. Since data comes from many sources and with different timing, Storm topology concept naturally solves concurrency issues while helping us to continuously merge, slice and clean all the data. Additionally with a few tricks and tools provided in Storm we can easily apply incremental update to improve the flow our data (1-5GB/minute).

+ +

With its simplicity, scalability, and flexibility, Storm does not only improve our current products but more importantly changes the way we work with data. Instead of keeping data static and crunching it once a while, we constantly move data all around, making use of different technologies, evaluating new ideas and building new products. We stream critical data to memory for fast access while continuously crunching and directing huge amount of data into various engines so that we can evaluate and make use of data instantly. Previously, this kind of system requires to setup and maintain quite a few things but with Storm all we need is half day of coding and a few seconds to deploy. In this sense we never think Storm is to serve our products but rather to evolve our products.

+
+RocketFuel + +

+At Rocket Fuel (an ad network) we are building a real time platform on top of Storm which imitates the time critical workflows of existing Hadoop based ETL pipeline. This platform tracks impressions, clicks, conversions, bid requests etc. in real time. We are using Kafka as message queue. To start with we are pushing per minute aggregations directly to MySQL, but we plan to go finer than one minute and may bring HBase in to the picture to handle increased write load. +

+
+QuickLizard + +

+QuickLizard builds solution for automated pricing for companies that have many products in their lists. Prices are influenced by multiple factors internal and external to company. +

+ +

+Currently we use Storm to choose products that need to be priced. We get real time stream of events from client site and filters them to get much more light stream of products that need to be processed by our procedures to get price recommendation. +

+ +

+In plans: use Storm also for real time data mining model calculation that should match products described on competitor sites to client products. +

+
+spider.io + +

+At spider.io we've been using Storm as a core part of our classification engine since October 2011. We run Storm topologies to combine, analyse and classify real-time streams of internet traffic, to identify suspicious or undesirable website activity. Over the past 7 months we've expanded our use of Storm, so it now manages most of our real-time processing. Our classifications are displayed in a custom analytics dashboard, where Storm's distributed remote procedure call interface is used to gather data from our database and metadata services. DRPC allows us to increase the responsiveness of our user interface by distributing processing across a cluster of Amazon EC2 instances. +

+
+8digits + +

+At 8digits, we are using Storm in our analytics engine, which is one of the most crucial parts of our infrastructure. We are utilizing several cloud servers with multiple cores each for the purpose of running a real-time system making several complex calculations. Storm is a proven, solid and a powerful framework for most of the big-data problems. +

+
+Alipay + +

+Alipay is China's leading third-party online payment platform. We are using Storm in many scenarios: +

+ +
    +
  1. +Calculate realtime trade quantity, trade amount, the TOP N seller trading information, user register count. More than 100 million messages per day. +
  2. +
  3. +Log processing, more than 6T data per day. +
  4. +
+
+NaviSite + +

+We are using Storm as part of our server event log monitoring/auditing system. We send log messages from thousands of servers into a RabbitMQ cluster and then use Storm to check each message against a set of regular expressions. If there is a match (< 1% of messages), then the message is sent to a bolt that stores data in a Mongo database. Right now we are handling a load of somewhere around 5-10k messages per second, however we tested our existing RabbitMQ + Storm clusters up to about 50k per second. We have plans to do real time intrusion detection as an enhancement to the current log message reporting system. +

+ +

+We have Storm deployed on the NaviSite Cloud platform. We have a ZK cluster of 3 small VMs, 1 Nimbus VM and 16 dual core/4GB VMs as supervisors. +

+
+Glyph + +

+Glyph is in the business of providing credit card rewards intelligence to consumers. At a given point of sale Glyph suggest its users what are the best cards to be used at a given merchant location that will provide maximum rewards. Glyph also provide suggestion on the cards the user should carry to earn maximum rewards based on his personal spending habits. Glyph provides this information to the user by retrieving and analyzing credit card transactions from banks. Storm is used in Glyph to perform this retrieval and analysis in realtime. We are using Memcached in conjuction with Storm for handling sessions. We are impressed by how Storm makes high availability and reliability of Glyph services possible. We are now using Storm and Clojure in building Glyph data analytics and insights services. We have open-sourced node-drpc wrapper module for easy Storm DRPC integration with NodeJS. +

+
+Heartbyte + +

+At Heartbyte, Storm is a central piece of our realtime audience participation platform. We are often required to process a 'vote' per second from hundreds of thousands of mobile devices simultaneously and process / aggregate all of the data within a second. Further, we are finding that Storm is a great alternative to other ingest tools for Hadoop/HBase, which we use for batch processing after our events conclude. +

+
+2lemetry + +

+2lemetry uses Storm to power it's real time analytics on top of the m2m.io offering. 2lemetry is partnered with Sprint, Verizon, AT&T, and Arrow Electronics to power IoT applications world wide. Some of 2lemetry's larger projects include RTX, Kontron, and Intel. 2lemetry also works with many professional sporting teams to parse data in real time. 2lemetry receives events for every touch of the ball in every MLS soccer match. Storm is used to look for trends like passing tendencies as they develop during the game. +

+
+Nodeable + +

+Nodeable uses Storm to deliver real-time continuous computation of the data we consume. Storm has made it significantly easier for us to scale our service more efficiently while ensuring the data we deliver is timely and accurate. +

+
+TwitSprout + +

+At TwitSprout, we use Storm to analyze activity on Twitter to monitor mentions of keywords (mostly client product and brand names) and trigger alerts when activity around a certain keyword spikes above normal levels. We also use Storm to back the data behind the live-infographics we produce for events sponsored by our clients. The infographics are usually in the form of a live dashboard that helps measure the audience buzz across social media as it relates to the event in realtime. +

+
+HappyElements + +

+HappyElements is a leading social game developer on Facebook and other SNS platforms. We developed a real time data analysis program based on storm to analyze user activity in real time. Storm is very easy to use, stable, scalable and maintainable. +

+
+IDEXX Laboratories + +

+IDEXX Laboratories is the leading maker of software and diagnostic instruments for the veterinary market. We collect and analyze veterinary medical data from thousands of veterinary clinics across the US. We recently embarked on a project to upgrade our aging data processing infrastructure that was unable to keep up with the rapid increase in the volume, velocity and variety of data that we were processing. +

+ +

+We are utilizing the Storm system to take in the data that is extracted from the medical records in a number of different schemas, transform it into a standard schema that we created and store it in an Oracle RDBMS database. It is basically a souped up distributed ETL system. Storm takes on the plumbing necessary for a distributed system and is very easy to write code for. The ability to create small pieces of functionality and connect them together gives us the ultimate flexibility to parallelize each of the pieces differently. +

+ +

+Our current cluster consists of four supervisor machines running 110 tasks inside 32 worker processes. We run two different topologies which receive messages and communicate with each other via RabbitMQ. The whole thing is deployed on Amazon Web Services and utilizes S3 for some intermediate storage, Redis as a key/value store and Oracle RDS for RDBMS storage. The bolts are all written in Java using the Spring framework with Hibernate as an ORM. +

+
+Umeng + +Umeng is the leading and largest provider of mobile app analytics and developer services platform in China. Storm powers Umeng's realtime analytics platform, processing billions of data points per day and growing. We also use Storm in other products which requires realtime processing and it has become the core infrastructure in our company. +
+Admaster + +

+We provide monitoring and precise delivery for Internet advertising. We use Storm to do the following: +

+ +
    +
  1. Calculate PV, UV of every advertisement.
  2. +
  3. Simple data cleaning: filter out data which format error, filter out cheating data (the pv more than certain value)
  4. +
+Our cluster has 8 nodes, process several billions messages per day, about 200GB. +
+SocialMetrix + +

+Since its release, Storm was a perfect fit to our needs of real time monitoring. Its powerful API, easy administration and deploy, enabled us to rapidly build solutions to monitor presidential elections, several major events and currently it is the processing core of our new product "Socialmetrix Eventia". +

+
+Needium + +

+At Needium we love Ruby and JRuby. The Storm platform offers the right balance between simplicity, flexibility and scalability. We created RedStorm, a Ruby DSL for Storm, to keep on using Ruby on top of the power of Storm by leveraging Storm's JVM foundation with JRuby. We currently use Storm as our Twitter realtime data processing pipeline. We have Storm topologies for content filtering, geolocalisation and classification. Storm allows us to architecture our pipeline for the Twitter full firehose scale. +

+
+Parse.ly + +

+Parse.ly is using Storm for its web/content analytics system. We have a home-grown data processing and storage system built with Python and Celery, with backend stores in Redis and MongoDB. We are now using Storm for real-time unique visitor counting and are exploring options for using it for some of our richer data sources such as social share data and semantic content metadata. +

+
+PARC + +

+High Performance Graph Analytics & Real-time Insights Research team at PARC uses Storm as one of the building blocks of their PARC Analytics Cloud infrastructure which comprises of Nebula based Openstack, Hadoop, SAP HANA, Storm, PARC Graph Analytics, and machine learning toolbox to enable researchers to process real-time data feeds from Sensors, web, network, social media, and security traces and easily ingest any other real-time data feeds of interest for PARC researchers. +

+

+PARC researchers are working with number of industry collaborators developing new tools, algorithms, and models to analyze massive amounts of e-commerce, web clickstreams, 3rd party syndicated data, cohort data, social media data streams, and structured data from RDBMS, NOSQL, and NEWSQL systems in near real-time. PARC team is developing a reference architecture and benchmarks for their near real-time automated insight discovery platform combining the power of all above tools and PARC’s applied research in machine learning, graph analytics, reasoning, clustering, and contextual recommendations. The High Performance Graph Analytics & Real-time Insights research at PARC is headed by Surendra Reddy. If you are interested to learn more about our use/experience of using Storm or to know more about our research or to collaborate with PARC in this area, please feel free to contact sureddy@parc.com. +

+
+GumGum + +

+GumGum, the leading in-image advertising platform for publishers and brands, uses Storm to produce real-time data. Storm and Trident-based topologies consume various ad-related events from Kafka and persist the aggregations in MySQL and HBase. This architecture will eventually replace most existing daily Hadoop map reduce jobs. There are also plans for Kafka + Storm to replace existing distributed queue processing infrastructure built with Amazon SQS. +

+
+CrowdFlower + +

+CrowdFlower is using Storm with Kafka to generalize our data stream +aggregation and realtime computation infrastructure. We replaced our +homegrown aggregation solutions with Storm because it simplified the +creation of fault tolerant systems. We were already using Zookeeper +and Kafka, so Storm allowed us to build more generic abstractions for +our analytics using tools that we had already deployed and +battle-tested in production. +

+ +

+We are currently writing to DynamoDB from Storm, so we are able to +scale our capacity quickly by bringing up additional supervisors and +tweaking the throughput on our Dynamo tables. We look forward to +exploring other uses for Storm in our system, especially with the +recent release of Trident. +

+
+Digital Sandbox + +

+At Digital Sandbox we use Storm to enable our open source information feed monitoring system. The system uses Storm to constantly monitor and pull data from structured and unstructured information sources across the internet. For each found item, our topology applies natural language processing based concept analysis, temporal analysis, geospatial analytics and a prioritization algorithm to enable users to monitor large special events, public safety operations, and topics of interest to a multitude of individual users and teams. +

+ +

+Our system is built using Storm for feed retrieval and annotation, Python with Flask and jQuery for business logic and web interfaces, and MongoDB for data persistence. We use NTLK for natural language processing and the WordNet, GeoNames, and OpenStreetMap databases to enable feed item concept extraction and geolocation. +

+
+Hallo + +With several mainstream celebrities and very popular YouTubers using Hallo to communicate with their fans, we needed a good solution to notify users via push notifications and make sure that the celebrity messages were delivered to follower timelines in near realtime. Our initial approach for broadcast push notifications would take anywhere from 2-3 hours. After re-engineering our solution on top of Storm, that time has been cut down to 5 minutes on a very small cluster. With the user base growing and user need for realtime communication, we are very happy knowing that we can easily scale Storm by adding nodes to maintain a baseline QoS for our users. +
+Keepcon + +We provide moderation services for classifieds, kids communities, newspapers, chat rooms, facebook fan pages, youtube channels, reviews, and all kind of UGC. We use storm for the integration with our clients, find evidences within each text, persisting on cassandra and elastic search and sending results back to our clients. +
+Visible Measures + +

+Visible Measures powers video campaigns and analytics for publishers and +advertisers, tracking data for hundreds of million of videos, and billions +of views. We are using Storm to process viewing behavior data in real time and make +the information immediately available to our customers. We read events from +various push and pull sources, including a Kestrel queue, filter and +enrich the events in Storm topologies, and persist the events to Redis, +HDFS and Vertica for real-time analytics and archiving. We are currently +experimenting with Trident topologies, and figuring out how to move more +of our Hadoop-based batch processing into Storm. +

+
+O2mc + +

+One of the core products of O2mc is called O2mc Community. O2mc Community performs multilingual, realtime sentiment analysis with very low latency and distributes the analyzed results to numerous clients. The input is extracted from source systems like Twitter, Facebook, e-mail and many more. After the analysis has taken place on Storm, the results are streamed to any output system ranging from HTTP streaming to clients to direct database insertion to an external business process engine to kickstart a process.

+
+The Ladders + +

+TheLadders has been committed to finding the right person for the right job since 2003. We're using Storm in a variety of ways and are happy with its versatility, robustness, and ease of development. We use Storm in conjunction with RabbitMQ for such things as sending hiring alerts: when a recruiter submits a job to our site, Storm processes that event and will aggregate jobseekers whose profiles match the position. That list is subsequently batch-processed to send an email to the list of jobseekers. We also use Storm to persist events for Business Intelligence and internal event tracking. We're continuing to find uses for Storm where fast, asynchronous, real-time event processing is a must. +

+
+SemLab + +

+SemLab develops software for knowledge discovery and information support. Our ViewerPro platform uses information extraction, natural language processing and semantic web technologies to extract structured data from unstructured sources, in domains such as financial news feeds and legal documents. We have succesfully adapted ViewerPro's processing framework to run on top of Storm. The transition to Storm has made ViewerPro a much more scalable product, allowing us to process more in less time. +

+
+Visual Revenue + +

+Here at Visual Revenue, we built a decision support system to help online editors to make choices on what, when, and where to promote their content in real-time. Storm is the backbone our real-time data processing and aggregation pipelines. +

+
+PeerIndex + +

+PeerIndex is working to deliver influence at scale. PeerIndex does this by exposing services built on top of our Influence Graph; a directed graph of who is influencing whom on the web. PeerIndex gathers data from a number of social networks to create the Influence Graph. We use Storm to process our social data, to provide real-time aggregations, and to crawl the web, before storing our data in a manner most suitable for our Hadoop based systems to batch process. Storm provided us with an intuitive API and has slotted in well with the rest of our architecture. PeerIndex looks forward to further investing resources into our Storm based real-time analytics. +

+
+ANTS.VN + +

+Big Data in Advertising is Vietnam's unique platform combines ad serving, a real-time bidding (RTB) exchange, Ad Server, Analytics, yield optimization, and content valuation to deliver the highest revenue across every desktop, tablet, and mobile screen. At ANTS.VN we use Storm to process large amounts of data to provide data real time, improve our Ad quality. This platform tracks impressions, clicks, conversions, bid requests etc. in real time. Together with Kafka, Redis, memcached and Cassandra based messaging, Storm enables us to build low-latency fault-tolerant distributed systems with ease. +

+
+Wayfair + +

+At Wayfair, we use storm as a platform to drive our core order processing pipeline as an event driven system. Storm allows us to reliably process tens of thousands of orders daily while providing us the assurance of seamless process scalability as our order load increases. Given the project’s ease of use and the immense support of the community, we’ve managed to implement our bolts in php, construct a simple puppet module for configuration management, and quickly solve arising issues. We can now focus most of our development efforts in the business layer, check out more information on how we use storm in our engineering blog.

+
+InnoQuant + +

+At InnoQuant, we use Storm as a backbone of our real-time big data analytics engine in MOCA platform. MOCA is a next generation, mobile-backend-as-a-service platform (MBaaS). It provides brands and app developers with real-time in-app tracking, context-aware push messaging, user micro-segmentation based on profile, time and geo-context as well as big data analytics. Storm-based pipeline is fed with events captured by native mobile SDKs (iOS, Android), scales nicely with connected mobile app users, delivers stream-based metrics and aggregations, and finally integrates with the rest of MOCA infrastructure, including columnar storage (Cassandra) and graph storage (Titan). +

+
+Fliptop + +

+Fliptop is a customer intelligence platform which allows customers to integrating their contacts, and campaign data, to enhance their prospect with social identities, and to find their best leads, and most influential customers. We have been using Storm for various tasks which requires scalability and reliability, including integrating with sales/marketing platform, data appending for contacts/leads, and computing scoring of contacts/leads. It's one of our most robust and scalable infrastructure. +

+
+Trovit + +

+Trovit is a search engine for classified ads present in 39 countries and different business categories (Real Estate, Cars, Jobs, Rentals, Products and Deals). Currently we use Storm to process and index ads in a distributed and low latency fashion. Combined with other technologies like Hadoop, Hbase and Solr has allowed us to build a scalable and low latency platform to serve search results to the end user. +

+
+OpenX + +

+OpenX is a unique platform combines ad serving, a real-time bidding (RTB) exchange, yield optimization, and content valuation to deliver the highest revenue across every desktop, tablet, and mobile screen +At OpenX we use Storm to process large amounts of data to provide real time Analytics. Storm provides us to process data real time to improve our Ad quality. +

+
+Keen IO + +

+Keen IO is an analytics backend-as-a-service. The Keen IO API makes it easy for customers to do internal analytics or expose analytics features to their customers. Keen IO uses Storm (DRPC) to query billion-event data sets at very low latencies. We also use Storm to control our ingestion pipeline, sourcing data from Kafka and storing it in Cassandra. +

+
+LivePerson + +

+LivePerson is a provider of Interaction-Service over the web. Interaction between an agent and a visitor in site can be achieved using phone call, chat, banners, etc.Using Storm, LivePerson can collect and process visitor data and provide information in real time to the agents about the visitor behavior. Moreover, LivePerson gets to better decisions about how to react to visitors in a way that best addresses their needs. +

+
+YieldBot + +

+Yieldbot connects ads to the real-time consumer intent streaming within premium publishers. To do this, Yieldbot leverages Storm for a wide variety of real-time processing tasks. We've open sourced our clojure DSL for writing trident topologies, marceline, which we use extensively. Events are read from Kafka, most state is stored in Cassandra, and we heavily use Storm's DRPC features. Our Storm use cases range from HTML processing, to hotness-style trending, to probabilistic rankings and cardinalities. Storm topologies touch virtually all of the events generated by the Yieldbot platform. +

+
+Equinix + +

+At Equinix, we use a number of Storm topologies to process and persist various data streams generated by sensors in our data centers. We also use Storm for real-time monitoring of different infrastructure components. Other few topologies are used for processing logs in real-time for internal IT systems which also provide insights in user behavior. +

+
+MineWhat + +

+MineWhat provides actionable analytics for ecommerce spanning every SKU,brand and category in the store. We use Storm to process raw click stream ingestion from Kafka and compute live analytics. Storm topologies powers our complex product to user interaction analysis. Multi language feature in storm is really kick-ass, we have bolts written in Node.js, Python and Ruby. Storm has been in our production site since Nov 2012. +

+
+Qihoo 360 + +

+360 have deployed about 50 realtime applications on top of storm including web page analysis, log processing, image processing, voice processing, etc. +

+

+The use case of storm at 360 is a bit special since we deployed storm on thounds of servers which are not dedicated for storm. Storm just use little cpu/memory/network resource on each server. However theses storm clusters leverage idle resources of servers at nearly zero cost to provide great computing power and it's realtime. It's amazing. +

+
+HolidayCheck + +

+HolidayCheck is an online travel site and agency available in 10 +languages worldwide visited by 30 million people a month. +We use Storm to deliver real-time hotel and holiday package offers +from multiple providers - reservation systems and affiliate travel +networks - in a low latency fashion based on user-selected criteria. +In further reservation steps we use DRPC for vacancy checks and +bookings of chosen offers. Along with Storm in the system for offers +delivery we use Scala, Akka, Hazelcast, Drools and MongoDB. Real-time +offer stream is delivered outside of the system back to the front-end +via websocket connections. +

+
+DataMine Lab + +

+DataMine Lab is a consulting company integrating Storm into its +portfolio of technologies. Storm powers range of our customers' +systems allowing us to build real time analytics on tens of millions +of visitors to the advertising platforms we helped to create. Together +with Redis, Cassandra and Hadoop, Storm allows us to provide real-time +distributed data platform at a global scale. +

+
+Wize Commerce + +

+Wize Commerce® is the smartest way to grow your digital business. For over ten years, we have been helping clients maximize their revenue and traffic using optimization technologies that operate at massive scale, and across digital ecosystems. We own and operate leading comparison shopping engines including Nextag®, PriceMachineTM, and guenstiger.de, and provide services to a wide ecosystem of partner sites that use our e-commerce platform. These sites together drive over $1B in annual merchant sales. +

+

+We use storm to power our core platform infrastructure and it has become a vital component of our search indexing system & Cassandra storage. Along with KAFKA, STORM has reduced our end-to-end latencies from several hours to few minutes, and being largest comparison shopping sites operator, pushing price updates to the live site is very important and storm helps a lot achieve the same. We are extensively using storm in production since Q1 2013. +

+
+Metamarkets + +

At Metamarkets, Apache Storm is used to process real-time event data streamed from Apache Kafka message brokers, and then to load that data into a Druid cluster, the low-latency data store at the heart of our real-time analytics service. Our Storm topologies perform various operations, ranging from simple filtering of "outdated" events, to transformations such as ID-to-name lookups, to complex multi-stream joins. Since our service is intended to respond to ad-hoc queries within seconds of ingesting events, the speed, flexibility, and robustness of those topologies make Storm a key piece of our real-time stack.

+
+Mighty Travels + +

We are using Storm to process real-time search data stream and +application logs. The part we like best about Storm is the ease of +scaling up basically just by throwing more machines at it.

+
+Polecat + +

Polecat's digital analyisis platform, MeaningMine, allows users to search all on-line news, blogs and social media in real-time and run bespoke analysis in order to inform corporate strategy and decision making for some of the world largest companies and governmental organisations.

+

+Polecat uses Storm to run an application we've called the 'Data Munger'. We run many different topologies on a multi host storm cluster to process tens of millions of online articles and posts that we collect each day. Storm handles our analysis of these documents so that we can provide insight on realtime data to our clients. We output our results from Storm into one of many large Apache Solr clusters for our end user applications to query (Polecat is also a contributor to Solr). We first starting developing our app to run on storm back in June 2012 and it has been live since roughly September 2012. We've found Storm to be an excellent fit for our needs here, and we've always found it extremely robust and fast. +

+
+Skylight by Tilde + +

Skylight is a production profiler for Ruby on Rails apps that focuses on providing detailed information about your running application that you can explore in an intuitive way. We use Storm to process traces from our agent into data structures that we can slice and dice for you in our web app.

+
+Ad4Game + +

We are an advertising network and we use Storm to calculate priorities in real time to know which ads to show for which website, visitor and country.

+
+Impetus Technologies + +

StreamAnalytix, a product of Impetus Technologies enables enterprises to analyze and respond to events in real-time at Big Data scale. Based on Apache Storm, StreamAnalytix is designed to rapidly build and deploy streaming analytics applications for any industry vertical, any data format, and any use case. This high-performance scalable platform comes with a pre-integrated package of components like Cassandra, Storm, Kafka and more. In addition, it also brings together the proven open source technology stack with Hadoop and NoSQL to provide massive scalability, dynamic data pipelines, and a visual designer for rapid application development.

+

+Through StreamAnalytix, the users can ingest, store and analyze millions of events per second and discover exceptions, patterns, and trends through live dashboards. It also provides seamless integration with indexing store (ElasticSearch) and NoSQL database (HBase, Cassandra, and Oracle NoSQL) for writing data in real-time. With the use of Storm, the product delivers high business value solutions such as log analytics, streaming ETL, deep social listening, Real-time marketing, business process acceleration and predictive maintenance. +

+
+Akazoo + +

+Akazoo is a platform providing music streaming services. Storm is the backbone of all our real-time analytical processing. We use it for tracking and analyzing application events and for various other stuff, including recommendations and parallel task execution. +

+
+Mapillary + +

+At Mapillary we use storm for a wide variety of tasks. Having a system which is 100% based on kafka input storm and trident makes reasoning about our data a breeze. +

+
+Gutscheinrausch.de + +

+We recently upgraded our existing IT infrastructure, using Storm as one of our main tools. +Each day we collect sales, clicks, visits and various ecommerce metrics from various different systems (webpages, affiliate reportings, networks, tracking-scripts etc). We process this continually generated data using Storm before entering it into the backend systems for further use. +

+

+Using Storm we were able to decouple our heterogeneous frontend-systems from our backends and take load off the data warehouse applications by inputting pre-processed data. This way we can easy collect and process all data and then do realtime OLAP queries using our propietary data warehouse technology. +

+

+We are mostly impressed by the high speed, low maintenance approach Storm has provided us with. Also being able to easily scale up the system using more machines is a big plus. Since we're a small team it allows us to focus more on our core business instead of the underlying technology. You could say it has taken our hearts by storm! +

+
+AppRiver + +

+We are using Storm to track internet threats from varied sources around the web. It is always fast and reliable. +

+
+MercadoLibre + +
diff --git a/docs/Project-ideas.md b/docs/Project-ideas.md new file mode 100644 index 00000000000..aa022ea4581 --- /dev/null +++ b/docs/Project-ideas.md @@ -0,0 +1,6 @@ +--- +layout: documentation +--- + * **DSLs for non-JVM languages:** These DSL's should be all-inclusive and not require any Java for the creation of topologies, spouts, or bolts. Since topologies are [Thrift](http://thrift.apache.org/) structs, Nimbus is a Thrift service, and bolts can be written in any language, this is possible. + * **Online machine learning algorithms:** Something like [Mahout](http://mahout.apache.org/) but for online algorithms + * **Suite of performance benchmarks:** These benchmarks should test Storm's performance on CPU and IO intensive workloads. There should be benchmarks for different classes of applications, such as stream processing (where throughput is the priority) and distributed RPC (where latency is the priority). diff --git a/docs/README.md b/docs/README.md new file mode 100644 index 00000000000..b26d3ff7d80 --- /dev/null +++ b/docs/README.md @@ -0,0 +1,61 @@ +# Apache Storm Website and Documentation +This is the source for the Release specific part of the Apache Storm website and documentation. It is statically generated using [jekyll](http://jekyllrb.com). + +## Generate Javadoc + +You have to generate javadoc on project root before generating document site. + +``` +mvn javadoc:javadoc +mvn javadoc:aggregate -DreportOutputDirectory=./docs/ -DdestDir=javadocs +``` + +You need to create distribution package with gpg certificate. Please refer [here](https://github.com/apache/storm/blob/master/DEVELOPER.md#packaging). + +## Site Generation +First install jekyll (assuming you have ruby installed): + +``` +gem install jekyll +``` + +Generate the site, and start a server locally: +``` +cd docs +jekyll serve -w +``` + +The `-w` option tells jekyll to watch for changes to files and regenerate the site automatically when any content changes. + +Point your browser to http://localhost:4000 + +By default, jekyll will generate the site in a `_site` directory. + +This will only show the portion of the documentation that is specific to this release. + +## Adding a new release to the website +In order to add a new relase, you must have committer access to Storm's subversion repository at https://svn.apache.org/repos/asf/storm/site. + +Release documentation is placed under the releases directory named after the release version. Most metadata about the release will be generated automatically from the name using a jekyll plugin. Or by plaing them in the _data/releases.yml file. + +To create a new release run the following from the main git directory + +``` +mvn javadoc:javadoc +mvn javadoc:aggregate -DreportOutputDirectory=./docs/ -DdestDir=javadocs +cd docs +mkdir ${path_to_svn}/releases/${release_name} +cp -r *.md images/ javadocs/ ${path_to_svn}/releases/${release_name} +cd ${path_to_svn} +svn add releases/${release_name} +svn commit +``` + +to publish a new release run + +``` +cd ${path_to_svn} +jekyll build -d publish/ +svn add publish/ #Add any new files +svn commit +``` diff --git a/docs/Rationale.md b/docs/Rationale.md new file mode 100644 index 00000000000..214266ebb55 --- /dev/null +++ b/docs/Rationale.md @@ -0,0 +1,31 @@ +--- +layout: documentation +--- +The past decade has seen a revolution in data processing. MapReduce, Hadoop, and related technologies have made it possible to store and process data at scales previously unthinkable. Unfortunately, these data processing technologies are not realtime systems, nor are they meant to be. There's no hack that will turn Hadoop into a realtime system; realtime data processing has a fundamentally different set of requirements than batch processing. + +However, realtime data processing at massive scale is becoming more and more of a requirement for businesses. The lack of a "Hadoop of realtime" has become the biggest hole in the data processing ecosystem. + +Storm fills that hole. + +Before Storm, you would typically have to manually build a network of queues and workers to do realtime processing. Workers would process messages off a queue, update databases, and send new messages to other queues for further processing. Unfortunately, this approach has serious limitations: + +1. **Tedious**: You spend most of your development time configuring where to send messages, deploying workers, and deploying intermediate queues. The realtime processing logic that you care about corresponds to a relatively small percentage of your codebase. +2. **Brittle**: There's little fault-tolerance. You're responsible for keeping each worker and queue up. +3. **Painful to scale**: When the message throughput get too high for a single worker or queue, you need to partition how the data is spread around. You need to reconfigure the other workers to know the new locations to send messages. This introduces moving parts and new pieces that can fail. + +Although the queues and workers paradigm breaks down for large numbers of messages, message processing is clearly the fundamental paradigm for realtime computation. The question is: how do you do it in a way that doesn't lose data, scales to huge volumes of messages, and is dead-simple to use and operate? + +Storm satisfies these goals. + +## Why Storm is important + +Storm exposes a set of primitives for doing realtime computation. Like how MapReduce greatly eases the writing of parallel batch processing, Storm's primitives greatly ease the writing of parallel realtime computation. + +The key properties of Storm are: + +1. **Extremely broad set of use cases**: Storm can be used for processing messages and updating databases (stream processing), doing a continuous query on data streams and streaming the results into clients (continuous computation), parallelizing an intense query like a search query on the fly (distributed RPC), and more. Storm's small set of primitives satisfy a stunning number of use cases. +2. **Scalable**: Storm scales to massive numbers of messages per second. To scale a topology, all you have to do is add machines and increase the parallelism settings of the topology. As an example of Storm's scale, one of Storm's initial applications processed 1,000,000 messages per second on a 10 node cluster, including hundreds of database calls per second as part of the topology. Storm's usage of Zookeeper for cluster coordination makes it scale to much larger cluster sizes. +3. **Guarantees no data loss**: A realtime system must have strong guarantees about data being successfully processed. A system that drops data has a very limited set of use cases. Storm guarantees that every message will be processed, and this is in direct contrast with other systems like S4. +4. **Extremely robust**: Unlike systems like Hadoop, which are notorious for being difficult to manage, Storm clusters just work. It is an explicit goal of the Storm project to make the user experience of managing Storm clusters as painless as possible. +5. **Fault-tolerant**: If there are faults during execution of your computation, Storm will reassign tasks as necessary. Storm makes sure that a computation can run forever (or until you kill the computation). +6. **Programming language agnostic**: Robust and scalable realtime processing shouldn't be limited to a single platform. Storm topologies and processing components can be defined in any language, making Storm accessible to nearly anyone. diff --git a/docs/Running-topologies-on-a-production-cluster.md b/docs/Running-topologies-on-a-production-cluster.md new file mode 100644 index 00000000000..248c929a66f --- /dev/null +++ b/docs/Running-topologies-on-a-production-cluster.md @@ -0,0 +1,75 @@ +--- +layout: documentation +--- +Running topologies on a production cluster is similar to running in [Local mode](Local-mode.html). Here are the steps: + +1) Define the topology (Use [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html) if defining using Java) + +2) Use [StormSubmitter](javadocs/backtype/storm/StormSubmitter.html) to submit the topology to the cluster. `StormSubmitter` takes as input the name of the topology, a configuration for the topology, and the topology itself. For example: + +```java +Config conf = new Config(); +conf.setNumWorkers(20); +conf.setMaxSpoutPending(5000); +StormSubmitter.submitTopology("mytopology", conf, topology); +``` + +3) Create a jar containing your code and all the dependencies of your code (except for Storm -- the Storm jars will be added to the classpath on the worker nodes). + +If you're using Maven, the [Maven Assembly Plugin](http://maven.apache.org/plugins/maven-assembly-plugin/) can do the packaging for you. Just add this to your pom.xml: + +```xml + + maven-assembly-plugin + + + jar-with-dependencies + + + + com.path.to.main.Class + + + + +``` +Then run mvn assembly:assembly to get an appropriately packaged jar. Make sure you [exclude](http://maven.apache.org/plugins/maven-assembly-plugin/examples/single/including-and-excluding-artifacts.html) the Storm jars since the cluster already has Storm on the classpath. + +4) Submit the topology to the cluster using the `storm` client, specifying the path to your jar, the classname to run, and any arguments it will use: + +`storm jar path/to/allmycode.jar org.me.MyTopology arg1 arg2 arg3` + +`storm jar` will submit the jar to the cluster and configure the `StormSubmitter` class to talk to the right cluster. In this example, after uploading the jar `storm jar` calls the main function on `org.me.MyTopology` with the arguments "arg1", "arg2", and "arg3". + +You can find out how to configure your `storm` client to talk to a Storm cluster on [Setting up development environment](Setting-up-development-environment.html). + +### Common configurations + +There are a variety of configurations you can set per topology. A list of all the configurations you can set can be found [here](javadocs/backtype/storm/Config.html). The ones prefixed with "TOPOLOGY" can be overridden on a topology-specific basis (the other ones are cluster configurations and cannot be overridden). Here are some common ones that are set for a topology: + +1. **Config.TOPOLOGY_WORKERS**: This sets the number of worker processes to use to execute the topology. For example, if you set this to 25, there will be 25 Java processes across the cluster executing all the tasks. If you had a combined 150 parallelism across all components in the topology, each worker process will have 6 tasks running within it as threads. +2. **Config.TOPOLOGY_ACKERS**: This sets the number of tasks that will track tuple trees and detect when a spout tuple has been fully processed. Ackers are an integral part of Storm's reliability model and you can read more about them on [Guaranteeing message processing](Guaranteeing-message-processing.html). +3. **Config.TOPOLOGY_MAX_SPOUT_PENDING**: This sets the maximum number of spout tuples that can be pending on a single spout task at once (pending means the tuple has not been acked or failed yet). It is highly recommended you set this config to prevent queue explosion. +4. **Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS**: This is the maximum amount of time a spout tuple has to be fully completed before it is considered failed. This value defaults to 30 seconds, which is sufficient for most topologies. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more information on how Storm's reliability model works. +5. **Config.TOPOLOGY_SERIALIZATIONS**: You can register more serializers to Storm using this config so that you can use custom types within tuples. + + +### Killing a topology + +To kill a topology, simply run: + +`storm kill {stormname}` + +Give the same name to `storm kill` as you used when submitting the topology. + +Storm won't kill the topology immediately. Instead, it deactivates all the spouts so that they don't emit any more tuples, and then Storm waits Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS seconds before destroying all the workers. This gives the topology enough time to complete any tuples it was processing when it got killed. + +### Updating a running topology + +To update a running topology, the only option currently is to kill the current topology and resubmit a new one. A planned feature is to implement a `storm swap` command that swaps a running topology with a new one, ensuring minimal downtime and no chance of both topologies processing tuples at the same time. + +### Monitoring topologies + +The best place to monitor a topology is using the Storm UI. The Storm UI provides information about errors happening in tasks and fine-grained stats on the throughput and latency performance of each component of each running topology. + +You can also look at the worker logs on the cluster machines. diff --git a/docs/SECURITY.md b/docs/SECURITY.md new file mode 100644 index 00000000000..495061a12d9 --- /dev/null +++ b/docs/SECURITY.md @@ -0,0 +1,79 @@ +--- +title: Running Apache Storm Securely +layout: documentation +documentation: true +--- +# 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. + +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. + +If the data your cluster is processing is sensitive it might be best to setup +IPsec to encrypt all traffic being sent between the hosts in the cluster. + +### Ports + +| Default Port | Storm Config | Client Hosts/Processes | Server | +|--------------|--------------|------------------------|--------| +| 2181 | `storm.zookeeper.port` | Nimbus, Supervisors, and Worker processes | Zookeeper | +| 6627 | `nimbus.thrift.port` | Storm clients, Supervisors, and UI | Nimbus | +| 8080 | `ui.port` | Client Web Browsers | UI | +| 8000 | `logviewer.port` | Client Web Browsers | Logviewer | +| 3772 | `drpc.port` | External DRPC Clients | DRPC | +| 3773 | `drpc.invocations.port` | Worker Processes | DRPC | +| 670{0,1,2,3} | `supervisor.slots.ports` | Worker Processes | Worker Processes | + +### UI/Logviewer + +The UI and logviewer processes provide a way to not only see what a cluster is +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 +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 + +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. + +### DRPC + +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. + +### Supervisors + +Supervisors are only clients they are not servers, and as such don't need special restrictions. + +### Workers + +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. + +### 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/docs/STORM-UI-REST-API.md b/docs/STORM-UI-REST-API.md new file mode 100644 index 00000000000..2109ab2224e --- /dev/null +++ b/docs/STORM-UI-REST-API.md @@ -0,0 +1,678 @@ +--- +title: Storm UI REST API +layout: documentation +documentation: true +--- + +# Storm UI REST API + +The Storm UI daemon provides a REST API that allows you to interact with a Storm cluster, which includes retrieving +metrics data and configuration information as well as management operations such as starting or stopping topologies. + + +# Data format + +The REST API returns JSON responses and supports JSONP. +Clients can pass a callback query parameter to wrap JSON in the callback function. + + +# Using the UI REST API + +_Note: It is recommended to ignore undocumented elements in the JSON response because future versions of Storm may not_ +_support those elements anymore._ + + +## REST API Base URL + +The REST API is part of the UI daemon of Storm (started by `storm ui`) and thus runs on the same host and port as the +Storm UI (the UI daemon is often run on the same host as the Nimbus daemon). The port is configured by `ui.port`, +which is set to `8080` by default (see [defaults.yaml](conf/defaults.yaml)). + +The API base URL would thus be: + + http://:/api/v1/... + +You can use a tool such as `curl` to talk to the REST API: + + # Request the cluster configuration. + # Note: We assume ui.port is configured to the default value of 8080. + $ curl http://:8080/api/v1/cluster/configuration + +##Impersonating a user in secure environment +In a secure environment an authenticated user can impersonate another user. To impersonate a user the caller must pass +`doAsUser` param or header with value set to the user that the request needs to be performed as. Please see SECURITY.MD +to learn more about how to setup impersonation ACLs and authorization. The rest API uses the same configs and acls that +are used by nimbus. + +Examples: + +```no-highlight + 1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1425844354\?doAsUser=testUSer1 + 2. curl 'http://localhost:8080/api/v1/topology/wordcount-1-1425844354/activate' -X POST -H 'doAsUser:testUSer1' +``` + +## GET Operations + +### /api/v1/cluster/configuration (GET) + +Returns the cluster configuration. + +Sample response (does not include all the data fields): + +```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 information such as nimbus uptime or number of supervisors. + +Response fields: + +|Field |Value|Description +|--- |--- |--- +|stormVersion|String| Storm version| +|nimbusUptime|String| Shows how long the cluster is running| +|supervisors|Integer| Number of supervisors running| +|topologies| Integer| Number of topologies 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 + { + "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 summary information for all supervisors. + +Response fields: + +|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 +{ + "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 summary information for all topologies. + +Response fields: + +|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 +{ + "topologies": [ + { + "id": "WordCount3-1-1402960825", + "name": "WordCount3", + "status": "ACTIVE", + "uptime": "6m 5s", + "tasksTotal": 28, + "workersTotal": 3, + "executorsTotal": 28 + } + ] +} +``` + +### /api/v1/topology/:id (GET) + +Returns topology information and statistics. Substitute 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 seconds| +|sys |String. Values 1 or 0. Default value 0| Controls including sys stats part of the response| + + +Response fields: + +|Field |Value |Description| +|--- |--- |--- +|id| String| Topology Id| +|name| String |Topology Name| +|uptime| String |How long the topology has been running| +|status| String |Current status of the topology, e.g. "ACTIVE"| +|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.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 messages executed * average execute latency / time window| +|bolts.processLatency| String (double value returned in String format) |Average time of the bolt to ack a message after it was received| +|bolts.executeLatency| String (double value returned in String format) |Average time to run the execute method of the bolt| +|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 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| + +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": "", + "errorLapsedSecs": null, + "failed": 0 + } + ], + "bolts": [ + { + "executors": 12, + "emitted": 184580, + "transferred": 0, + "acked": 184640, + "executeLatency": "0.048", + "tasks": 12, + "executed": 184620, + "processLatency": "0.043", + "boltId": "count", + "lastError": "", + "errorLapsedSecs": null, + "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": "", + "errorLapsedSecs": null, + "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 seconds| +|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 id| +|name | String | Topology name| +|componentType | String | component 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 Errors | List of component errors| +|componentErrors.time| Long | Timestamp when the exception occurred | +|componentErrors.errorHost| String | host name for the error| +|componentErrors.errorPort| String | port for the error| +|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 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) |Average time of the bolt to ack a message after it was received| +|boltStats.acked| Long |Number of messages acked| +|boltStats.failed| Long |Number of messages failed| + +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":[{"time": 1406006074000, + "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", + "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 + } + ] +} +``` + +## POST Operations + +### /api/v1/topology/:id/activate (POST) + +Activates a topology. + +|Parameter |Value |Description | +|----------|--------|-------------| +|id |String (required)| Topology Id | + +Sample Response: + +```json +{"topologyOperation":"activate","topologyId":"wordcount-1-1420308665","status":"success"} +``` + + +### /api/v1/topology/:id/deactivate (POST) + +Deactivates a topology. + +|Parameter |Value |Description | +|----------|--------|-------------| +|id |String (required)| Topology Id | + +Sample Response: + +```json +{"topologyOperation":"deactivate","topologyId":"wordcount-1-1420308665","status":"success"} +``` + + +### /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 | +|rebalanceOptions| Json (optional) | topology rebalance options | + + +Sample rebalanceOptions json: + +```json +{"rebalanceOptions" : {"numWorkers" : 2, "executors" : {"spout" :4, "count" : 10}}, "callback" : "foo"} +``` + +Examples: + +```no-highlight +curl -i -b ~/cookiejar.txt -c ~/cookiejar.txt -X POST +-H "Content-Type: application/json" +-d '{"rebalanceOptions": {"numWorkers": 2, "executors": { "spout" : "5", "split": 7, "count": 5 }}, "callback":"foo"}' +http://localhost:8080/api/v1/topology/wordcount-1-1420308665/rebalance/0 +``` + +Sample Response: + +```json +{"topologyOperation":"rebalance","topologyId":"wordcount-1-1420308665","status":"success"} +``` + + + +### /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 | + +Caution: Small wait times (0-5 seconds) may increase the probability of triggering the bug reported in +[STORM-112](https://issues.apache.org/jira/browse/STORM-112), which may result in broker Supervisor +daemons. + +Sample Response: + +```json +{"topologyOperation":"kill","topologyId":"wordcount-1-1420308665","status":"success"} +``` + +## API errors + +The API returns 500 HTTP status codes in case of any errors. + +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" +} +``` diff --git a/docs/Serialization-(prior-to-0.6.0).md b/docs/Serialization-(prior-to-0.6.0).md new file mode 100644 index 00000000000..e4a0d4fd0d1 --- /dev/null +++ b/docs/Serialization-(prior-to-0.6.0).md @@ -0,0 +1,50 @@ +--- +layout: documentation +--- +Tuples can be comprised of objects of any types. Since Storm is a distributed system, it needs to know how to serialize and deserialize objects when they're passed between tasks. By default Storm can serialize ints, shorts, longs, floats, doubles, bools, bytes, strings, and byte arrays, but if you want to use another type in your tuples, you'll need to implement a custom serializer. + +### Dynamic typing + +There are no type declarations for fields in a Tuple. You put objects in fields and Storm figures out the serialization dynamically. Before we get to the interface for serialization, let's spend a moment understanding why Storm's tuples are dynamically typed. + +Adding static typing to tuple fields would add large amount of complexity to Storm's API. Hadoop, for example, statically types its keys and values but requires a huge amount of annotations on the part of the user. Hadoop's API is a burden to use and the "type safety" isn't worth it. Dynamic typing is simply easier to use. + +Further than that, it's not possible to statically type Storm's tuples in any reasonable way. Suppose a Bolt subscribes to multiple streams. The tuples from all those streams may have different types across the fields. When a Bolt receives a `Tuple` in `execute`, that tuple could have come from any stream and so could have any combination of types. There might be some reflection magic you can do to declare a different method for every tuple stream a bolt subscribes to, but Storm opts for the simpler, straightforward approach of dynamic typing. + +Finally, another reason for using dynamic typing is so Storm can be used in a straightforward manner from dynamically typed languages like Clojure and JRuby. + +### Custom serialization + +Let's dive into Storm's API for defining custom serializations. There are two steps you need to take as a user to create a custom serialization: implement the serializer, and register the serializer to Storm. + +#### Creating a serializer + +Custom serializers implement the [ISerialization](javadocs/backtype/storm/serialization/ISerialization.html) interface. Implementations specify how to serialize and deserialize types into a binary format. + +The interface looks like this: + +```java +public interface ISerialization { + public boolean accept(Class c); + public void serialize(T object, DataOutputStream stream) throws IOException; + public T deserialize(DataInputStream stream) throws IOException; +} +``` + +Storm uses the `accept` method to determine if a type can be serialized by this serializer. Remember, Storm's tuples are dynamically typed so Storm determines what serializer to use at runtime. + +`serialize` writes the object out to the output stream in binary format. The field must be written in a way such that it can be deserialized later. For example, if you're writing out a list of objects, you'll need to write out the size of the list first so that you know how many elements to deserialize. + +`deserialize` reads the serialized object off of the stream and returns it. + +You can see example serialization implementations in the source for [SerializationFactory](https://github.com/apache/incubator-storm/blob/0.5.4/src/jvm/backtype/storm/serialization/SerializationFactory.java) + +#### Registering a serializer + +Once you create a serializer, you need to tell Storm it exists. This is done through the Storm configuration (See [Concepts](Concepts.html) for information about how configuration works in Storm). You can register serializations either through the config given when submitting a topology or in the storm.yaml files across your cluster. + +Serializer registrations are done through the Config.TOPOLOGY_SERIALIZATIONS config and is simply a list of serialization class names. + +Storm provides helpers for registering serializers in a topology config. The [Config](javadocs/backtype/storm/Config.html) class has a method called `addSerialization` that takes in a serializer class to add to the config. + +There's an advanced config called Config.TOPOLOGY_SKIP_MISSING_SERIALIZATIONS. If you set this to true, Storm will ignore any serializations that are registered but do not have their code available on the classpath. Otherwise, Storm will throw errors when it can't find a serialization. This is useful if you run many topologies on a cluster that each have different serializations, but you want to declare all the serializations across all topologies in the `storm.yaml` files. diff --git a/docs/Serialization.md b/docs/Serialization.md new file mode 100644 index 00000000000..4c271b4178f --- /dev/null +++ b/docs/Serialization.md @@ -0,0 +1,60 @@ +--- +layout: documentation +--- +This page is about how the serialization system in Storm works for versions 0.6.0 and onwards. Storm used a different serialization system prior to 0.6.0 which is documented on [Serialization (prior to 0.6.0)](Serialization-\(prior-to-0.6.0\).html). + +Tuples can be comprised of objects of any types. Since Storm is a distributed system, it needs to know how to serialize and deserialize objects when they're passed between tasks. + +Storm uses [Kryo](http://code.google.com/p/kryo/) for serialization. Kryo is a flexible and fast serialization library that produces small serializations. + +By default, Storm can serialize primitive types, strings, byte arrays, ArrayList, HashMap, HashSet, and the Clojure collection types. If you want to use another type in your tuples, you'll need to register a custom serializer. + +### Dynamic typing + +There are no type declarations for fields in a Tuple. You put objects in fields and Storm figures out the serialization dynamically. Before we get to the interface for serialization, let's spend a moment understanding why Storm's tuples are dynamically typed. + +Adding static typing to tuple fields would add large amount of complexity to Storm's API. Hadoop, for example, statically types its keys and values but requires a huge amount of annotations on the part of the user. Hadoop's API is a burden to use and the "type safety" isn't worth it. Dynamic typing is simply easier to use. + +Further than that, it's not possible to statically type Storm's tuples in any reasonable way. Suppose a Bolt subscribes to multiple streams. The tuples from all those streams may have different types across the fields. When a Bolt receives a `Tuple` in `execute`, that tuple could have come from any stream and so could have any combination of types. There might be some reflection magic you can do to declare a different method for every tuple stream a bolt subscribes to, but Storm opts for the simpler, straightforward approach of dynamic typing. + +Finally, another reason for using dynamic typing is so Storm can be used in a straightforward manner from dynamically typed languages like Clojure and JRuby. + +### Custom serialization + +As mentioned, Storm uses Kryo for serialization. To implement custom serializers, you need to register new serializers with Kryo. It's highly recommended that you read over [Kryo's home page](http://code.google.com/p/kryo/) to understand how it handles custom serialization. + +Adding custom serializers is done through the "topology.kryo.register" property in your topology config. It takes a list of registrations, where each registration can take one of two forms: + +1. The name of a class to register. In this case, Storm will use Kryo's `FieldsSerializer` to serialize the class. This may or may not be optimal for the class -- see the Kryo docs for more details. +2. A map from the name of a class to register to an implementation of [com.esotericsoftware.kryo.Serializer](http://code.google.com/p/kryo/source/browse/trunk/src/com/esotericsoftware/kryo/Serializer.java). + +Let's look at an example. + +``` +topology.kryo.register: + - com.mycompany.CustomType1 + - com.mycompany.CustomType2: com.mycompany.serializer.CustomType2Serializer + - com.mycompany.CustomType3 +``` + +`com.mycompany.CustomType1` and `com.mycompany.CustomType3` will use the `FieldsSerializer`, whereas `com.mycompany.CustomType2` will use `com.mycompany.serializer.CustomType2Serializer` for serialization. + +Storm provides helpers for registering serializers in a topology config. The [Config](javadocs/backtype/storm/Config.html) class has a method called `registerSerialization` that takes in a registration to add to the config. + +There's an advanced config called `Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS`. If you set this to true, Storm will ignore any serializations that are registered but do not have their code available on the classpath. Otherwise, Storm will throw errors when it can't find a serialization. This is useful if you run many topologies on a cluster that each have different serializations, but you want to declare all the serializations across all topologies in the `storm.yaml` files. + +### Java serialization + +If Storm encounters a type for which it doesn't have a serialization registered, it will use Java serialization if possible. If the object can't be serialized with Java serialization, then Storm will throw an error. + +Beware that Java serialization is extremely expensive, both in terms of CPU cost as well as the size of the serialized object. It is highly recommended that you register custom serializers when you put the topology in production. The Java serialization behavior is there so that it's easy to prototype new topologies. + +You can turn off the behavior to fall back on Java serialization by setting the `Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION` config to false. + +### Component-specific serialization registrations + +Storm 0.7.0 lets you set component-specific configurations (read more about this at [Configuration](Configuration.html)). Of course, if one component defines a serialization that serialization will need to be available to other bolts -- otherwise they won't be able to receive messages from that component! + +When a topology is submitted, a single set of serializations is chosen to be used by all components in the topology for sending messages. This is done by merging the component-specific serializer registrations with the regular set of serialization registrations. If two components define serializers for the same class, one of the serializers is chosen arbitrarily. + +To force a serializer for a particular class if there's a conflict between two component-specific registrations, just define the serializer you want to use in the topology-specific configuration. The topology-specific configuration has precedence over component-specific configurations for serialization registrations. diff --git a/docs/Serializers.md b/docs/Serializers.md new file mode 100644 index 00000000000..071c8851177 --- /dev/null +++ b/docs/Serializers.md @@ -0,0 +1,4 @@ +--- +layout: documentation +--- +* [storm-json](https://github.com/rapportive-oss/storm-json): Simple JSON serializer for Storm diff --git a/docs/Setting-up-a-Storm-cluster.md b/docs/Setting-up-a-Storm-cluster.md new file mode 100644 index 00000000000..e139523de58 --- /dev/null +++ b/docs/Setting-up-a-Storm-cluster.md @@ -0,0 +1,83 @@ +--- +layout: documentation +--- +This page outlines the steps for getting a Storm cluster up and running. If you're on AWS, you should check out the [storm-deploy](https://github.com/nathanmarz/storm-deploy/wiki) project. [storm-deploy](https://github.com/nathanmarz/storm-deploy/wiki) completely automates the provisioning, configuration, and installation of Storm clusters on EC2. It also sets up Ganglia for you so you can monitor CPU, disk, and network usage. + +If you run into difficulties with your Storm cluster, first check for a solution is in the [Troubleshooting](Troubleshooting.html) page. Otherwise, email the mailing list. + +Here's a summary of the steps for setting up a Storm cluster: + +1. Set up a Zookeeper cluster +2. Install dependencies on Nimbus and worker machines +3. Download and extract a Storm release to Nimbus and worker machines +4. Fill in mandatory configurations into storm.yaml +5. Launch daemons under supervision using "storm" script and a supervisor of your choice + +### Set up a Zookeeper cluster + +Storm uses Zookeeper for coordinating the cluster. Zookeeper **is not** used for message passing, so the load Storm places on Zookeeper is quite low. Single node Zookeeper clusters should be sufficient for most cases, but if you want failover or are deploying large Storm clusters you may want larger Zookeeper clusters. Instructions for deploying Zookeeper are [here](http://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html). + +A few notes about Zookeeper deployment: + +1. It's critical that you run Zookeeper under supervision, since Zookeeper is fail-fast and will exit the process if it encounters any error case. See [here](http://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html#sc_supervision) for more details. +2. It's critical that you set up a cron to compact Zookeeper's data and transaction logs. The Zookeeper daemon does not do this on its own, and if you don't set up a cron, Zookeeper will quickly run out of disk space. See [here](http://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html#sc_maintenance) for more details. + +### Install dependencies on Nimbus and worker machines + +Next you need to install Storm's dependencies on Nimbus and the worker machines. These are: + +1. Java 6 +2. Python 2.6.6 + +These are the versions of the dependencies that have been tested with Storm. Storm may or may not work with different versions of Java and/or Python. + + +### Download and extract a Storm release to Nimbus and worker machines + +Next, download a Storm release and extract the zip file somewhere on Nimbus and each of the worker machines. The Storm releases can be downloaded [from here](http://github.com/apache/incubator-storm/downloads). + +### Fill in mandatory configurations into storm.yaml + +The Storm release contains a file at `conf/storm.yaml` that configures the Storm daemons. You can see the default configuration values [here](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml). storm.yaml overrides anything in defaults.yaml. There's a few configurations that are mandatory to get a working cluster: + +1) **storm.zookeeper.servers**: This is a list of the hosts in the Zookeeper cluster for your Storm cluster. It should look something like: + +```yaml +storm.zookeeper.servers: + - "111.222.333.444" + - "555.666.777.888" +``` + +If the port that your Zookeeper cluster uses is different than the default, you should set **storm.zookeeper.port** as well. + +2) **storm.local.dir**: The Nimbus and Supervisor daemons require a directory on the local disk to store small amounts of state (like jars, confs, and things like that). You should create that directory on each machine, give it proper permissions, and then fill in the directory location using this config. For example: + +```yaml +storm.local.dir: "/mnt/storm" +``` + +3) **nimbus.host**: The worker nodes need to know which machine is the master in order to download topology jars and confs. For example: + +```yaml +nimbus.host: "111.222.333.44" +``` + +4) **supervisor.slots.ports**: For each worker machine, you configure how many workers run on that machine with this config. Each worker uses a single port for receiving messages, and this setting defines which ports are open for use. If you define five ports here, then Storm will allocate up to five workers to run on this machine. If you define three ports, Storm will only run up to three. By default, this setting is configured to run 4 workers on the ports 6700, 6701, 6702, and 6703. For example: + +```yaml +supervisor.slots.ports: + - 6700 + - 6701 + - 6702 + - 6703 +``` + +### Launch daemons under supervision using "storm" script and a supervisor of your choice + +The last step is to launch all the Storm daemons. It is critical that you run each of these daemons under supervision. Storm is a __fail-fast__ system which means the processes will halt whenever an unexpected error is encountered. Storm is designed so that it can safely halt at any point and recover correctly when the process is restarted. This is why Storm keeps no state in-process -- if Nimbus or the Supervisors restart, the running topologies are unaffected. Here's how to run the Storm daemons: + +1. **Nimbus**: Run the command "bin/storm nimbus" under supervision on the master machine. +2. **Supervisor**: Run the command "bin/storm supervisor" under supervision on each worker machine. The supervisor daemon is responsible for starting and stopping worker processes on that machine. +3. **UI**: Run the Storm UI (a site you can access from the browser that gives diagnostics on the cluster and topologies) by running the command "bin/storm ui" under supervision. The UI can be accessed by navigating your web browser to http://{nimbus host}:8080. + +As you can see, running the daemons is very straightforward. The daemons will log to the logs/ directory in wherever you extracted the Storm release. diff --git a/docs/Setting-up-a-Storm-project-in-Eclipse.md b/docs/Setting-up-a-Storm-project-in-Eclipse.md new file mode 100644 index 00000000000..5137cd9e32a --- /dev/null +++ b/docs/Setting-up-a-Storm-project-in-Eclipse.md @@ -0,0 +1 @@ +- fill me in \ No newline at end of file diff --git a/docs/Setting-up-development-environment.md b/docs/Setting-up-development-environment.md new file mode 100644 index 00000000000..07ba670bbec --- /dev/null +++ b/docs/Setting-up-development-environment.md @@ -0,0 +1,39 @@ +--- +layout: documentation +--- +This page outlines what you need to do to get a Storm development environment set up. In summary, the steps are: + +1. Download a [Storm release](/releases.html) , unpack it, and put the unpacked `bin/` directory on your PATH +2. To be able to start and stop topologies on a remote cluster, put the cluster information in `~/.storm/storm.yaml` + +More detail on each of these steps is below. + +### What is a development environment? + +Storm has two modes of operation: local mode and remote mode. In local mode, you can develop and test topologies completely in process on your local machine. In remote mode, you submit topologies for execution on a cluster of machines. + +A Storm development environment has everything installed so that you can develop and test Storm topologies in local mode, package topologies for execution on a remote cluster, and submit/kill topologies on a remote cluster. + +Let's quickly go over the relationship between your machine and a remote cluster. A Storm cluster is managed by a master node called "Nimbus". Your machine communicates with Nimbus to submit code (packaged as a jar) and topologies for execution on the cluster, and Nimbus will take care of distributing that code around the cluster and assigning workers to run your topology. Your machine uses a command line client called `storm` to communicate with Nimbus. The `storm` client is only used for remote mode; it is not used for developing and testing topologies in local mode. + +### Installing a Storm release locally + +If you want to be able to submit topologies to a remote cluster from your machine, you should install a Storm release locally. Installing a Storm release will give you the `storm` client that you can use to interact with remote clusters. To install Storm locally, download a release [from here](/releases.html) and unzip it somewhere on your computer. Then add the unpacked `bin/` directory onto your `PATH` and make sure the `bin/storm` script is executable. + +Installing a Storm release locally is only for interacting with remote clusters. For developing and testing topologies in local mode, it is recommended that you use Maven to include Storm as a dev dependency for your project. You can read more about using Maven for this purpose on [Maven](Maven.html). + +### Starting and stopping topologies on a remote cluster + +The previous step installed the `storm` client on your machine which is used to communicate with remote Storm clusters. Now all you have to do is tell the client which Storm cluster to talk to. To do this, all you have to do is put the host address of the master in the `~/.storm/storm.yaml` file. It should look something like this: + +``` +nimbus.host: "123.45.678.890" +``` + +Alternatively, if you use the [storm-deploy](https://github.com/nathanmarz/storm-deploy) project to provision Storm clusters on AWS, it will automatically set up your ~/.storm/storm.yaml file. You can manually attach to a Storm cluster (or switch between multiple clusters) using the "attach" command, like so: + +``` +lein run :deploy --attach --name mystormcluster +``` + +More information is on the storm-deploy [wiki](https://github.com/nathanmarz/storm-deploy/wiki) diff --git a/docs/Spout-implementations.md b/docs/Spout-implementations.md new file mode 100644 index 00000000000..10ddd427cb3 --- /dev/null +++ b/docs/Spout-implementations.md @@ -0,0 +1,8 @@ +--- +layout: documentation +--- +* [storm-kestrel](https://github.com/nathanmarz/storm-kestrel): Adapter to use Kestrel as a spout +* [storm-amqp-spout](https://github.com/rapportive-oss/storm-amqp-spout): Adapter to use AMQP source as a spout +* [storm-jms](https://github.com/ptgoetz/storm-jms): Adapter to use a JMS source as a spout +* [storm-redis-pubsub](https://github.com/sorenmacbeth/storm-redis-pubsub): A spout that subscribes to a Redis pubsub stream +* [storm-beanstalkd-spout](https://github.com/haitaoyao/storm-beanstalkd-spout): A spout that subscribes to a beanstalkd queue diff --git a/docs/Storm-multi-language-protocol-(versions-0.7.0-and-below).md b/docs/Storm-multi-language-protocol-(versions-0.7.0-and-below).md new file mode 100644 index 00000000000..1d4422f7b00 --- /dev/null +++ b/docs/Storm-multi-language-protocol-(versions-0.7.0-and-below).md @@ -0,0 +1,122 @@ +--- +layout: documentation +--- +This page explains the multilang protocol for versions 0.7.0 and below. The protocol changed in version 0.7.1. + +# Storm Multi-Language Protocol + +## The ShellBolt + +Support for multiple languages is implemented via the ShellBolt class. This +class implements the IBolt interfaces and implements the protocol for +executing a script or program via the shell using Java's ProcessBuilder class. + +## Output fields + +Output fields are part of the Thrift definition of the topology. This means that when you multilang in Java, you need to create a bolt that extends ShellBolt, implements IRichBolt, and declared the fields in `declareOutputFields`. +You can learn more about this on [Concepts](Concepts.html) + +## Protocol Preamble + +A simple protocol is implemented via the STDIN and STDOUT of the executed +script or program. A mix of simple strings and JSON encoded data are exchanged +with the process making support possible for pretty much any language. + +# Packaging Your Stuff + +To run a ShellBolt on a cluster, the scripts that are shelled out to must be +in the `resources/` directory within the jar submitted to the master. + +However, During development or testing on a local machine, the resources +directory just needs to be on the classpath. + +## The Protocol + +Notes: +* Both ends of this protocol use a line-reading mechanism, so be sure to +trim off newlines from the input and to append them to your output. +* All JSON inputs and outputs are terminated by a single line contained "end". +* The bullet points below are written from the perspective of the script writer's +STDIN and STDOUT. + + +* Your script will be executed by the Bolt. +* STDIN: A string representing a path. This is a PID directory. +Your script should create an empty file named with it's pid in this directory. e.g. +the PID is 1234, so an empty file named 1234 is created in the directory. This +file lets the supervisor know the PID so it can shutdown the process later on. +* STDOUT: Your PID. This is not JSON encoded, just a string. ShellBolt will log the PID to its log. +* STDIN: (JSON) The Storm configuration. Various settings and properties. +* STDIN: (JSON) The Topology context +* The rest happens in a while(true) loop +* STDIN: A tuple! This is a JSON encoded structure like this: + +``` +{ + // The tuple's id + "id": -6955786537413359385, + // The id of the component that created this tuple + "comp": 1, + // The id of the stream this tuple was emitted to + "stream": 1, + // The id of the task that created this tuple + "task": 9, + // All the values in this tuple + "tuple": ["snow white and the seven dwarfs", "field2", 3] +} +``` + +* STDOUT: The results of your bolt, JSON encoded. This can be a sequence of acks, fails, emits, and/or logs. Emits look like: + +``` +{ + "command": "emit", + // The ids of the tuples this output tuples should be anchored to + "anchors": [1231231, -234234234], + // The id of the stream this tuple was emitted to. Leave this empty to emit to default stream. + "stream": 1, + // If doing an emit direct, indicate the task to sent the tuple to + "task": 9, + // All the values in this tuple + "tuple": ["field1", 2, 3] +} +``` + +An ack looks like: + +``` +{ + "command": "ack", + // the id of the tuple to ack + "id": 123123 +} +``` + +A fail looks like: + +``` +{ + "command": "fail", + // the id of the tuple to fail + "id": 123123 +} +``` + +A "log" will log a message in the worker log. It looks like: + +``` +{ + "command": "log", + // the message to log + "msg": "hello world!" + +} +``` + +* STDOUT: emit "sync" as a single line by itself when the bolt has finished emitting/acking/failing and is ready for the next input + +### sync + +Note: This command is not JSON encoded, it is sent as a simple string. + +This lets the parent bolt know that the script has finished processing and is ready for another tuple. diff --git a/docs/Structure-of-the-codebase.md b/docs/Structure-of-the-codebase.md new file mode 100644 index 00000000000..8ac66f431b4 --- /dev/null +++ b/docs/Structure-of-the-codebase.md @@ -0,0 +1,140 @@ +--- +layout: documentation +--- +There are three distinct layers to Storm's codebase. + +First, Storm was designed from the very beginning to be compatible with multiple languages. Nimbus is a Thrift service and topologies are defined as Thrift structures. The usage of Thrift allows Storm to be used from any language. + +Second, all of Storm's interfaces are specified as Java interfaces. So even though there's a lot of Clojure in Storm's implementation, all usage must go through the Java API. This means that every feature of Storm is always available via Java. + +Third, Storm's implementation is largely in Clojure. Line-wise, Storm is about half Java code, half Clojure code. But Clojure is much more expressive, so in reality the great majority of the implementation logic is in Clojure. + +The following sections explain each of these layers in more detail. + +### storm.thrift + +The first place to look to understand the structure of Storm's codebase is the [storm.thrift](https://github.com/apache/incubator-storm/blob/master/storm-core/src/storm.thrift) file. + +Storm uses [this fork](https://github.com/nathanmarz/thrift/tree/storm) of Thrift (branch 'storm') to produce the generated code. This "fork" is actually Thrift 7 with all the Java packages renamed to be `org.apache.thrift7`. Otherwise, it's identical to Thrift 7. This fork was done because of the lack of backwards compatibility in Thrift and the need for many people to use other versions of Thrift in their Storm topologies. + +Every spout or bolt in a topology is given a user-specified identifier called the "component id". The component id is used to specify subscriptions from a bolt to the output streams of other spouts or bolts. A [StormTopology](https://github.com/apache/incubator-storm/blob/master/storm-core/src/storm.thrift#L91) structure contains a map from component id to component for each type of component (spouts and bolts). + +Spouts and bolts have the same Thrift definition, so let's just take a look at the [Thrift definition for bolts](https://github.com/apache/incubator-storm/blob/master/storm-core/src/storm.thrift#L79). It contains a `ComponentObject` struct and a `ComponentCommon` struct. + +The `ComponentObject` defines the implementation for the bolt. It can be one of three types: + +1. A serialized java object (that implements [IBolt](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/task/IBolt.java)) +2. A `ShellComponent` object that indicates the implementation is in another language. Specifying a bolt this way will cause Storm to instantiate a [ShellBolt](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/task/ShellBolt.java) object to handle the communication between the JVM-based worker process and the non-JVM-based implementation of the component. +3. A `JavaObject` structure which tells Storm the classname and constructor arguments to use to instantiate that bolt. This is useful if you want to define a topology in a non-JVM language. This way, you can make use of JVM-based spouts and bolts without having to create and serialize a Java object yourself. + +`ComponentCommon` defines everything else for this component. This includes: + +1. What streams this component emits and the metadata for each stream (whether it's a direct stream, the fields declaration) +2. What streams this component consumes (specified as a map from component_id:stream_id to the stream grouping to use) +3. The parallelism for this component +4. The component-specific [configuration](https://github.com/apache/incubator-storm/wiki/Configuration) for this component + +Note that the structure spouts also have a `ComponentCommon` field, and so spouts can also have declarations to consume other input streams. Yet the Storm Java API does not provide a way for spouts to consume other streams, and if you put any input declarations there for a spout you would get an error when you tried to submit the topology. The reason that spouts have an input declarations field is not for users to use, but for Storm itself to use. Storm adds implicit streams and bolts to the topology to set up the [acking framework](https://github.com/apache/incubator-storm/wiki/Guaranteeing-message-processing), and two of these implicit streams are from the acker bolt to each spout in the topology. The acker sends "ack" or "fail" messages along these streams whenever a tuple tree is detected to be completed or failed. The code that transforms the user's topology into the runtime topology is located [here](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/common.clj#L279). + +### Java interfaces + +The interfaces for Storm are generally specified as Java interfaces. The main interfaces are: + +1. [IRichBolt](javadocs/backtype/storm/topology/IRichBolt.html) +2. [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html) +3. [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html) + +The strategy for the majority of the interfaces is to: + +1. Specify the interface using a Java interface +2. Provide a base class that provides default implementations when appropriate + +You can see this strategy at work with the [BaseRichSpout](javadocs/backtype/storm/topology/base/BaseRichSpout.html) class. + +Spouts and bolts are serialized into the Thrift definition of the topology as described above. + +One subtle aspect of the interfaces is the difference between `IBolt` and `ISpout` vs. `IRichBolt` and `IRichSpout`. The main difference between them is the addition of the `declareOutputFields` method in the "Rich" versions of the interfaces. The reason for the split is that the output fields declaration for each output stream needs to be part of the Thrift struct (so it can be specified from any language), but as a user you want to be able to declare the streams as part of your class. What `TopologyBuilder` does when constructing the Thrift representation is call `declareOutputFields` to get the declaration and convert it into the Thrift structure. The conversion happens [at this portion](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java#L205) of the `TopologyBuilder` code. + + +### Implementation + +Specifying all the functionality via Java interfaces ensures that every feature of Storm is available via Java. Moreso, the focus on Java interfaces ensures that the user experience from Java-land is pleasant as well. + +The implementation of Storm, on the other hand, is primarily in Clojure. While the codebase is about 50% Java and 50% Clojure in terms of LOC, most of the implementation logic is in Clojure. There are two notable exceptions to this, and that is the [DRPC](https://github.com/apache/incubator-storm/wiki/Distributed-RPC) and [transactional topologies](https://github.com/apache/incubator-storm/wiki/Transactional-topologies) implementations. These are implemented purely in Java. This was done to serve as an illustration for how to implement a higher level abstraction on Storm. The DRPC and transactional topologies implementations are in the [backtype.storm.coordination](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/coordination), [backtype.storm.drpc](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/drpc), and [backtype.storm.transactional](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/transactional) packages. + +Here's a summary of the purpose of the main Java packages and Clojure namespace: + +#### Java packages + +[backtype.storm.coordination](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/coordination): Implements the pieces required to coordinate batch-processing on top of Storm, which both DRPC and transactional topologies use. `CoordinatedBolt` is the most important class here. + +[backtype.storm.drpc](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/drpc): Implementation of the DRPC higher level abstraction + +[backtype.storm.generated](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/generated): The generated Thrift code for Storm (generated using [this fork](https://github.com/nathanmarz/thrift) of Thrift, which simply renames the packages to org.apache.thrift7 to avoid conflicts with other Thrift versions) + +[backtype.storm.grouping](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/grouping): Contains interface for making custom stream groupings + +[backtype.storm.hooks](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/hooks): Interfaces for hooking into various events in Storm, such as when tasks emit tuples, when tuples are acked, etc. User guide for hooks is [here](https://github.com/apache/incubator-storm/wiki/Hooks). + +[backtype.storm.serialization](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/serialization): Implementation of how Storm serializes/deserializes tuples. Built on top of [Kryo](http://code.google.com/p/kryo/). + +[backtype.storm.spout](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/spout): Definition of spout and associated interfaces (like the `SpoutOutputCollector`). Also contains `ShellSpout` which implements the protocol for defining spouts in non-JVM languages. + +[backtype.storm.task](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/task): Definition of bolt and associated interfaces (like `OutputCollector`). Also contains `ShellBolt` which implements the protocol for defining bolts in non-JVM languages. Finally, `TopologyContext` is defined here as well, which is provided to spouts and bolts so they can get data about the topology and its execution at runtime. + +[backtype.storm.testing](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/testing): Contains a variety of test bolts and utilities used in Storm's unit tests. + +[backtype.storm.topology](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/topology): Java layer over the underlying Thrift structure to provide a clean, pure-Java API to Storm (users don't have to know about Thrift). `TopologyBuilder` is here as well as the helpful base classes for the different spouts and bolts. The slightly-higher level `IBasicBolt` interface is here, which is a simpler way to write certain kinds of bolts. + +[backtype.storm.transactional](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/transactional): Implementation of transactional topologies. + +[backtype.storm.tuple](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/tuple): Implementation of Storm's tuple data model. + +[backtype.storm.utils](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/tuple): Data structures and miscellaneous utilities used throughout the codebase. + + +#### Clojure namespaces + +[backtype.storm.bootstrap](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/bootstrap.clj): Contains a helpful macro to import all the classes and namespaces that are used throughout the codebase. + +[backtype.storm.clojure](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/clojure.clj): Implementation of the Clojure DSL for Storm. + +[backtype.storm.cluster](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/cluster.clj): All Zookeeper logic used in Storm daemons is encapsulated in this file. This code manages how cluster state (like what tasks are running where, what spout/bolt each task runs as) is mapped to the Zookeeper "filesystem" API. + +[backtype.storm.command.*](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/command): These namespaces implement various commands for the `storm` command line client. These implementations are very short. + +[backtype.storm.config](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/config.clj): Implementation of config reading/parsing code for Clojure. Also has utility functions for determining what local path nimbus/supervisor/daemons should be using for various things. e.g. the `master-inbox` function will return the local path that Nimbus should use when jars are uploaded to it. + +[backtype.storm.daemon.acker](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/acker.clj): Implementation of the "acker" bolt, which is a key part of how Storm guarantees data processing. + +[backtype.storm.daemon.common](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/common.clj): Implementation of common functions used in Storm daemons, like getting the id for a topology based on the name, mapping a user's topology into the one that actually executes (with implicit acking streams and acker bolt added - see `system-topology!` function), and definitions for the various heartbeat and other structures persisted by Storm. + +[backtype.storm.daemon.drpc](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/drpc.clj): Implementation of the DRPC server for use with DRPC topologies. + +[backtype.storm.daemon.nimbus](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/nimbus.clj): Implementation of Nimbus. + +[backtype.storm.daemon.supervisor](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/supervisor.clj): Implementation of Supervisor. + +[backtype.storm.daemon.task](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/task.clj): Implementation of an individual task for a spout or bolt. Handles message routing, serialization, stats collection for the UI, as well as the spout-specific and bolt-specific execution implementations. + +[backtype.storm.daemon.worker](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/worker.clj): Implementation of a worker process (which will contain many tasks within). Implements message transferring and task launching. + +[backtype.storm.event](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/event.clj): Implements a simple asynchronous function executor. Used in various places in Nimbus and Supervisor to make functions execute in serial to avoid any race conditions. + +[backtype.storm.log](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/log.clj): Defines the functions used to log messages to log4j. + +[backtype.storm.messaging.*](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/messaging): Defines a higher level interface to implementing point to point messaging. In local mode Storm uses in-memory Java queues to do this; on a cluster, it uses ZeroMQ. The generic interface is defined in protocol.clj. + +[backtype.storm.stats](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/stats.clj): Implementation of stats rollup routines used when sending stats to ZK for use by the UI. Does things like windowed and rolling aggregations at multiple granularities. + +[backtype.storm.testing](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/testing.clj): Implementation of facilities used to test Storm topologies. Includes time simulation, `complete-topology` for running a fixed set of tuples through a topology and capturing the output, tracker topologies for having fine grained control over detecting when a cluster is "idle", and other utilities. + +[backtype.storm.thrift](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/thrift.clj): Clojure wrappers around the generated Thrift API to make working with Thrift structures more pleasant. + +[backtype.storm.timer](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/timer.clj): Implementation of a background timer to execute functions in the future or on a recurring interval. Storm couldn't use the [Timer](http://docs.oracle.com/javase/1.4.2/docs/api/java/util/Timer.html) class because it needed integration with time simulation in order to be able to unit test Nimbus and the Supervisor. + +[backtype.storm.ui.*](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/ui): Implementation of Storm UI. Completely independent from rest of code base and uses the Nimbus Thrift API to get data. + +[backtype.storm.util](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/util.clj): Contains generic utility functions used throughout the code base. + +[backtype.storm.zookeeper](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/zookeeper.clj): Clojure wrapper around the Zookeeper API and implements some "high-level" stuff like "mkdirs" and "delete-recursive". diff --git a/docs/Support-for-non-java-languages.md b/docs/Support-for-non-java-languages.md new file mode 100644 index 00000000000..724d106c2e0 --- /dev/null +++ b/docs/Support-for-non-java-languages.md @@ -0,0 +1,7 @@ +--- +layout: documentation +--- +* [Scala DSL](https://github.com/velvia/ScalaStorm) +* [JRuby DSL](https://github.com/colinsurprenant/storm-jruby) +* [Clojure DSL](Clojure-DSL.html) +* [io-storm](https://github.com/gphat/io-storm): Perl multilang adapter diff --git a/docs/Transactional-topologies.md b/docs/Transactional-topologies.md new file mode 100644 index 00000000000..1271a21ee25 --- /dev/null +++ b/docs/Transactional-topologies.md @@ -0,0 +1,359 @@ +--- +layout: documentation +--- +**NOTE**: Transactional topologies have been deprecated -- use the [Trident](Trident-tutorial.html) framework instead. + +__________________________________________________________________________ + +Storm [guarantees data processing](Guaranteeing-message-processing.html) by providing an at least once processing guarantee. The most common question asked about Storm is "Given that tuples can be replayed, how do you do things like counting on top of Storm? Won't you overcount?" + +Storm 0.7.0 introduces transactional topologies, which enable you to get exactly once messaging semantics for pretty much any computation. So you can do things like counting in a fully-accurate, scalable, and fault-tolerant way. + +Like [Distributed RPC](Distributed-RPC.html), transactional topologies aren't so much a feature of Storm as they are a higher level abstraction built on top of Storm's primitives of streams, spouts, bolts, and topologies. + +This page explains the transactional topology abstraction, how to use the API, and provides details as to its implementation. + +## Concepts + +Let's build up to Storm's abstraction for transactional topologies one step at a time. Let's start by looking at the simplest possible approach, and then we'll iterate on the design until we reach Storm's design. + +### Design 1 + +The core idea behind transactional topologies is to provide a _strong ordering_ on the processing of data. The simplest manifestation of this, and the first design we'll look at, is processing the tuples one at a time and not moving on to the next tuple until the current tuple has been successfully processed by the topology. + +Each tuple is associated with a transaction id. If the tuple fails and needs to be replayed, then it is emitted with the exact same transaction id. A transaction id is an integer that increments for every tuple, so the first tuple will have transaction id `1`, the second id `2`, and so on. + +The strong ordering of tuples gives you the capability to achieve exactly-once semantics even in the case of tuple replay. Let's look at an example of how you would do this. + +Suppose you want to do a global count of the tuples in the stream. Instead of storing just the count in the database, you instead store the count and the latest transaction id together as one value in the database. When your code updates the count in the db, it should update the count *only if the transaction id in the database differs from the transaction id for the tuple currently being processed*. Consider the two cases: + +1. *The transaction id in the database is different than the current transaction id:* Because of the strong ordering of transactions, we know for sure that the current tuple isn't represented in that count. So we can safely increment the count and update the transaction id. +2. *The transaction id is the same as the current transaction id:* Then we know that this tuple is already incorporated into the count and can skip the update. The tuple must have failed after updating the database but before reporting success back to Storm. + +This logic and the strong ordering of transactions ensures that the count in the database will be accurate even if tuples are replayed. Credit for this trick of storing a transaction id in the database along with the value goes to the Kafka devs, particularly [this design document](http://incubator.apache.org/kafka/07/design.html). + +Furthermore, notice that the topology can safely update many sources of state in the same transaction and achieve exactly-once semantics. If there's a failure, any updates that already succeeded will skip on the retry, and any updates that failed will properly retry. For example, if you were processing a stream of tweeted urls, you could update a database that stores a tweet count for each url as well as a database that stores a tweet count for each domain. + +There is a significant problem though with this design of processing one tuple at time. Having to wait for each tuple to be _completely processed_ before moving on to the next one is horribly inefficient. It entails a huge amount of database calls (at least one per tuple), and this design makes very little use of the parallelization capabilities of Storm. So it isn't very scalable. + +### Design 2 + +Instead of processing one tuple at a time, a better approach is to process a batch of tuples for each transaction. So if you're doing a global count, you would increment the count by the number of tuples in the entire batch. If a batch fails, you replay the exact batch that failed. Instead of assigning a transaction id to each tuple, you assign a transaction id to each batch, and the processing of the batches is strongly ordered. Here's a diagram of this design: + +![Storm cluster](images/transactional-batches.png) + +So if you're processing 1000 tuples per batch, your application will do 1000x less database operations than design 1. Additionally, it takes advantage of Storm's parallelization capabilities as the computation for each batch can be parallelized. + +While this design is significantly better than design 1, it's still not as resource-efficient as possible. The workers in the topology spend a lot of time being idle waiting for the other portions of the computation to finish. For example, in a topology like this: + +![Storm cluster](images/transactional-design-2.png) + +After bolt 1 finishes its portion of the processing, it will be idle until the rest of the bolts finish and the next batch can be emitted from the spout. + +### Design 3 (Storm's design) + +A key realization is that not all the work for processing batches of tuples needs to be strongly ordered. For example, when computing a global count, there's two parts to the computation: + +1. Computing the partial count for the batch +2. Updating the global count in the database with the partial count + +The computation of #2 needs to be strongly ordered across the batches, but there's no reason you shouldn't be able to _pipeline_ the computation of the batches by computing #1 for many batches in parallel. So while batch 1 is working on updating the database, batches 2 through 10 can compute their partial counts. + +Storm accomplishes this distinction by breaking the computation of a batch into two phases: + +1. The processing phase: this is the phase that can be done in parallel for many batches +2. The commit phase: The commit phases for batches are strongly ordered. So the commit for batch 2 is not done until the commit for batch 1 has been successful. + +The two phases together are called a "transaction". Many batches can be in the processing phase at a given moment, but only one batch can be in the commit phase. If there's any failure in the processing or commit phase for a batch, the entire transaction is replayed (both phases). + +## Design details + +When using transactional topologies, Storm does the following for you: + +1. *Manages state:* Storm stores in Zookeeper all the state necessary to do transactional topologies. This includes the current transaction id as well as the metadata defining the parameters for each batch. +2. *Coordinates the transactions:* Storm will manage everything necessary to determine which transactions should be processing or committing at any point. +3. *Fault detection:* Storm leverages the acking framework to efficiently determine when a batch has successfully processed, successfully committed, or failed. Storm will then replay batches appropriately. You don't have to do any acking or anchoring -- Storm manages all of this for you. +4. *First class batch processing API*: Storm layers an API on top of regular bolts to allow for batch processing of tuples. Storm manages all the coordination for determining when a task has received all the tuples for that particular transaction. Storm will also take care of cleaning up any accumulated state for each transaction (like the partial counts). + +Finally, another thing to note is that transactional topologies require a source queue that can replay an exact batch of messages. Technologies like [Kestrel](https://github.com/robey/kestrel) can't do this. [Apache Kafka](http://incubator.apache.org/kafka/index.html) is a perfect fit for this kind of spout, and [storm-kafka](https://github.com/nathanmarz/storm-contrib/tree/master/storm-kafka) in [storm-contrib](https://github.com/nathanmarz/storm-contrib) contains a transactional spout implementation for Kafka. + +## The basics through example + +You build transactional topologies by using [TransactionalTopologyBuilder](javadocs/backtype/storm/transactional/TransactionalTopologyBuilder.html). Here's the transactional topology definition for a topology that computes the global count of tuples from the input stream. This code comes from [TransactionalGlobalCount](https://github.com/nathanmarz/storm-starter/blob/master/src/jvm/storm/starter/TransactionalGlobalCount.java) in storm-starter. + +```java +MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, new Fields("word"), PARTITION_TAKE_PER_BATCH); +TransactionalTopologyBuilder builder = new TransactionalTopologyBuilder("global-count", "spout", spout, 3); +builder.setBolt("partial-count", new BatchCount(), 5) + .shuffleGrouping("spout"); +builder.setBolt("sum", new UpdateGlobalCount()) + .globalGrouping("partial-count"); +``` + +`TransactionalTopologyBuilder` takes as input in the constructor an id for the transactional topology, an id for the spout within the topology, a transactional spout, and optionally the parallelism for the transactional spout. The id for the transactional topology is used to store state about the progress of topology in Zookeeper, so that if you restart the topology it will continue where it left off. + +A transactional topology has a single `TransactionalSpout` that is defined in the constructor of `TransactionalTopologyBuilder`. In this example, `MemoryTransactionalSpout` is used which reads in data from an in-memory partitioned source of data (the `DATA` variable). The second argument defines the fields for the data, and the third argument specifies the maximum number of tuples to emit from each partition per batch of tuples. The interface for defining your own transactional spouts is discussed later on in this tutorial. + +Now on to the bolts. This topology parallelizes the computation of the global count. The first bolt, `BatchCount`, randomly partitions the input stream using a shuffle grouping and emits the count for each partition. The second bolt, `UpdateGlobalCount`, does a global grouping and sums together the partial counts to get the count for the batch. It then updates the global count in the database if necessary. + +Here's the definition of `BatchCount`: + +```java +public static class BatchCount extends BaseBatchBolt { + Object _id; + BatchOutputCollector _collector; + + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + _count++; + } + + @Override + public void finishBatch() { + _collector.emit(new Values(_id, _count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "count")); + } +} +``` + +A new instance of this object is created for every batch that's being processed. The actual bolt this runs within is called [BatchBoltExecutor](https://github.com/apache/incubator-storm/blob/0.7.0/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java) and manages the creation and cleanup for these objects. + +The `prepare` method parameterizes this batch bolt with the Storm config, the topology context, an output collector, and the id for this batch of tuples. In the case of transactional topologies, the id will be a [TransactionAttempt](javadocs/backtype/storm/transactional/TransactionAttempt.html) object. The batch bolt abstraction can be used in Distributed RPC as well which uses a different type of id for the batches. `BatchBolt` can actually be parameterized with the type of the id, so if you only intend to use the batch bolt for transactional topologies, you can extend `BaseTransactionalBolt` which has this definition: + +```java +public abstract class BaseTransactionalBolt extends BaseBatchBolt { +} +``` + +All tuples emitted within a transactional topology must have the `TransactionAttempt` as the first field of the tuple. This lets Storm identify which tuples belong to which batches. So when you emit tuples you need to make sure to meet this requirement. + +The `TransactionAttempt` contains two values: the "transaction id" and the "attempt id". The "transaction id" is the unique id chosen for this batch and is the same no matter how many times the batch is replayed. The "attempt id" is a unique id for this particular batch of tuples and lets Storm distinguish tuples from different emissions of the same batch. Without the attempt id, Storm could confuse a replay of a batch with tuples from a prior time that batch was emitted. This would be disastrous. + +The transaction id increases by 1 for every batch emitted. So the first batch has id "1", the second has id "2", and so on. + +The `execute` method is called for every tuple in the batch. You should accumulate state for the batch in a local instance variable every time this method is called. The `BatchCount` bolt increments a local counter variable for every tuple. + +Finally, `finishBatch` is called when the task has received all tuples intended for it for this particular batch. `BatchCount` emits the partial count to the output stream when this method is called. + +Here's the definition of `UpdateGlobalCount`: + +```java +public static class UpdateGlobalCount extends BaseTransactionalBolt implements ICommitter { + TransactionAttempt _attempt; + BatchOutputCollector _collector; + + int _sum = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt attempt) { + _collector = collector; + _attempt = attempt; + } + + @Override + public void execute(Tuple tuple) { + _sum+=tuple.getInteger(1); + } + + @Override + public void finishBatch() { + Value val = DATABASE.get(GLOBAL_COUNT_KEY); + Value newval; + if(val == null || !val.txid.equals(_attempt.getTransactionId())) { + newval = new Value(); + newval.txid = _attempt.getTransactionId(); + if(val==null) { + newval.count = _sum; + } else { + newval.count = _sum + val.count; + } + DATABASE.put(GLOBAL_COUNT_KEY, newval); + } else { + newval = val; + } + _collector.emit(new Values(_attempt, newval.count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "sum")); + } +} +``` + +`UpdateGlobalCount` is specific to transactional topologies so it extends `BaseTransactionalBolt`. In the `execute` method, `UpdateGlobalCount` accumulates the count for this batch by summing together the partial batches. The interesting stuff happens in `finishBatch`. + +First, notice that this bolt implements the `ICommitter` interface. This tells Storm that the `finishBatch` method of this bolt should be part of the commit phase of the transaction. So calls to `finishBatch` for this bolt will be strongly ordered by transaction id (calls to `execute` on the other hand can happen during either the processing or commit phases). An alternative way to mark a bolt as a committer is to use the `setCommitterBolt` method in `TransactionalTopologyBuilder` instead of `setBolt`. + +The code for `finishBatch` in `UpdateGlobalCount` gets the current value from the database and compares its transaction id to the transaction id for this batch. If they are the same, it does nothing. Otherwise, it increments the value in the database by the partial count for this batch. + +A more involved transactional topology example that updates multiple databases idempotently can be found in storm-starter in the [TransactionalWords](https://github.com/nathanmarz/storm-starter/blob/master/src/jvm/storm/starter/TransactionalWords.java) class. + +## Transactional Topology API + +This section outlines the different pieces of the transactional topology API. + +### Bolts + +There are three kinds of bolts possible in a transactional topology: + +1. [BasicBolt](javadocs/backtype/storm/topology/base/BaseBasicBolt.html): This bolt doesn't deal with batches of tuples and just emits tuples based on a single tuple of input. +2. [BatchBolt](javadocs/backtype/storm/topology/base/BaseBatchBolt.html): This bolt processes batches of tuples. `execute` is called for each tuple, and `finishBatch` is called when the batch is complete. +3. BatchBolt's that are marked as committers: The only difference between this bolt and a regular batch bolt is when `finishBatch` is called. A committer bolt has `finishedBatch` called during the commit phase. The commit phase is guaranteed to occur only after all prior batches have successfully committed, and it will be retried until all bolts in the topology succeed the commit for the batch. There are two ways to make a `BatchBolt` a committer, by having the `BatchBolt` implement the [ICommitter](javadocs/backtype/storm/transactional/ICommitter.html) marker interface, or by using the `setCommiterBolt` method in `TransactionalTopologyBuilder`. + +#### Processing phase vs. commit phase in bolts + +To nail down the difference between the processing phase and commit phase of a transaction, let's look at an example topology: + +![Storm cluster](images/transactional-commit-flow.png) + +In this topology, only the bolts with a red outline are committers. + +During the processing phase, bolt A will process the complete batch from the spout, call `finishBatch` and send its tuples to bolts B and C. Bolt B is a committer so it will process all the tuples but finishBatch won't be called. Bolt C also will not have `finishBatch` called because it doesn't know if it has received all the tuples from Bolt B yet (because Bolt B is waiting for the transaction to commit). Finally, Bolt D will receive any tuples Bolt C emitted during invocations of its `execute` method. + +When the batch commits, `finishBatch` is called on Bolt B. Once it finishes, Bolt C can now detect that it has received all the tuples and will call `finishBatch`. Finally, Bolt D will receive its complete batch and call `finishBatch`. + +Notice that even though Bolt D is a committer, it doesn't have to wait for a second commit message when it receives the whole batch. Since it receives the whole batch during the commit phase, it goes ahead and completes the transaction. + +Committer bolts act just like batch bolts during the commit phase. The only difference between committer bolts and batch bolts is that committer bolts will not call `finishBatch` during the processing phase of a transaction. + +#### Acking + +Notice that you don't have to do any acking or anchoring when working with transactional topologies. Storm manages all of that underneath the hood. The acking strategy is heavily optimized. + +#### Failing a transaction + +When using regular bolts, you can call the `fail` method on `OutputCollector` to fail the tuple trees of which that tuple is a member. Since transactional topologies hide the acking framework from you, they provide a different mechanism to fail a batch (and cause the batch to be replayed). Just throw a [FailedException](javadocs/backtype/storm/topology/FailedException.html). Unlike regular exceptions, this will only cause that particular batch to replay and will not crash the process. + +### Transactional spout + +The `TransactionalSpout` interface is completely different from a regular `Spout` interface. A `TransactionalSpout` implementation emits batches of tuples and must ensure that the same batch of tuples is always emitted for the same transaction id. + +A transactional spout looks like this while a topology is executing: + +![Storm cluster](images/transactional-spout-structure.png) + +The coordinator on the left is a regular Storm spout that emits a tuple whenever a batch should be emitted for a transaction. The emitters execute as a regular Storm bolt and are responsible for emitting the actual tuples for the batch. The emitters subscribe to the "batch emit" stream of the coordinator using an all grouping. + +The need to be idempotent with respect to the tuples it emits requires a `TransactionalSpout` to store a small amount of state. The state is stored in Zookeeper. + +The details of implementing a `TransactionalSpout` are in [the Javadoc](javadocs/backtype/storm/transactional/ITransactionalSpout.html). + +#### Partitioned Transactional Spout + +A common kind of transactional spout is one that reads the batches from a set of partitions across many queue brokers. For example, this is how [TransactionalKafkaSpout](https://github.com/nathanmarz/storm-contrib/blob/master/storm-kafka/src/jvm/storm/kafka/TransactionalKafkaSpout.java) works. An `IPartitionedTransactionalSpout` automates the bookkeeping work of managing the state for each partition to ensure idempotent replayability. See [the Javadoc](javadocs/backtype/storm/transactional/partitioned/IPartitionedTransactionalSpout.html) for more details. + +### Configuration + +There's two important bits of configuration for transactional topologies: + +1. *Zookeeper:* By default, transactional topologies will store state in the same Zookeeper instance as used to manage the Storm cluster. You can override this with the "transactional.zookeeper.servers" and "transactional.zookeeper.port" configs. +2. *Number of active batches permissible at once:* You must set a limit to the number of batches that can be processed at once. You configure this using the "topology.max.spout.pending" config. If you don't set this config, it will default to 1. + +## What if you can't emit the same batch of tuples for a given transaction id? + +So far the discussion around transactional topologies has assumed that you can always emit the exact same batch of tuples for the same transaction id. So what do you do if this is not possible? + +Consider an example of when this is not possible. Suppose you are reading tuples from a partitioned message broker (stream is partitioned across many machines), and a single transaction will include tuples from all the individual machines. Now suppose one of the nodes goes down at the same time that a transaction fails. Without that node, it is impossible to replay the same batch of tuples you just played for that transaction id. The processing in your topology will halt as its unable to replay the identical batch. The only possible solution is to emit a different batch for that transaction id than you emitted before. Is it possible to still achieve exactly-once messaging semantics even if the batches change? + +It turns out that you can still achieve exactly-once messaging semantics in your processing with a non-idempotent transactional spout, although this requires a bit more work on your part in developing the topology. + +If a batch can change for a given transaction id, then the logic we've been using so far of "skip the update if the transaction id in the database is the same as the id for the current transaction" is no longer valid. This is because the current batch is different than the batch for the last time the transaction was committed, so the result will not necessarily be the same. You can fix this problem by storing a little bit more state in the database. Let's again use the example of storing a global count in the database and suppose the partial count for the batch is stored in the `partialCount` variable. + +Instead of storing a value in the database that looks like this: + +```java +class Value { + Object count; + BigInteger txid; +} +``` + +For non-idempotent transactional spouts you should instead store a value that looks like this: + +```java +class Value { + Object count; + BigInteger txid; + Object prevCount; +} +``` + +The logic for the update is as follows: + +1. If the transaction id for the current batch is the same as the transaction id in the database, set `val.count = val.prevCount + partialCount`. +2. Otherwise, set `val.prevCount = val.count`, `val.count = val.count + partialCount` and `val.txid = batchTxid`. + +This logic works because once you commit a particular transaction id for the first time, all prior transaction ids will never be committed again. + +There's a few more subtle aspects of transactional topologies that make opaque transactional spouts possible. + +When a transaction fails, all subsequent transactions in the processing phase are considered failed as well. Each of those transactions will be re-emitted and reprocessed. Without this behavior, the following situation could happen: + +1. Transaction A emits tuples 1-50 +2. Transaction B emits tuples 51-100 +3. Transaction A fails +4. Transaction A emits tuples 1-40 +5. Transaction A commits +6. Transaction B commits +7. Transaction C emits tuples 101-150 + +In this scenario, tuples 41-50 are skipped. By failing all subsequent transactions, this would happen instead: + +1. Transaction A emits tuples 1-50 +2. Transaction B emits tuples 51-100 +3. Transaction A fails (and causes Transaction B to fail) +4. Transaction A emits tuples 1-40 +5. Transaction B emits tuples 41-90 +5. Transaction A commits +6. Transaction B commits +7. Transaction C emits tuples 91-140 + +By failing all subsequent transactions on failure, no tuples are skipped. This also shows that a requirement of transactional spouts is that they always emit where the last transaction left off. + +A non-idempotent transactional spout is more concisely referred to as an "OpaqueTransactionalSpout" (opaque is the opposite of idempotent). [IOpaquePartitionedTransactionalSpout](javadocs/backtype/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.html) is an interface for implementing opaque partitioned transactional spouts, of which [OpaqueTransactionalKafkaSpout](https://github.com/nathanmarz/storm-contrib/blob/kafka0.7/storm-kafka/src/jvm/storm/kafka/OpaqueTransactionalKafkaSpout.java) is an example. `OpaqueTransactionalKafkaSpout` can withstand losing individual Kafka nodes without sacrificing accuracy as long as you use the update strategy as explained in this section. + +## Implementation + +The implementation for transactional topologies is very elegant. Managing the commit protocol, detecting failures, and pipelining batches seem complex, but everything turns out to be a straightforward mapping to Storm's primitives. + +How the data flow works: + +Here's how transactional spout works: + +1. Transactional spout is a subtopology consisting of a coordinator spout and an emitter bolt +2. The coordinator is a regular spout with a parallelism of 1 +3. The emitter is a bolt with a parallelism of P, connected to the coordinator's "batch" stream using an all grouping +4. When the coordinator determines it's time to enter the processing phase for a transaction, it emits a tuple containing the TransactionAttempt and the metadata for that transaction to the "batch" stream +5. Because of the all grouping, every single emitter task receives the notification that it's time to emit its portion of the tuples for that transaction attempt +6. Storm automatically manages the anchoring/acking necessary throughout the whole topology to determine when a transaction has completed the processing phase. The key here is that *the root tuple was created by the coordinator, so the coordinator will receive an "ack" if the processing phase succeeds, and a "fail" if it doesn't succeed for any reason (failure or timeout). +7. If the processing phase succeeds, and all prior transactions have successfully committed, the coordinator emits a tuple containing the TransactionAttempt to the "commit" stream. +8. All committing bolts subscribe to the commit stream using an all grouping, so that they will all receive a notification when the commit happens. +9. Like the processing phase, the coordinator uses the acking framework to determine whether the commit phase succeeded or not. If it receives an "ack", it marks that transaction as complete in zookeeper. + +More notes: + +- Transactional spouts are a sub-topology consisting of a spout and a bolt + - the spout is the coordinator and contains a single task + - the bolt is the emitter + - the bolt subscribes to the coordinator with an all grouping + - serialization of metadata is handled by kryo. kryo is initialized ONLY with the registrations defined in the component configuration for the transactionalspout +- the coordinator uses the acking framework to determine when a batch has been successfully processed, and then to determine when a batch has been successfully committed. +- state is stored in zookeeper using RotatingTransactionalState +- commiting bolts subscribe to the coordinators commit stream using an all grouping +- CoordinatedBolt is used to detect when a bolt has received all the tuples for a particular batch. + - this is the same abstraction that is used in DRPC + - for commiting bolts, it waits to receive a tuple from the coordinator's commit stream before calling finishbatch + - so it can't call finishbatch until it's received all tuples from all subscribed components AND its received the commit stream tuple (for committers). this ensures that it can't prematurely call finishBatch diff --git a/docs/Trident-API-Overview.md b/docs/Trident-API-Overview.md new file mode 100644 index 00000000000..3b68645f4f8 --- /dev/null +++ b/docs/Trident-API-Overview.md @@ -0,0 +1,311 @@ +--- +layout: documentation +--- +# Trident API overview + +The core data model in Trident is the "Stream", processed as a series of batches. A stream is partitioned among the nodes in the cluster, and operations applied to a stream are applied in parallel across each partition. + +There are five kinds of operations in Trident: + +1. Operations that apply locally to each partition and cause no network transfer +2. Repartitioning operations that repartition a stream but otherwise don't change the contents (involves network transfer) +3. Aggregation operations that do network transfer as part of the operation +4. Operations on grouped streams +5. Merges and joins + +## Partition-local operations + +Partition-local operations involve no network transfer and are applied to each batch partition independently. + +### Functions + +A function takes in a set of input fields and emits zero or more tuples as output. The fields of the output tuple are appended to the original input tuple in the stream. If a function emits no tuples, the original input tuple is filtered out. Otherwise, the input tuple is duplicated for each output tuple. Suppose you have this function: + +```java +public class MyFunction extends BaseFunction { + public void execute(TridentTuple tuple, TridentCollector collector) { + for(int i=0; i < tuple.getInteger(0); i++) { + collector.emit(new Values(i)); + } + } +} +``` + +Now suppose you have a stream in the variable "mystream" with the fields ["a", "b", "c"] with the following tuples: + +``` +[1, 2, 3] +[4, 1, 6] +[3, 0, 8] +``` + +If you run this code: + +```java +mystream.each(new Fields("b"), new MyFunction(), new Fields("d"))) +``` + +The resulting tuples would have fields ["a", "b", "c", "d"] and look like this: + +``` +[1, 2, 3, 0] +[1, 2, 3, 1] +[4, 1, 6, 0] +``` + +### Filters + +Filters take in a tuple as input and decide whether or not to keep that tuple or not. Suppose you had this filter: + +```java +public class MyFilter extends BaseFunction { + public boolean isKeep(TridentTuple tuple) { + return tuple.getInteger(0) == 1 && tuple.getInteger(1) == 2; + } +} +``` + +Now suppose you had these tuples with fields ["a", "b", "c"]: + +``` +[1, 2, 3] +[2, 1, 1] +[2, 3, 4] +``` + +If you ran this code: + +```java +mystream.each(new Fields("b", "a"), new MyFilter()) +``` + +The resulting tuples would be: + +``` +[2, 1, 1] +``` + +### partitionAggregate + +partitionAggregate runs a function on each partition of a batch of tuples. Unlike functions, the tuples emitted by partitionAggregate replace the input tuples given to it. Consider this example: + +```java +mystream.partitionAggregate(new Fields("b"), new Sum(), new Fields("sum")) +``` + +Suppose the input stream contained fields ["a", "b"] and the following partitions of tuples: + +``` +Partition 0: +["a", 1] +["b", 2] + +Partition 1: +["a", 3] +["c", 8] + +Partition 2: +["e", 1] +["d", 9] +["d", 10] +``` + +Then the output stream of that code would contain these tuples with one field called "sum": + +``` +Partition 0: +[3] + +Partition 1: +[11] + +Partition 2: +[20] +``` + +There are three different interfaces for defining aggregators: CombinerAggregator, ReducerAggregator, and Aggregator. + +Here's the interface for CombinerAggregator: + +```java +public interface CombinerAggregator extends Serializable { + T init(TridentTuple tuple); + T combine(T val1, T val2); + T zero(); +} +``` + +A CombinerAggregator returns a single tuple with a single field as output. CombinerAggregators run the init function on each input tuple and use the combine function to combine values until there's only one value left. If there's no tuples in the partition, the CombinerAggregator emits the output of the zero function. For example, here's the implementation of Count: + +```java +public class Count implements CombinerAggregator { + public Long init(TridentTuple tuple) { + return 1L; + } + + public Long combine(Long val1, Long val2) { + return val1 + val2; + } + + public Long zero() { + return 0L; + } +} +``` + +The benefits of CombinerAggregators are seen when you use the with the aggregate method instead of partitionAggregate. In that case, Trident automatically optimizes the computation by doing partial aggregations before transferring tuples over the network. + +A ReducerAggregator has the following interface: + +```java +public interface ReducerAggregator extends Serializable { + T init(); + T reduce(T curr, TridentTuple tuple); +} +``` + +A ReducerAggregator produces an initial value with init, and then it iterates on that value for each input tuple to produce a single tuple with a single value as output. For example, here's how you would define Count as a ReducerAggregator: + +```java +public class Count implements ReducerAggregator { + public Long init() { + return 0L; + } + + public Long reduce(Long curr, TridentTuple tuple) { + return curr + 1; + } +} +``` + +ReducerAggregator can also be used with persistentAggregate, as you'll see later. + +The most general interface for performing aggregations is Aggregator, which looks like this: + +```java +public interface Aggregator extends Operation { + T init(Object batchId, TridentCollector collector); + void aggregate(T state, TridentTuple tuple, TridentCollector collector); + void complete(T state, TridentCollector collector); +} +``` + +Aggregators can emit any number of tuples with any number of fields. They can emit tuples at any point during execution. Aggregators execute in the following way: + +1. The init method is called before processing the batch. The return value of init is an Object that will represent the state of the aggregation and will be passed into the aggregate and complete methods. +2. The aggregate method is called for each input tuple in the batch partition. This method can update the state and optionally emit tuples. +3. The complete method is called when all tuples for the batch partition have been processed by aggregate. + +Here's how you would implement Count as an Aggregator: + +```java +public class CountAgg extends BaseAggregator { + static class CountState { + long count = 0; + } + + public CountState init(Object batchId, TridentCollector collector) { + return new CountState(); + } + + public void aggregate(CountState state, TridentTuple tuple, TridentCollector collector) { + state.count+=1; + } + + public void complete(CountState state, TridentCollector collector) { + collector.emit(new Values(state.count)); + } +} +``` + +Sometimes you want to execute multiple aggregators at the same time. This is called chaining and can be accomplished like this: + +```java +mystream.chainedAgg() + .partitionAggregate(new Count(), new Fields("count")) + .partitionAggregate(new Fields("b"), new Sum(), new Fields("sum")) + .chainEnd() +``` + +This code will run the Count and Sum aggregators on each partition. The output will contain a single tuple with the fields ["count", "sum"]. + +### stateQuery and partitionPersist + +stateQuery and partitionPersist query and update sources of state, respectively. You can read about how to use them on [Trident state doc](Trident-state.html). + +### projection + +The projection method on Stream keeps only the fields specified in the operation. If you had a Stream with fields ["a", "b", "c", "d"] and you ran this code: + +```java +mystream.project(new Fields("b", "d")) +``` + +The output stream would contain only the fields ["b", "d"]. + + +## Repartitioning operations + +Repartitioning operations run a function to change how the tuples are partitioned across tasks. The number of partitions can also change as a result of repartitioning (for example, if the parallelism hint is greater after repartioning). Repartitioning requires network transfer. Here are the repartitioning functions: + +1. shuffle: Use random round robin algorithm to evenly redistribute tuples across all target partitions +2. broadcast: Every tuple is replicated to all target partitions. This can useful during DRPC – for example, if you need to do a stateQuery on every partition of data. +3. partitionBy: partitionBy takes in a set of fields and does semantic partitioning based on that set of fields. The fields are hashed and modded by the number of target partitions to select the target partition. partitionBy guarantees that the same set of fields always goes to the same target partition. +4. global: All tuples are sent to the same partition. The same partition is chosen for all batches in the stream. +5. batchGlobal: All tuples in the batch are sent to the same partition. Different batches in the stream may go to different partitions. +6. partition: This method takes in a custom partitioning function that implements backtype.storm.grouping.CustomStreamGrouping + +## Aggregation operations + +Trident has aggregate and persistentAggregate methods for doing aggregations on Streams. aggregate is run on each batch of the stream in isolation, while persistentAggregate will aggregation on all tuples across all batches in the stream and store the result in a source of state. + +Running aggregate on a Stream does a global aggregation. When you use a ReducerAggregator or an Aggregator, the stream is first repartitioned into a single partition, and then the aggregation function is run on that partition. When you use a CombinerAggregator, on the other hand, first Trident will compute partial aggregations of each partition, then repartition to a single partition, and then finish the aggregation after the network transfer. CombinerAggregator's are far more efficient and should be used when possible. + +Here's an example of using aggregate to get a global count for a batch: + +```java +mystream.aggregate(new Count(), new Fields("count")) +``` + +Like partitionAggregate, aggregators for aggregate can be chained. However, if you chain a CombinerAggregator with a non-CombinerAggregator, Trident is unable to do the partial aggregation optimization. + +You can read more about how to use persistentAggregate in the [Trident state doc](https://github.com/apache/incubator-storm/wiki/Trident-state). + +## Operations on grouped streams + +The groupBy operation repartitions the stream by doing a partitionBy on the specified fields, and then within each partition groups tuples together whose group fields are equal. For example, here's an illustration of a groupBy operation: + +![Grouping](images/grouping.png) + +If you run aggregators on a grouped stream, the aggregation will be run within each group instead of against the whole batch. persistentAggregate can also be run on a GroupedStream, in which case the results will be stored in a [MapState](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/storm/trident/state/map/MapState.java) with the key being the grouping fields. You can read more about persistentAggregate in the [Trident state doc](Trident-state.html). + +Like regular streams, aggregators on grouped streams can be chained. + +## Merges and joins + +The last part of the API is combining different streams together. The simplest way to combine streams is to merge them into one stream. You can do that with the TridentTopology#merge method, like so: + +```java +topology.merge(stream1, stream2, stream3); +``` + +Trident will name the output fields of the new, merged stream as the output fields of the first stream. + +Another way to combine streams is with a join. Now, a standard join, like the kind from SQL, require finite input. So they don't make sense with infinite streams. Joins in Trident only apply within each small batch that comes off of the spout. + +Here's an example join between a stream containing fields ["key", "val1", "val2"] and another stream containing ["x", "val1"]: + +```java +topology.join(stream1, new Fields("key"), stream2, new Fields("x"), new Fields("key", "a", "b", "c")); +``` + +This joins stream1 and stream2 together using "key" and "x" as the join fields for each respective stream. Then, Trident requires that all the output fields of the new stream be named, since the input streams could have overlapping field names. The tuples emitted from the join will contain: + +1. First, the list of join fields. In this case, "key" corresponds to "key" from stream1 and "x" from stream2. +2. Next, a list of all non-join fields from all streams, in order of how the streams were passed to the join method. In this case, "a" and "b" correspond to "val1" and "val2" from stream1, and "c" corresponds to "val1" from stream2. + +When a join happens between streams originating from different spouts, those spouts will be synchronized with how they emit batches. That is, a batch of processing will include tuples from each spout. + +You might be wondering – how do you do something like a "windowed join", where tuples from one side of the join are joined against the last hour of tuples from the other side of the join. + +To do this, you would make use of partitionPersist and stateQuery. The last hour of tuples from one side of the join would be stored and rotated in a source of state, keyed by the join field. Then the stateQuery would do lookups by the join field to perform the "join". diff --git a/docs/Trident-spouts.md b/docs/Trident-spouts.md new file mode 100644 index 00000000000..92330a7cfb2 --- /dev/null +++ b/docs/Trident-spouts.md @@ -0,0 +1,42 @@ +--- +layout: documentation +--- +# Trident spouts + +Like in the vanilla Storm API, spouts are the source of streams in a Trident topology. On top of the vanilla Storm spouts, Trident exposes additional APIs for more sophisticated spouts. + +There is an inextricable link between how you source your data streams and how you update state (e.g. databases) based on those data streams. See [Trident state doc](Trident-state.html) for an explanation of this – understanding this link is imperative for understanding the spout options available. + +Regular Storm spouts will be non-transactional spouts in a Trident topology. To use a regular Storm IRichSpout, create the stream like this in a TridentTopology: + +```java +TridentTopology topology = new TridentTopology(); +topology.newStream("myspoutid", new MyRichSpout()); +``` + +All spouts in a Trident topology are required to be given a unique identifier for the stream – this identifier must be unique across all topologies run on the cluster. Trident will use this identifier to store metadata about what the spout has consumed in Zookeeper, including the txid and any metadata associated with the spout. + +You can configure the Zookeeper storage of spout metadata via the following configuration options: + +1. `transactional.zookeeper.servers`: A list of Zookeeper hostnames +2. `transactional.zookeeper.port`: The port of the Zookeeper cluster +3. `transactional.zookeeper.root`: The root dir in Zookeeper where metadata is stored. Metadata will be stored at the path / + +## Pipelining + +By default, Trident processes a single batch at a time, waiting for the batch to succeed or fail before trying another batch. You can get significantly higher throughput – and lower latency of processing of each batch – by pipelining the batches. You configure the maximum amount of batches to be processed simultaneously with the "topology.max.spout.pending" property. + +Even while processing multiple batches simultaneously, Trident will order any state updates taking place in the topology among batches. For example, suppose you're doing a global count aggregation into a database. The idea is that while you're updating the count in the database for batch 1, you can still be computing the partial counts for batches 2 through 10. Trident won't move on to the state updates for batch 2 until the state updates for batch 1 have succeeded. This is essential for achieving exactly-once processing semantics, as outline in [Trident state doc](Trident-state.html). + +## Trident spout types + +Here are the following spout APIs available: + +1. [ITridentSpout](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/storm/trident/spout/ITridentSpout.java): The most general API that can support transactional or opaque transactional semantics. Generally you'll use one of the partitioned flavors of this API rather than this one directly. +2. [IBatchSpout](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/storm/trident/spout/IBatchSpout.java): A non-transactional spout that emits batches of tuples at a time +3. [IPartitionedTridentSpout](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/storm/trident/spout/IPartitionedTridentSpout.java): A transactional spout that reads from a partitioned data source (like a cluster of Kafka servers) +4. [IOpaquePartitionedTridentSpout](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/storm/trident/spout/IOpaquePartitionedTridentSpout.java): An opaque transactional spout that reads from a partitioned data source + +And, like mentioned in the beginning of this tutorial, you can use regular IRichSpout's as well. + + diff --git a/docs/Trident-state.md b/docs/Trident-state.md new file mode 100644 index 00000000000..2ace8c82dbb --- /dev/null +++ b/docs/Trident-state.md @@ -0,0 +1,330 @@ +--- +layout: documentation +--- +# State in Trident + +Trident has first-class abstractions for reading from and writing to stateful sources. The state can either be internal to the topology – e.g., kept in-memory and backed by HDFS – or externally stored in a database like Memcached or Cassandra. There's no difference in the Trident API for either case. + +Trident manages state in a fault-tolerant way so that state updates are idempotent in the face of retries and failures. This lets you reason about Trident topologies as if each message were processed exactly-once. + +There's various levels of fault-tolerance possible when doing state updates. Before getting to those, let's look at an example that illustrates the tricks necessary to achieve exactly-once semantics. Suppose that you're doing a count aggregation of your stream and want to store the running count in a database. Now suppose you store in the database a single value representing the count, and every time you process a new tuple you increment the count. + +When failures occur, tuples will be replayed. This brings up a problem when doing state updates (or anything with side effects) – you have no idea if you've ever successfully updated the state based on this tuple before. Perhaps you never processed the tuple before, in which case you should increment the count. Perhaps you've processed the tuple and successfully incremented the count, but the tuple failed processing in another step. In this case, you should not increment the count. Or perhaps you saw the tuple before but got an error when updating the database. In this case, you *should* update the database. + +By just storing the count in the database, you have no idea whether or not this tuple has been processed before. So you need more information in order to make the right decision. Trident provides the following semantics which are sufficient for achieving exactly-once processing semantics: + +1. Tuples are processed as small batches (see [the tutorial](Trident-tutorial.html)) +2. Each batch of tuples is given a unique id called the "transaction id" (txid). If the batch is replayed, it is given the exact same txid. +3. State updates are ordered among batches. That is, the state updates for batch 3 won't be applied until the state updates for batch 2 have succeeded. + +With these primitives, your State implementation can detect whether or not the batch of tuples has been processed before and take the appropriate action to update the state in a consistent way. The action you take depends on the exact semantics provided by your input spouts as to what's in each batch. There's three kinds of spouts possible with respect to fault-tolerance: "non-transactional", "transactional", and "opaque transactional". Likewise, there's three kinds of state possible with respect to fault-tolerance: "non-transactional", "transactional", and "opaque transactional". Let's take a look at each spout type and see what kind of fault-tolerance you can achieve with each. + +## Transactional spouts + +Remember, Trident processes tuples as small batches with each batch being given a unique transaction id. The properties of spouts vary according to the guarantees they can provide as to what's in each batch. A transactional spout has the following properties: + +1. Batches for a given txid are always the same. Replays of batches for a txid will exact same set of tuples as the first time that batch was emitted for that txid. +2. There's no overlap between batches of tuples (tuples are in one batch or another, never multiple). +3. Every tuple is in a batch (no tuples are skipped) + +This is a pretty easy type of spout to understand, the stream is divided into fixed batches that never change. storm-contrib has [an implementation of a transactional spout](https://github.com/nathanmarz/storm-contrib/blob/{{page.version}}/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java) for Kafka. + +You might be wondering – why wouldn't you just always use a transactional spout? They're simple and easy to understand. One reason you might not use one is because they're not necessarily very fault-tolerant. For example, the way TransactionalTridentKafkaSpout works is the batch for a txid will contain tuples from all the Kafka partitions for a topic. Once a batch has been emitted, any time that batch is re-emitted in the future the exact same set of tuples must be emitted to meet the semantics of transactional spouts. Now suppose a batch is emitted from TransactionalTridentKafkaSpout, the batch fails to process, and at the same time one of the Kafka nodes goes down. You're now incapable of replaying the same batch as you did before (since the node is down and some partitions for the topic are not unavailable), and processing will halt. + +This is why "opaque transactional" spouts exist – they are fault-tolerant to losing source nodes while still allowing you to achieve exactly-once processing semantics. We'll cover those spouts in the next section though. + +(One side note – once Kafka supports replication, it will be possible to have transactional spouts that are fault-tolerant to node failure, but that feature does not exist yet.) + +Before we get to "opaque transactional" spouts, let's look at how you would design a State implementation that has exactly-once semantics for transactional spouts. This State type is called a "transactional state" and takes advantage of the fact that any given txid is always associated with the exact same set of tuples. + +Suppose your topology computes word count and you want to store the word counts in a key/value database. The key will be the word, and the value will contain the count. You've already seen that storing just the count as the value isn't sufficient to know whether you've processed a batch of tuples before. Instead, what you can do is store the transaction id with the count in the database as an atomic value. Then, when updating the count, you can just compare the transaction id in the database with the transaction id for the current batch. If they're the same, you skip the update – because of the strong ordering, you know for sure that the value in the database incorporates the current batch. If they're different, you increment the count. This logic works because the batch for a txid never changes, and Trident ensures that state updates are ordered among batches. + +Consider this example of why it works. Suppose you are processing txid 3 which consists of the following batch of tuples: + +``` +["man"] +["man"] +["dog"] +``` + +Suppose the database currently holds the following key/value pairs: + +``` +man => [count=3, txid=1] +dog => [count=4, txid=3] +apple => [count=10, txid=2] +``` + +The txid associated with "man" is txid 1. Since the current txid is 3, you know for sure that this batch of tuples is not represented in that count. So you can go ahead and increment the count by 2 and update the txid. On the other hand, the txid for "dog" is the same as the current txid. So you know for sure that the increment from the current batch is already represented in the database for the "dog" key. So you can skip the update. After completing updates, the database looks like this: + +``` +man => [count=5, txid=3] +dog => [count=4, txid=3] +apple => [count=10, txid=2] +``` + +Let's now look at opaque transactional spouts and how to design states for that type of spout. + +## Opaque transactional spouts + +As described before, an opaque transactional spout cannot guarantee that the batch of tuples for a txid remains constant. An opaque transactional spout has the following property: + +1. Every tuple is *successfully* processed in exactly one batch. However, it's possible for a tuple to fail to process in one batch and then succeed to process in a later batch. + +[OpaqueTridentKafkaSpout](https://github.com/nathanmarz/storm-contrib/blob/{{page.version}}/storm-kafka/src/jvm/storm/kafka/trident/OpaqueTridentKafkaSpout.java) is a spout that has this property and is fault-tolerant to losing Kafka nodes. Whenever it's time for OpaqueTridentKafkaSpout to emit a batch, it emits tuples starting from where the last batch finished emitting. This ensures that no tuple is ever skipped or successfully processed by multiple batches. + +With opaque transactional spouts, it's no longer possible to use the trick of skipping state updates if the transaction id in the database is the same as the transaction id for the current batch. This is because the batch may have changed between state updates. + +What you can do is store more state in the database. Rather than store a value and transaction id in the database, you instead store a value, transaction id, and the previous value in the database. Let's again use the example of storing a count in the database. Suppose the partial count for your batch is "2" and it's time to apply a state update. Suppose the value in the database looks like this: + +``` +{ value = 4, + prevValue = 1, + txid = 2 +} +``` + +Suppose your current txid is 3, different than what's in the database. In this case, you set "prevValue" equal to "value", increment "value" by your partial count, and update the txid. The new database value will look like this: + +``` +{ value = 6, + prevValue = 4, + txid = 3 +} +``` + +Now suppose your current txid is 2, equal to what's in the database. Now you know that the "value" in the database contains an update from a previous batch for your current txid, but that batch may have been different so you have to ignore it. What you do in this case is increment "prevValue" by your partial count to compute the new "value". You then set the value in the database to this: + +``` +{ value = 3, + prevValue = 1, + txid = 2 +} +``` + +This works because of the strong ordering of batches provided by Trident. Once Trident moves onto a new batch for state updates, it will never go back to a previous batch. And since opaque transactional spouts guarantee no overlap between batches – that each tuple is successfully processed by one batch – you can safely update based on the previous value. + +## Non-transactional spouts + +Non-transactional spouts don't provide any guarantees about what's in each batch. So it might have at-most-once processing, in which case tuples are not retried after failed batches. Or it might have at-least-once processing, where tuples can be processed successfully by multiple batches. There's no way to achieve exactly-once semantics for this kind of spout. + +## Summary of spout and state types + +This diagram shows which combinations of spouts / states enable exactly-once messaging semantics: + +![Spouts vs States](images/spout-vs-state.png) + +Opaque transactional states have the strongest fault-tolerance, but this comes at the cost of needing to store the txid and two values in the database. Transactional states require less state in the database, but only work with transactional spouts. Finally, non-transactional states require the least state in the database but cannot achieve exactly-once semantics. + +The state and spout types you choose are a tradeoff between fault-tolerance and storage costs, and ultimately your application requirements will determine which combination is right for you. + +## State APIs + +You've seen the intricacies of what it takes to achieve exactly-once semantics. The nice thing about Trident is that it internalizes all the fault-tolerance logic within the State – as a user you don't have to deal with comparing txids, storing multiple values in the database, or anything like that. You can write code like this: + +```java +TridentTopology topology = new TridentTopology(); +TridentState wordCounts = + topology.newStream("spout1", spout) + .each(new Fields("sentence"), new Split(), new Fields("word")) + .groupBy(new Fields("word")) + .persistentAggregate(MemcachedState.opaque(serverLocations), new Count(), new Fields("count")) + .parallelismHint(6); +``` + +All the logic necessary to manage opaque transactional state logic is internalized in the MemcachedState.opaque call. Additionally, updates are automatically batched to minimize roundtrips to the database. + +The base State interface just has two methods: + +```java +public interface State { + void beginCommit(Long txid); // can be null for things like partitionPersist occurring off a DRPC stream + void commit(Long txid); +} +``` + +You're told when a state update is beginning, when a state update is ending, and you're given the txid in each case. Trident assumes nothing about how your state works, what kind of methods there are to update it, and what kind of methods there are to read from it. + +Suppose you have a home-grown database that contains user location information and you want to be able to access it from Trident. Your State implementation would have methods for getting and setting user information: + +```java +public class LocationDB implements State { + public void beginCommit(Long txid) { + } + + public void commit(Long txid) { + } + + public void setLocation(long userId, String location) { + // code to access database and set location + } + + public String getLocation(long userId) { + // code to get location from database + } +} +``` + +You then provide Trident a StateFactory that can create instances of your State object within Trident tasks. The StateFactory for your LocationDB might look something like this: + +```java +public class LocationDBFactory implements StateFactory { + public State makeState(Map conf, int partitionIndex, int numPartitions) { + return new LocationDB(); + } +} +``` + +Trident provides the QueryFunction interface for writing Trident operations that query a source of state, and the StateUpdater interface for writing Trident operations that update a source of state. For example, let's write an operation "QueryLocation" that queries the LocationDB for the locations of users. Let's start off with how you would use it in a topology. Let's say this topology consumes an input stream of userids: + +```java +TridentTopology topology = new TridentTopology(); +TridentState locations = topology.newStaticState(new LocationDBFactory()); +topology.newStream("myspout", spout) + .stateQuery(locations, new Fields("userid"), new QueryLocation(), new Fields("location")) +``` + +Now let's take a look at what the implementation of QueryLocation would look like: + +```java +public class QueryLocation extends BaseQueryFunction { + public List batchRetrieve(LocationDB state, List inputs) { + List ret = new ArrayList(); + for(TridentTuple input: inputs) { + ret.add(state.getLocation(input.getLong(0))); + } + return ret; + } + + public void execute(TridentTuple tuple, String location, TridentCollector collector) { + collector.emit(new Values(location)); + } +} +``` + +QueryFunction's execute in two steps. First, Trident collects a batch of reads together and passes them to batchRetrieve. In this case, batchRetrieve will receive multiple user ids. batchRetrieve is expected to return a list of results that's the same size as the list of input tuples. The first element of the result list corresponds to the result for the first input tuple, the second is the result for the second input tuple, and so on. + +You can see that this code doesn't take advantage of the batching that Trident does, since it just queries the LocationDB one at a time. So a better way to write the LocationDB would be like this: + +```java +public class LocationDB implements State { + public void beginCommit(Long txid) { + } + + public void commit(Long txid) { + } + + public void setLocationsBulk(List userIds, List locations) { + // set locations in bulk + } + + public List bulkGetLocations(List userIds) { + // get locations in bulk + } +} +``` + +Then, you can write the QueryLocation function like this: + +```java +public class QueryLocation extends BaseQueryFunction { + public List batchRetrieve(LocationDB state, List inputs) { + List userIds = new ArrayList(); + for(TridentTuple input: inputs) { + userIds.add(input.getLong(0)); + } + return state.bulkGetLocations(userIds); + } + + public void execute(TridentTuple tuple, String location, TridentCollector collector) { + collector.emit(new Values(location)); + } +} +``` + +This code will be much more efficient by reducing roundtrips to the database. + +To update state, you make use of the StateUpdater interface. Here's a StateUpdater that updates a LocationDB with new location information: + +```java +public class LocationUpdater extends BaseStateUpdater { + public void updateState(LocationDB state, List tuples, TridentCollector collector) { + List ids = new ArrayList(); + List locations = new ArrayList(); + for(TridentTuple t: tuples) { + ids.add(t.getLong(0)); + locations.add(t.getString(1)); + } + state.setLocationsBulk(ids, locations); + } +} +``` + +Here's how you would use this operation in a Trident topology: + +```java +TridentTopology topology = new TridentTopology(); +TridentState locations = + topology.newStream("locations", locationsSpout) + .partitionPersist(new LocationDBFactory(), new Fields("userid", "location"), new LocationUpdater()) +``` + +The partitionPersist operation updates a source of state. The StateUpdater receives the State and a batch of tuples with updates to that State. This code just grabs the userids and locations from the input tuples and does a bulk set into the State. + +partitionPersist returns a TridentState object representing the location db being updated by the Trident topology. You could then use this state in stateQuery operations elsewhere in the topology. + +You can also see that StateUpdaters are given a TridentCollector. Tuples emitted to this collector go to the "new values stream". In this case, there's nothing interesting to emit to that stream, but if you were doing something like updating counts in a database, you could emit the updated counts to that stream. You can then get access to the new values stream for further processing via the TridentState#newValuesStream method. + +## persistentAggregate + +Trident has another method for updating States called persistentAggregate. You've seen this used in the streaming word count example, shown again below: + +```java +TridentTopology topology = new TridentTopology(); +TridentState wordCounts = + topology.newStream("spout1", spout) + .each(new Fields("sentence"), new Split(), new Fields("word")) + .groupBy(new Fields("word")) + .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count")) +``` + +persistentAggregate is an additional abstraction built on top of partitionPersist that knows how to take a Trident aggregator and use it to apply updates to the source of state. In this case, since this is a grouped stream, Trident expects the state you provide to implement the "MapState" interface. The grouping fields will be the keys in the state, and the aggregation result will be the values in the state. The "MapState" interface looks like this: + +```java +public interface MapState extends State { + List multiGet(List> keys); + List multiUpdate(List> keys, List updaters); + void multiPut(List> keys, List vals); +} +``` + +When you do aggregations on non-grouped streams (a global aggregation), Trident expects your State object to implement the "Snapshottable" interface: + +```java +public interface Snapshottable extends State { + T get(); + T update(ValueUpdater updater); + void set(T o); +} +``` + +[MemoryMapState](https://github.com/apache/incubator-storm/blob/{{page.version}}/storm-core/src/jvm/storm/trident/testing/MemoryMapState.java) and [MemcachedState](https://github.com/nathanmarz/trident-memcached/blob/master/src/jvm/trident/memcached/MemcachedState.java) each implement both of these interfaces. + +## Implementing Map States + +Trident makes it easy to implement MapState's, doing almost all the work for you. The OpaqueMap, TransactionalMap, and NonTransactionalMap classes implement all the logic for doing the respective fault-tolerance logic. You simply provide these classes with an IBackingMap implementation that knows how to do multiGets and multiPuts of the respective key/values. IBackingMap looks like this: + +```java +public interface IBackingMap { + List multiGet(List> keys); + void multiPut(List> keys, List vals); +} +``` + +OpaqueMap's will call multiPut with [OpaqueValue](https://github.com/apache/incubator-storm/blob/{{page.version}}/storm-core/src/jvm/storm/trident/state/OpaqueValue.java)'s for the vals, TransactionalMap's will give [TransactionalValue](https://github.com/apache/incubator-storm/blob/{{page.version}}/storm-core/src/jvm/storm/trident/state/TransactionalValue.java)'s for the vals, and NonTransactionalMaps will just pass the objects from the topology through. + +Trident also provides the [CachedMap](https://github.com/apache/incubator-storm/blob/{{page.version}}/storm-core/src/jvm/storm/trident/state/map/CachedMap.java) class to do automatic LRU caching of map key/vals. + +Finally, Trident provides the [SnapshottableMap](https://github.com/apache/incubator-storm/blob/{{page.version}}/storm-core/src/jvm/storm/trident/state/map/SnapshottableMap.java) class that turns a MapState into a Snapshottable object, by storing global aggregations into a fixed key. + +Take a look at the implementation of [MemcachedState](https://github.com/nathanmarz/trident-memcached/blob/master/src/jvm/trident/memcached/MemcachedState.java) to see how all these utilities can be put together to make a high performance MapState implementation. MemcachedState allows you to choose between opaque transactional, transactional, and non-transactional semantics. diff --git a/docs/Trident-tutorial.md b/docs/Trident-tutorial.md new file mode 100644 index 00000000000..862dd8b6610 --- /dev/null +++ b/docs/Trident-tutorial.md @@ -0,0 +1,253 @@ +--- +layout: documentation +--- +# Trident tutorial + +Trident is a high-level abstraction for doing realtime computing on top of Storm. It allows you to seamlessly intermix high throughput (millions of messages per second), stateful stream processing with low latency distributed querying. If you're familiar with high level batch processing tools like Pig or Cascading, the concepts of Trident will be very familiar – Trident has joins, aggregations, grouping, functions, and filters. In addition to these, Trident adds primitives for doing stateful, incremental processing on top of any database or persistence store. Trident has consistent, exactly-once semantics, so it is easy to reason about Trident topologies. + +## Illustrative example + +Let's look at an illustrative example of Trident. This example will do two things: + +1. Compute streaming word count from an input stream of sentences +2. Implement queries to get the sum of the counts for a list of words + +For the purposes of illustration, this example will read an infinite stream of sentences from the following source: + +```java +FixedBatchSpout spout = new FixedBatchSpout(new Fields("sentence"), 3, + new Values("the cow jumped over the moon"), + new Values("the man went to the store and bought some candy"), + new Values("four score and seven years ago"), + new Values("how many apples can you eat")); +spout.setCycle(true); +``` + +This spout cycles through that set of sentences over and over to produce the sentence stream. Here's the code to do the streaming word count part of the computation: + +```java +TridentTopology topology = new TridentTopology(); +TridentState wordCounts = + topology.newStream("spout1", spout) + .each(new Fields("sentence"), new Split(), new Fields("word")) + .groupBy(new Fields("word")) + .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count")) + .parallelismHint(6); +``` + +Let's go through the code line by line. First a TridentTopology object is created, which exposes the interface for constructing Trident computations. TridentTopology has a method called newStream that creates a new stream of data in the topology reading from an input source. In this case, the input source is just the FixedBatchSpout defined from before. Input sources can also be queue brokers like Kestrel or Kafka. Trident keeps track of a small amount of state for each input source (metadata about what it has consumed) in Zookeeper, and the "spout1" string here specifies the node in Zookeeper where Trident should keep that metadata. + +Trident processes the stream as small batches of tuples. For example, the incoming stream of sentences might be divided into batches like so: + +![Batched stream](images/batched-stream.png) + +Generally the size of those small batches will be on the order of thousands or millions of tuples, depending on your incoming throughput. + +Trident provides a fully fledged batch processing API to process those small batches. The API is very similar to what you see in high level abstractions for Hadoop like Pig or Cascading: you can do group by's, joins, aggregations, run functions, run filters, and so on. Of course, processing each small batch in isolation isn't that interesting, so Trident provides functions for doing aggregations across batches and persistently storing those aggregations – whether in memory, in Memcached, in Cassandra, or some other store. Finally, Trident has first-class functions for querying sources of realtime state. That state could be updated by Trident (like in this example), or it could be an independent source of state. + +Back to the example, the spout emits a stream containing one field called "sentence". The next line of the topology definition applies the Split function to each tuple in the stream, taking the "sentence" field and splitting it into words. Each sentence tuple creates potentially many word tuples – for instance, the sentence "the cow jumped over the moon" creates six "word" tuples. Here's the definition of Split: + +```java +public class Split extends BaseFunction { + public void execute(TridentTuple tuple, TridentCollector collector) { + String sentence = tuple.getString(0); + for(String word: sentence.split(" ")) { + collector.emit(new Values(word)); + } + } +} +``` + +As you can see, it's really simple. It simply grabs the sentence, splits it on whitespace, and emits a tuple for each word. + +The rest of the topology computes word count and keeps the results persistently stored. First the stream is grouped by the "word" field. Then, each group is persistently aggregated using the Count aggregator. The persistentAggregate function knows how to store and update the results of the aggregation in a source of state. In this example, the word counts are kept in memory, but this can be trivially swapped to use Memcached, Cassandra, or any other persistent store. Swapping this topology to store counts in Memcached is as simple as replacing the persistentAggregate line with this (using [trident-memcached](https://github.com/nathanmarz/trident-memcached)), where the "serverLocations" is a list of host/ports for the Memcached cluster: + +```java +.persistentAggregate(MemcachedState.transactional(serverLocations), new Count(), new Fields("count")) +MemcachedState.transactional() +``` + +The values stored by persistentAggregate represents the aggregation of all batches ever emitted by the stream. + +One of the cool things about Trident is that it has fully fault-tolerant, exactly-once processing semantics. This makes it easy to reason about your realtime processing. Trident persists state in a way so that if failures occur and retries are necessary, it won't perform multiple updates to the database for the same source data. + +The persistentAggregate method transforms a Stream into a TridentState object. In this case the TridentState object represents all the word counts. We will use this TridentState object to implement the distributed query portion of the computation. + +The next part of the topology implements a low latency distributed query on the word counts. The query takes as input a whitespace separated list of words and return the sum of the counts for those words. These queries are executed just like normal RPC calls, except they are parallelized in the background. Here's an example of how you might invoke one of these queries: + +```java +DRPCClient client = new DRPCClient("drpc.server.location", 3772); +System.out.println(client.execute("words", "cat dog the man"); +// prints the JSON-encoded result, e.g.: "[[5078]]" +``` + +As you can see, it looks just like a regular remote procedure call (RPC), except it's executing in parallel across a Storm cluster. The latency for small queries like this are typically around 10ms. More intense DRPC queries can take longer of course, although the latency largely depends on how many resources you have allocated for the computation. + +The implementation of the distributed query portion of the topology looks like this: + +```java +topology.newDRPCStream("words") + .each(new Fields("args"), new Split(), new Fields("word")) + .groupBy(new Fields("word")) + .stateQuery(wordCounts, new Fields("word"), new MapGet(), new Fields("count")) + .each(new Fields("count"), new FilterNull()) + .aggregate(new Fields("count"), new Sum(), new Fields("sum")); +``` + +The same TridentTopology object is used to create the DRPC stream, and the function is named "words". The function name corresponds to the function name given in the first argument of execute when using a DRPCClient. + +Each DRPC request is treated as its own little batch processing job that takes as input a single tuple representing the request. The tuple contains one field called "args" that contains the argument provided by the client. In this case, the argument is a whitespace separated list of words. + +First, the Split function is used to split the arguments for the request into its constituent words. The stream is grouped by "word", and the stateQuery operator is used to query the TridentState object that the first part of the topology generated. stateQuery takes in a source of state – in this case, the word counts computed by the other portion of the topology – and a function for querying that state. In this case, the MapGet function is invoked, which gets the count for each word. Since the DRPC stream is grouped the exact same way as the TridentState was (by the "word" field), each word query is routed to the exact partition of the TridentState object that manages updates for that word. + +Next, words that didn't have a count are filtered out via the FilterNull filter and the counts are summed using the Sum aggregator to get the result. Then, Trident automatically sends the result back to the waiting client. + +Trident is intelligent about how it executes a topology to maximize performance. There's two interesting things happening automatically in this topology: + +1. Operations that read from or write to state (like persistentAggregate and stateQuery) automatically batch operations to that state. So if there's 20 updates that need to be made to the database for the current batch of processing, rather than do 20 read requests and 20 writes requests to the database, Trident will automatically batch up the reads and writes, doing only 1 read request and 1 write request (and in many cases, you can use caching in your State implementation to eliminate the read request). So you get the best of both words of convenience – being able to express your computation in terms of what should be done with each tuple – and performance. +2. Trident aggregators are heavily optimized. Rather than transfer all tuples for a group to the same machine and then run the aggregator, Trident will do partial aggregations when possible before sending tuples over the network. For example, the Count aggregator computes the count on each partition, sends the partial count over the network, and then sums together all the partial counts to get the total count. This technique is similar to the use of combiners in MapReduce. + +Let's look at another example of Trident. + +## Reach + +The next example is a pure DRPC topology that computes the reach of a URL on demand. Reach is the number of unique people exposed to a URL on Twitter. To compute reach, you need to fetch all the people who ever tweeted a URL, fetch all the followers of all those people, unique that set of followers, and that count that uniqued set. Computing reach is too intense for a single machine – it can require thousands of database calls and tens of millions of tuples. With Storm and Trident, you can parallelize the computation of each step across a cluster. + +This topology will read from two sources of state. One database maps URLs to a list of people who tweeted that URL. The other database maps a person to a list of followers for that person. The topology definition looks like this: + +```java +TridentState urlToTweeters = + topology.newStaticState(getUrlToTweetersState()); +TridentState tweetersToFollowers = + topology.newStaticState(getTweeterToFollowersState()); + +topology.newDRPCStream("reach") + .stateQuery(urlToTweeters, new Fields("args"), new MapGet(), new Fields("tweeters")) + .each(new Fields("tweeters"), new ExpandList(), new Fields("tweeter")) + .shuffle() + .stateQuery(tweetersToFollowers, new Fields("tweeter"), new MapGet(), new Fields("followers")) + .parallelismHint(200) + .each(new Fields("followers"), new ExpandList(), new Fields("follower")) + .groupBy(new Fields("follower")) + .aggregate(new One(), new Fields("one")) + .parallelismHint(20) + .aggregate(new Count(), new Fields("reach")); +``` + +The topology creates TridentState objects representing each external database using the newStaticState method. These can then be queried in the topology. Like all sources of state, queries to these databases will be automatically batched for maximum efficiency. + +The topology definition is straightforward – it's just a simple batch processing job. First, the urlToTweeters database is queried to get the list of people who tweeted the URL for this request. That returns a list, so the ExpandList function is invoked to create a tuple for each tweeter. + +Next, the followers for each tweeter must be fetched. It's important that this step be parallelized, so shuffle is invoked to evenly distribute the tweeters among all workers for the topology. Then, the followers database is queried to get the list of followers for each tweeter. You can see that this portion of the topology is given a large parallelism since this is the most intense portion of the computation. + +Next, the set of followers is uniqued and counted. This is done in two steps. First a "group by" is done on the batch by "follower", running the "One" aggregator on each group. The "One" aggregator simply emits a single tuple containing the number one for each group. Then, the ones are summed together to get the unique count of the followers set. Here's the definition of the "One" aggregator: + +```java +public class One implements CombinerAggregator { + public Integer init(TridentTuple tuple) { + return 1; + } + + public Integer combine(Integer val1, Integer val2) { + return 1; + } + + public Integer zero() { + return 1; + } +} +``` + +This is a "combiner aggregator", which knows how to do partial aggregations before transferring tuples over the network to maximize efficiency. Sum is also defined as a combiner aggregator, so the global sum done at the end of the topology will be very efficient. + +Let's now look at Trident in more detail. + +## Fields and tuples + +The Trident data model is the TridentTuple which is a named list of values. During a topology, tuples are incrementally built up through a sequence of operations. Operations generally take in a set of input fields and emit a set of "function fields". The input fields are used to select a subset of the tuple as input to the operation, while the "function fields" name the fields the operation emits. + +Consider this example. Suppose you have a stream called "stream" that contains the fields "x", "y", and "z". To run a filter MyFilter that takes in "y" as input, you would say: + +```java +stream.each(new Fields("y"), new MyFilter()) +``` + +Suppose the implementation of MyFilter is this: + +```java +public class MyFilter extends BaseFilter { + public boolean isKeep(TridentTuple tuple) { + return tuple.getInteger(0) < 10; + } +} +``` + +This will keep all tuples whose "y" field is less than 10. The TridentTuple given as input to MyFilter will only contain the "y" field. Note that Trident is able to project a subset of a tuple extremely efficiently when selecting the input fields: the projection is essentially free. + +Let's now look at how "function fields" work. Suppose you had this function: + +```java +public class AddAndMultiply extends BaseFunction { + public void execute(TridentTuple tuple, TridentCollector collector) { + int i1 = tuple.getInteger(0); + int i2 = tuple.getInteger(1); + collector.emit(new Values(i1 + i2, i1 * i2)); + } +} +``` + +This function takes two numbers as input and emits two new values: the addition of the numbers and the multiplication of the numbers. Suppose you had a stream with the fields "x", "y", and "z". You would use this function like this: + +```java +stream.each(new Fields("x", "y"), new AddAndMultiply(), new Fields("added", "multiplied")); +``` + +The output of functions is additive: the fields are added to the input tuple. So the output of this each call would contain tuples with the five fields "x", "y", "z", "added", and "multiplied". "added" corresponds to the first value emitted by AddAndMultiply, while "multiplied" corresponds to the second value. + +With aggregators, on the other hand, the function fields replace the input tuples. So if you had a stream containing the fields "val1" and "val2", and you did this: + +```java +stream.aggregate(new Fields("val2"), new Sum(), new Fields("sum")) +``` + +The output stream would only contain a single tuple with a single field called "sum", representing the sum of all "val2" fields in that batch. + +With grouped streams, the output will contain the grouping fields followed by the fields emitted by the aggregator. For example: + +```java +stream.groupBy(new Fields("val1")) + .aggregate(new Fields("val2"), new Sum(), new Fields("sum")) +``` + +In this example, the output will contain the fields "val1" and "sum". + +## State + +A key problem to solve with realtime computation is how to manage state so that updates are idempotent in the face of failures and retries. It's impossible to eliminate failures, so when a node dies or something else goes wrong, batches need to be retried. The question is – how do you do state updates (whether external databases or state internal to the topology) so that it's like each message was only processed only once? + +This is a tricky problem, and can be illustrated with the following example. Suppose that you're doing a count aggregation of your stream and want to store the running count in a database. If you store only the count in the database and it's time to apply a state update for a batch, there's no way to know if you applied that state update before. The batch could have been attempted before, succeeded in updating the database, and then failed at a later step. Or the batch could have been attempted before and failed to update the database. You just don't know. + +Trident solves this problem by doing two things: + +1. Each batch is given a unique id called the "transaction id". If a batch is retried it will have the exact same transaction id. +2. State updates are ordered among batches. That is, the state updates for batch 3 won't be applied until the state updates for batch 2 have succeeded. + +With these two primitives, you can achieve exactly-once semantics with your state updates. Rather than store just the count in the database, what you can do instead is store the transaction id with the count in the database as an atomic value. Then, when updating the count, you can just compare the transaction id in the database with the transaction id for the current batch. If they're the same, you skip the update – because of the strong ordering, you know for sure that the value in the database incorporates the current batch. If they're different, you increment the count. + +Of course, you don't have to do this logic manually in your topologies. This logic is wrapped by the State abstraction and done automatically. Nor is your State object required to implement the transaction id trick: if you don't want to pay the cost of storing the transaction id in the database, you don't have to. In that case the State will have at-least-once-processing semantics in the case of failures (which may be fine for your application). You can read more about how to implement a State and the various fault-tolerance tradeoffs possible [in this doc](Trident-state.html). + +A State is allowed to use whatever strategy it wants to store state. So it could store state in an external database or it could keep the state in-memory but backed by HDFS (like how HBase works). State's are not required to hold onto state forever. For example, you could have an in-memory State implementation that only keeps the last X hours of data available and drops anything older. Take a look at the implementation of the [Memcached integration](https://github.com/nathanmarz/trident-memcached/blob/master/src/jvm/trident/memcached/MemcachedState.java) for an example State implementation. + +## Execution of Trident topologies + +Trident topologies compile down into as efficient of a Storm topology as possible. Tuples are only sent over the network when a repartitioning of the data is required, such as if you do a groupBy or a shuffle. So if you had this Trident topology: + +![Compiling Trident to Storm 1](images/trident-to-storm1.png) + +It would compile into Storm spouts/bolts like this: + +![Compiling Trident to Storm 2](images/trident-to-storm2.png) + +## Conclusion + +Trident makes realtime computation elegant. You've seen how high throughput stream processing, state manipulation, and low-latency querying can be seamlessly intermixed via Trident's API. Trident lets you express your realtime computations in a natural way while still getting maximal performance. diff --git a/docs/Troubleshooting.md b/docs/Troubleshooting.md new file mode 100644 index 00000000000..c9df2984a04 --- /dev/null +++ b/docs/Troubleshooting.md @@ -0,0 +1,144 @@ +--- +layout: documentation +--- +## Troubleshooting + +This page lists issues people have run into when using Storm along with their solutions. + +### Worker processes are crashing on startup with no stack trace + +Possible symptoms: + + * Topologies work with one node, but workers crash with multiple nodes + +Solutions: + + * You may have a misconfigured subnet, where nodes can't locate other nodes based on their hostname. ZeroMQ sometimes crashes the process when it can't resolve a host. There are two solutions: + * Make a mapping from hostname to IP address in /etc/hosts + * Set up an internal DNS so that nodes can locate each other based on hostname. + +### Nodes are unable to communicate with each other + +Possible symptoms: + + * Every spout tuple is failing + * Processing is not working + +Solutions: + + * Storm doesn't work with ipv6. You can force ipv4 by adding `-Djava.net.preferIPv4Stack=true` to the supervisor child options and restarting the supervisor. + * You may have a misconfigured subnet. See the solutions for `Worker processes are crashing on startup with no stack trace` + +### Topology stops processing tuples after awhile + +Symptoms: + + * Processing works fine for awhile, and then suddenly stops and spout tuples start failing en masse. + +Solutions: + + * This is a known issue with ZeroMQ 2.1.10. Downgrade to ZeroMQ 2.1.7. + +### Not all supervisors appear in Storm UI + +Symptoms: + + * Some supervisor processes are missing from the Storm UI + * List of supervisors in Storm UI changes on refreshes + +Solutions: + + * Make sure the supervisor local dirs are independent (e.g., not sharing a local dir over NFS) + * Try deleting the local dirs for the supervisors and restarting the daemons. Supervisors create a unique id for themselves and store it locally. When that id is copied to other nodes, Storm gets confused. + +### "Multiple defaults.yaml found" error + +Symptoms: + + * When deploying a topology with "storm jar", you get this error + +Solution: + + * You're most likely including the Storm jars inside your topology jar. When packaging your topology jar, don't include the Storm jars as Storm will put those on the classpath for you. + +### "NoSuchMethodError" when running storm jar + +Symptoms: + + * When running storm jar, you get a cryptic "NoSuchMethodError" + +Solution: + + * You're deploying your topology with a different version of Storm than you built your topology against. Make sure the storm client you use comes from the same version as the version you compiled your topology against. + + +### Kryo ConcurrentModificationException + +Symptoms: + + * At runtime, you get a stack trace like the following: + +``` +java.lang.RuntimeException: java.util.ConcurrentModificationException + at backtype.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:84) + at backtype.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:55) + at backtype.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:56) + at backtype.storm.disruptor$consume_loop_STAR_$fn__1597.invoke(disruptor.clj:67) + at backtype.storm.util$async_loop$fn__465.invoke(util.clj:377) + at clojure.lang.AFn.run(AFn.java:24) + at java.lang.Thread.run(Thread.java:679) +Caused by: java.util.ConcurrentModificationException + at java.util.LinkedHashMap$LinkedHashIterator.nextEntry(LinkedHashMap.java:390) + at java.util.LinkedHashMap$EntryIterator.next(LinkedHashMap.java:409) + at java.util.LinkedHashMap$EntryIterator.next(LinkedHashMap.java:408) + at java.util.HashMap.writeObject(HashMap.java:1016) + at sun.reflect.GeneratedMethodAccessor17.invoke(Unknown Source) + at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) + at java.lang.reflect.Method.invoke(Method.java:616) + at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:959) + at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1480) + at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1416) + at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1174) + at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:346) + at backtype.storm.serialization.SerializableSerializer.write(SerializableSerializer.java:21) + at com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:554) + at com.esotericsoftware.kryo.serializers.CollectionSerializer.write(CollectionSerializer.java:77) + at com.esotericsoftware.kryo.serializers.CollectionSerializer.write(CollectionSerializer.java:18) + at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:472) + at backtype.storm.serialization.KryoValuesSerializer.serializeInto(KryoValuesSerializer.java:27) +``` + +Solution: + + * This means that you're emitting a mutable object as an output tuple. Everything you emit into the output collector must be immutable. What's happening is that your bolt is modifying the object while it is being serialized to be sent over the network. + + +### NullPointerException from deep inside Storm + +Symptoms: + + * You get a NullPointerException that looks something like: + +``` +java.lang.RuntimeException: java.lang.NullPointerException + at backtype.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:84) + at backtype.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:55) + at backtype.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:56) + at backtype.storm.disruptor$consume_loop_STAR_$fn__1596.invoke(disruptor.clj:67) + at backtype.storm.util$async_loop$fn__465.invoke(util.clj:377) + at clojure.lang.AFn.run(AFn.java:24) + at java.lang.Thread.run(Thread.java:662) +Caused by: java.lang.NullPointerException + at backtype.storm.serialization.KryoTupleSerializer.serialize(KryoTupleSerializer.java:24) + at backtype.storm.daemon.worker$mk_transfer_fn$fn__4126$fn__4130.invoke(worker.clj:99) + at backtype.storm.util$fast_list_map.invoke(util.clj:771) + at backtype.storm.daemon.worker$mk_transfer_fn$fn__4126.invoke(worker.clj:99) + at backtype.storm.daemon.executor$start_batch_transfer__GT_worker_handler_BANG_$fn__3904.invoke(executor.clj:205) + at backtype.storm.disruptor$clojure_handler$reify__1584.onEvent(disruptor.clj:43) + at backtype.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:81) + ... 6 more +``` + +Solution: + + * This is caused by having multiple threads issue methods on the `OutputCollector`. All emits, acks, and fails must happen on the same thread. One subtle way this can happen is if you make a `IBasicBolt` that emits on a separate thread. `IBasicBolt`'s automatically ack after execute is called, so this would cause multiple threads to use the `OutputCollector` leading to this exception. When using a basic bolt, all emits must happen in the same thread that runs `execute`. diff --git a/docs/Tutorial.md b/docs/Tutorial.md new file mode 100644 index 00000000000..73bf9a4a920 --- /dev/null +++ b/docs/Tutorial.md @@ -0,0 +1,310 @@ +--- +layout: documentation +--- +In this tutorial, you'll learn how to create Storm topologies and deploy them to a Storm cluster. Java will be the main language used, but a few examples will use Python to illustrate Storm's multi-language capabilities. + +## Preliminaries + +This tutorial uses examples from the [storm-starter](http://github.com/nathanmarz/storm-starter) project. It's recommended that you clone the project and follow along with the examples. Read [Setting up a development environment](Setting-up-development-environment.html) and [Creating a new Storm project](Creating-a-new-Storm-project.html) to get your machine set up. + +## Components of a Storm cluster + +A Storm cluster is superficially similar to a Hadoop cluster. Whereas on Hadoop you run "MapReduce jobs", on Storm you run "topologies". "Jobs" and "topologies" themselves are very different -- one key difference is that a MapReduce job eventually finishes, whereas a topology processes messages forever (or until you kill it). + +There are two kinds of nodes on a Storm cluster: the master node and the worker nodes. The master node runs a daemon called "Nimbus" that is similar to Hadoop's "JobTracker". Nimbus is responsible for distributing code around the cluster, assigning tasks to machines, and monitoring for failures. + +Each worker node runs a daemon called the "Supervisor". The supervisor listens for work assigned to its machine and starts and stops worker processes as necessary based on what Nimbus has assigned to it. Each worker process executes a subset of a topology; a running topology consists of many worker processes spread across many machines. + +![Storm cluster](images/storm-cluster.png) + +All coordination between Nimbus and the Supervisors is done through a [Zookeeper](http://zookeeper.apache.org/) cluster. Additionally, the Nimbus daemon and Supervisor daemons are fail-fast and stateless; all state is kept in Zookeeper or on local disk. This means you can kill -9 Nimbus or the Supervisors and they'll start back up like nothing happened. This design leads to Storm clusters being incredibly stable. + +## Topologies + +To do realtime computation on Storm, you create what are called "topologies". A topology is a graph of computation. Each node in a topology contains processing logic, and links between nodes indicate how data should be passed around between nodes. + +Running a topology is straightforward. First, you package all your code and dependencies into a single jar. Then, you run a command like the following: + +``` +storm jar all-my-code.jar backtype.storm.MyTopology arg1 arg2 +``` + +This runs the class `backtype.storm.MyTopology` with the arguments `arg1` and `arg2`. The main function of the class defines the topology and submits it to Nimbus. The `storm jar` part takes care of connecting to Nimbus and uploading the jar. + +Since topology definitions are just Thrift structs, and Nimbus is a Thrift service, you can create and submit topologies using any programming language. The above example is the easiest way to do it from a JVM-based language. See [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)] for more information on starting and stopping topologies. + +## Streams + +The core abstraction in Storm is the "stream". A stream is an unbounded sequence of tuples. Storm provides the primitives for transforming a stream into a new stream in a distributed and reliable way. For example, you may transform a stream of tweets into a stream of trending topics. + +The basic primitives Storm provides for doing stream transformations are "spouts" and "bolts". Spouts and bolts have interfaces that you implement to run your application-specific logic. + +A spout is a source of streams. For example, a spout may read tuples off of a [Kestrel](http://github.com/nathanmarz/storm-kestrel) queue and emit them as a stream. Or a spout may connect to the Twitter API and emit a stream of tweets. + +A bolt consumes any number of input streams, does some processing, and possibly emits new streams. Complex stream transformations, like computing a stream of trending topics from a stream of tweets, require multiple steps and thus multiple bolts. Bolts can do anything from run functions, filter tuples, do streaming aggregations, do streaming joins, talk to databases, and more. + +Networks of spouts and bolts are packaged into a "topology" which is the top-level abstraction that you submit to Storm clusters for execution. A topology is a graph of stream transformations where each node is a spout or bolt. Edges in the graph indicate which bolts are subscribing to which streams. When a spout or bolt emits a tuple to a stream, it sends the tuple to every bolt that subscribed to that stream. + +![A Storm topology](images/topology.png) + +Links between nodes in your topology indicate how tuples should be passed around. For example, if there is a link between Spout A and Bolt B, a link from Spout A to Bolt C, and a link from Bolt B to Bolt C, then everytime Spout A emits a tuple, it will send the tuple to both Bolt B and Bolt C. All of Bolt B's output tuples will go to Bolt C as well. + +Each node in a Storm topology executes in parallel. In your topology, you can specify how much parallelism you want for each node, and then Storm will spawn that number of threads across the cluster to do the execution. + +A topology runs forever, or until you kill it. Storm will automatically reassign any failed tasks. Additionally, Storm guarantees that there will be no data loss, even if machines go down and messages are dropped. + +## Data model + +Storm uses tuples as its data model. A tuple is a named list of values, and a field in a tuple can be an object of any type. Out of the box, Storm supports all the primitive types, strings, and byte arrays as tuple field values. To use an object of another type, you just need to implement [a serializer](Serialization.html) for the type. + +Every node in a topology must declare the output fields for the tuples it emits. For example, this bolt declares that it emits 2-tuples with the fields "double" and "triple": + +```java +public class DoubleAndTripleBolt extends BaseRichBolt { + private OutputCollectorBase _collector; + + @Override + public void prepare(Map conf, TopologyContext context, OutputCollectorBase collector) { + _collector = collector; + } + + @Override + public void execute(Tuple input) { + int val = input.getInteger(0); + _collector.emit(input, new Values(val*2, val*3)); + _collector.ack(input); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("double", "triple")); + } +} +``` + +The `declareOutputFields` function declares the output fields `["double", "triple"]` for the component. The rest of the bolt will be explained in the upcoming sections. + +## A simple topology + +Let's take a look at a simple topology to explore the concepts more and see how the code shapes up. Let's look at the `ExclamationTopology` definition from storm-starter: + +```java +TopologyBuilder builder = new TopologyBuilder(); +builder.setSpout("words", new TestWordSpout(), 10); +builder.setBolt("exclaim1", new ExclamationBolt(), 3) + .shuffleGrouping("words"); +builder.setBolt("exclaim2", new ExclamationBolt(), 2) + .shuffleGrouping("exclaim1"); +``` + +This topology contains a spout and two bolts. The spout emits words, and each bolt appends the string "!!!" to its input. The nodes are arranged in a line: the spout emits to the first bolt which then emits to the second bolt. If the spout emits the tuples ["bob"] and ["john"], then the second bolt will emit the words ["bob!!!!!!"] and ["john!!!!!!"]. + +This code defines the nodes using the `setSpout` and `setBolt` methods. These methods take as input a user-specified id, an object containing the processing logic, and the amount of parallelism you want for the node. In this example, the spout is given id "words" and the bolts are given ids "exclaim1" and "exclaim2". + +The object containing the processing logic implements the [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html) interface for spouts and the [IRichBolt](javadocs/backtype/storm/topology/IRichBolt.html) interface for bolts. + +The last parameter, how much parallelism you want for the node, is optional. It indicates how many threads should execute that component across the cluster. If you omit it, Storm will only allocate one thread for that node. + +`setBolt` returns an [InputDeclarer](javadocs/backtype/storm/topology/InputDeclarer.html) object that is used to define the inputs to the Bolt. Here, component "exclaim1" declares that it wants to read all the tuples emitted by component "words" using a shuffle grouping, and component "exclaim2" declares that it wants to read all the tuples emitted by component "exclaim1" using a shuffle grouping. "shuffle grouping" means that tuples should be randomly distributed from the input tasks to the bolt's tasks. There are many ways to group data between components. These will be explained in a few sections. + +If you wanted component "exclaim2" to read all the tuples emitted by both component "words" and component "exclaim1", you would write component "exclaim2"'s definition like this: + +```java +builder.setBolt("exclaim2", new ExclamationBolt(), 5) + .shuffleGrouping("words") + .shuffleGrouping("exclaim1"); +``` + +As you can see, input declarations can be chained to specify multiple sources for the Bolt. + +Let's dig into the implementations of the spouts and bolts in this topology. Spouts are responsible for emitting new messages into the topology. `TestWordSpout` in this topology emits a random word from the list ["nathan", "mike", "jackson", "golda", "bertels"] as a 1-tuple every 100ms. The implementation of `nextTuple()` in TestWordSpout looks like this: + +```java +public void nextTuple() { + Utils.sleep(100); + final String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"}; + final Random rand = new Random(); + final String word = words[rand.nextInt(words.length)]; + _collector.emit(new Values(word)); +} +``` + +As you can see, the implementation is very straightforward. + +`ExclamationBolt` appends the string "!!!" to its input. Let's take a look at the full implementation for `ExclamationBolt`: + +```java +public static class ExclamationBolt implements IRichBolt { + OutputCollector _collector; + + public void prepare(Map conf, TopologyContext context, OutputCollector collector) { + _collector = collector; + } + + public void execute(Tuple tuple) { + _collector.emit(tuple, new Values(tuple.getString(0) + "!!!")); + _collector.ack(tuple); + } + + public void cleanup() { + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + + public Map getComponentConfiguration() { + return null; + } +} +``` + +The `prepare` method provides the bolt with an `OutputCollector` that is used for emitting tuples from this bolt. Tuples can be emitted at anytime from the bolt -- in the `prepare`, `execute`, or `cleanup` methods, or even asynchronously in another thread. This `prepare` implementation simply saves the `OutputCollector` as an instance variable to be used later on in the `execute` method. + +The `execute` method receives a tuple from one of the bolt's inputs. The `ExclamationBolt` grabs the first field from the tuple and emits a new tuple with the string "!!!" appended to it. If you implement a bolt that subscribes to multiple input sources, you can find out which component the [Tuple](javadocs/backtype/storm/tuple/Tuple.html) came from by using the `Tuple#getSourceComponent` method. + +There's a few other things going in in the `execute` method, namely that the input tuple is passed as the first argument to `emit` and the input tuple is acked on the final line. These are part of Storm's reliability API for guaranteeing no data loss and will be explained later in this tutorial. + +The `cleanup` method is called when a Bolt is being shutdown and should cleanup any resources that were opened. There's no guarantee that this method will be called on the cluster: for example, if the machine the task is running on blows up, there's no way to invoke the method. The `cleanup` method is intended for when you run topologies in [local mode](Local-mode.html) (where a Storm cluster is simulated in process), and you want to be able to run and kill many topologies without suffering any resource leaks. + +The `declareOutputFields` method declares that the `ExclamationBolt` emits 1-tuples with one field called "word". + +The `getComponentConfiguration` method allows you to configure various aspects of how this component runs. This is a more advanced topic that is explained further on [Configuration](Configuration.html). + +Methods like `cleanup` and `getComponentConfiguration` are often not needed in a bolt implementation. You can define bolts more succinctly by using a base class that provides default implementations where appropriate. `ExclamationBolt` can be written more succinctly by extending `BaseRichBolt`, like so: + +```java +public static class ExclamationBolt extends BaseRichBolt { + OutputCollector _collector; + + public void prepare(Map conf, TopologyContext context, OutputCollector collector) { + _collector = collector; + } + + public void execute(Tuple tuple) { + _collector.emit(tuple, new Values(tuple.getString(0) + "!!!")); + _collector.ack(tuple); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } +} +``` + +## Running ExclamationTopology in local mode + +Let's see how to run the `ExclamationTopology` in local mode and see that it's working. + +Storm has two modes of operation: local mode and distributed mode. In local mode, Storm executes completely in process by simulating worker nodes with threads. Local mode is useful for testing and development of topologies. When you run the topologies in storm-starter, they'll run in local mode and you'll be able to see what messages each component is emitting. You can read more about running topologies in local mode on [Local mode](Local-mode.html). + +In distributed mode, Storm operates as a cluster of machines. When you submit a topology to the master, you also submit all the code necessary to run the topology. The master will take care of distributing your code and allocating workers to run your topology. If workers go down, the master will reassign them somewhere else. You can read more about running topologies on a cluster on [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)]. + +Here's the code that runs `ExclamationTopology` in local mode: + +```java +Config conf = new Config(); +conf.setDebug(true); +conf.setNumWorkers(2); + +LocalCluster cluster = new LocalCluster(); +cluster.submitTopology("test", conf, builder.createTopology()); +Utils.sleep(10000); +cluster.killTopology("test"); +cluster.shutdown(); +``` + +First, the code defines an in-process cluster by creating a `LocalCluster` object. Submitting topologies to this virtual cluster is identical to submitting topologies to distributed clusters. It submits a topology to the `LocalCluster` by calling `submitTopology`, which takes as arguments a name for the running topology, a configuration for the topology, and then the topology itself. + +The name is used to identify the topology so that you can kill it later on. A topology will run indefinitely until you kill it. + +The configuration is used to tune various aspects of the running topology. The two configurations specified here are very common: + +1. **TOPOLOGY_WORKERS** (set with `setNumWorkers`) specifies how many _processes_ you want allocated around the cluster to execute the topology. Each component in the topology will execute as many _threads_. The number of threads allocated to a given component is configured through the `setBolt` and `setSpout` methods. Those _threads_ exist within worker _processes_. Each worker _process_ contains within it some number of _threads_ for some number of components. For instance, you may have 300 threads specified across all your components and 50 worker processes specified in your config. Each worker process will execute 6 threads, each of which of could belong to a different component. You tune the performance of Storm topologies by tweaking the parallelism for each component and the number of worker processes those threads should run within. +2. **TOPOLOGY_DEBUG** (set with `setDebug`), when set to true, tells Storm to log every message every emitted by a component. This is useful in local mode when testing topologies, but you probably want to keep this turned off when running topologies on the cluster. + +There's many other configurations you can set for the topology. The various configurations are detailed on [the Javadoc for Config](javadocs/backtype/storm/Config.html). + +To learn about how to set up your development environment so that you can run topologies in local mode (such as in Eclipse), see [Creating a new Storm project](Creating-a-new-Storm-project.html). + +## Stream groupings + +A stream grouping tells a topology how to send tuples between two components. Remember, spouts and bolts execute in parallel as many tasks across the cluster. If you look at how a topology is executing at the task level, it looks something like this: + +![Tasks in a topology](images/topology-tasks.png) + +When a task for Bolt A emits a tuple to Bolt B, which task should it send the tuple to? + +A "stream grouping" answers this question by telling Storm how to send tuples between sets of tasks. Before we dig into the different kinds of stream groupings, let's take a look at another topology from [storm-starter](http://github.com/nathanmarz/storm-starter). This [WordCountTopology](https://github.com/nathanmarz/storm-starter/blob/master/src/jvm/storm/starter/WordCountTopology.java) reads sentences off of a spout and streams out of `WordCountBolt` the total number of times it has seen that word before: + +```java +TopologyBuilder builder = new TopologyBuilder(); + +builder.setSpout("sentences", new RandomSentenceSpout(), 5); +builder.setBolt("split", new SplitSentence(), 8) + .shuffleGrouping("sentences"); +builder.setBolt("count", new WordCount(), 12) + .fieldsGrouping("split", new Fields("word")); +``` + +`SplitSentence` emits a tuple for each word in each sentence it receives, and `WordCount` keeps a map in memory from word to count. Each time `WordCount` receives a word, it updates its state and emits the new word count. + +There's a few different kinds of stream groupings. + +The simplest kind of grouping is called a "shuffle grouping" which sends the tuple to a random task. A shuffle grouping is used in the `WordCountTopology` to send tuples from `RandomSentenceSpout` to the `SplitSentence` bolt. It has the effect of evenly distributing the work of processing the tuples across all of `SplitSentence` bolt's tasks. + +A more interesting kind of grouping is the "fields grouping". A fields grouping is used between the `SplitSentence` bolt and the `WordCount` bolt. It is critical for the functioning of the `WordCount` bolt that the same word always go to the same task. Otherwise, more than one task will see the same word, and they'll each emit incorrect values for the count since each has incomplete information. A fields grouping lets you group a stream by a subset of its fields. This causes equal values for that subset of fields to go to the same task. Since `WordCount` subscribes to `SplitSentence`'s output stream using a fields grouping on the "word" field, the same word always goes to the same task and the bolt produces the correct output. + +Fields groupings are the basis of implementing streaming joins and streaming aggregations as well as a plethora of other use cases. Underneath the hood, fields groupings are implemented using mod hashing. + +There's a few other kinds of stream groupings. You can read more about them on [Concepts](Concepts.html). + +## Defining Bolts in other languages + +Bolts can be defined in any language. Bolts written in another language are executed as subprocesses, and Storm communicates with those subprocesses with JSON messages over stdin/stdout. The communication protocol just requires an ~100 line adapter library, and Storm ships with adapter libraries for Ruby, Python, and Fancy. + +Here's the definition of the `SplitSentence` bolt from `WordCountTopology`: + +```java +public static class SplitSentence extends ShellBolt implements IRichBolt { + public SplitSentence() { + super("python", "splitsentence.py"); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } +} +``` + +`SplitSentence` overrides `ShellBolt` and declares it as running using `python` with the arguments `splitsentence.py`. Here's the implementation of `splitsentence.py`: + +```python +import storm + +class SplitSentenceBolt(storm.BasicBolt): + def process(self, tup): + words = tup.values[0].split(" ") + for word in words: + storm.emit([word]) + +SplitSentenceBolt().run() +``` + +For more information on writing spouts and bolts in other languages, and to learn about how to create topologies in other languages (and avoid the JVM completely), see [Using non-JVM languages with Storm](Using-non-JVM-languages-with-Storm.html). + +## Guaranteeing message processing + +Earlier on in this tutorial, we skipped over a few aspects of how tuples are emitted. Those aspects were part of Storm's reliability API: how Storm guarantees that every message coming off a spout will be fully processed. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for information on how this works and what you have to do as a user to take advantage of Storm's reliability capabilities. + +## Transactional topologies + +Storm guarantees that every message will be played through the topology at least once. A common question asked is "how do you do things like counting on top of Storm? Won't you overcount?" Storm has a feature called transactional topologies that let you achieve exactly-once messaging semantics for most computations. Read more about transactional topologies [here](Transactional-topologies.html). + +## Distributed RPC + +This tutorial showed how to do basic stream processing on top of Storm. There's lots more things you can do with Storm's primitives. One of the most interesting applications of Storm is Distributed RPC, where you parallelize the computation of intense functions on the fly. Read more about Distributed RPC [here](Distributed-RPC.html). + +## Conclusion + +This tutorial gave a broad overview of developing, testing, and deploying Storm topologies. The rest of the documentation dives deeper into all the aspects of using Storm. diff --git a/docs/Understanding-the-parallelism-of-a-Storm-topology.md b/docs/Understanding-the-parallelism-of-a-Storm-topology.md new file mode 100644 index 00000000000..adc4c41a6d6 --- /dev/null +++ b/docs/Understanding-the-parallelism-of-a-Storm-topology.md @@ -0,0 +1,121 @@ +--- +layout: documentation +--- +# What makes a running topology: worker processes, executors and tasks + +Storm distinguishes between the following three main entities that are used to actually run a topology in a Storm cluster: + +1. Worker processes +2. Executors (threads) +3. Tasks + +Here is a simple illustration of their relationships: + +![The relationships of worker processes, executors (threads) and tasks in Storm](images/relationships-worker-processes-executors-tasks.png) + +A _worker process_ executes a subset of a topology. A worker process belongs to a specific topology and may run one or more executors for one or more components (spouts or bolts) of this topology. A running topology consists of many such processes running on many machines within a Storm cluster. + +An _executor_ is a thread that is spawned by a worker process. It may run one or more tasks for the same component (spout or bolt). + +A _task_ performs the actual data processing — each spout or bolt that you implement in your code executes as many tasks across the cluster. The number of tasks for a component is always the same throughout the lifetime of a topology, but the number of executors (threads) for a component can change over time. This means that the following condition holds true: ``#threads ≤ #tasks``. By default, the number of tasks is set to be the same as the number of executors, i.e. Storm will run one task per thread. + +# Configuring the parallelism of a topology + +Note that in Storm’s terminology "parallelism" is specifically used to describe the so-called _parallelism hint_, which means the initial number of executor (threads) of a component. In this document though we use the term "parallelism" in a more general sense to describe how you can configure not only the number of executors but also the number of worker processes and the number of tasks of a Storm topology. We will specifically call out when "parallelism" is used in the normal, narrow definition of Storm. + +The following sections give an overview of the various configuration options and how to set them in your code. There is more than one way of setting these options though, and the table lists only some of them. Storm currently has the following [order of precedence for configuration settings](Configuration.html): ``defaults.yaml`` < ``storm.yaml`` < topology-specific configuration < internal component-specific configuration < external component-specific configuration. + +## Number of worker processes + +* Description: How many worker processes to create _for the topology_ across machines in the cluster. +* Configuration option: [TOPOLOGY_WORKERS](javadocs/backtype/storm/Config.html#TOPOLOGY_WORKERS) +* How to set in your code (examples): + * [Config#setNumWorkers](javadocs/backtype/storm/Config.html) + +## Number of executors (threads) + +* Description: How many executors to spawn _per component_. +* Configuration option: ? +* How to set in your code (examples): + * [TopologyBuilder#setSpout()](javadocs/backtype/storm/topology/TopologyBuilder.html) + * [TopologyBuilder#setBolt()](javadocs/backtype/storm/topology/TopologyBuilder.html) + * Note that as of Storm 0.8 the ``parallelism_hint`` parameter now specifies the initial number of executors (not tasks!) for that bolt. + +## Number of tasks + +* Description: How many tasks to create _per component_. +* Configuration option: [TOPOLOGY_TASKS](javadocs/backtype/storm/Config.html#TOPOLOGY_TASKS) +* How to set in your code (examples): + * [ComponentConfigurationDeclarer#setNumTasks()](javadocs/backtype/storm/topology/ComponentConfigurationDeclarer.html) + + +Here is an example code snippet to show these settings in practice: + +```java +topologyBuilder.setBolt("green-bolt", new GreenBolt(), 2) + .setNumTasks(4) + .shuffleGrouping("blue-spout); +``` + +In the above code we configured Storm to run the bolt ``GreenBolt`` with an initial number of two executors and four associated tasks. Storm will run two tasks per executor (thread). If you do not explicitly configure the number of tasks, Storm will run by default one task per executor. + +# Example of a running topology + +The following illustration shows how a simple topology would look like in operation. The topology consists of three components: one spout called ``BlueSpout`` and two bolts called ``GreenBolt`` and ``YellowBolt``. The components are linked such that ``BlueSpout`` sends its output to ``GreenBolt``, which in turns sends its own output to ``YellowBolt``. + +![Example of a running topology in Storm](images/example-of-a-running-topology.png) + +The ``GreenBolt`` was configured as per the code snippet above whereas ``BlueSpout`` and ``YellowBolt`` only set the parallelism hint (number of executors). Here is the relevant code: + +```java +Config conf = new Config(); +conf.setNumWorkers(2); // use two worker processes + +topologyBuilder.setSpout("blue-spout", new BlueSpout(), 2); // set parallelism hint to 2 + +topologyBuilder.setBolt("green-bolt", new GreenBolt(), 2) + .setNumTasks(4) + .shuffleGrouping("blue-spout"); + +topologyBuilder.setBolt("yellow-bolt", new YellowBolt(), 6) + .shuffleGrouping("green-bolt"); + +StormSubmitter.submitTopology( + "mytopology", + conf, + topologyBuilder.createTopology() + ); +``` + +And of course Storm comes with additional configuration settings to control the parallelism of a topology, including: + +* [TOPOLOGY_MAX_TASK_PARALLELISM](javadocs/backtype/storm/Config.html#TOPOLOGY_MAX_TASK_PARALLELISM): This setting puts a ceiling on the number of executors that can be spawned for a single component. It is typically used during testing to limit the number of threads spawned when running a topology in local mode. You can set this option via e.g. [Config#setMaxTaskParallelism()](javadocs/backtype/storm/Config.html). + +# How to change the parallelism of a running topology + +A nifty feature of Storm is that you can increase or decrease the number of worker processes and/or executors without being required to restart the cluster or the topology. The act of doing so is called rebalancing. + +You have two options to rebalance a topology: + +1. Use the Storm web UI to rebalance the topology. +2. Use the CLI tool storm rebalance as described below. + +Here is an example of using the CLI tool: + +``` +# Reconfigure the topology "mytopology" to use 5 worker processes, +# the spout "blue-spout" to use 3 executors and +# the bolt "yellow-bolt" to use 10 executors. + +$ storm rebalance mytopology -n 5 -e blue-spout=3 -e yellow-bolt=10 +``` + +# References for this article + +* [Concepts](Concepts.html) +* [Configuration](Configuration.html) +* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)] +* [Local mode](Local-mode.html) +* [Tutorial](Tutorial.html) +* [Storm API documentation](javadocs/), most notably the class ``Config`` + diff --git a/docs/Using-non-JVM-languages-with-Storm.md b/docs/Using-non-JVM-languages-with-Storm.md new file mode 100644 index 00000000000..7b2a2f20d33 --- /dev/null +++ b/docs/Using-non-JVM-languages-with-Storm.md @@ -0,0 +1,52 @@ +--- +layout: documentation +--- +- two pieces: creating topologies and implementing spouts and bolts in other languages +- creating topologies in another language is easy since topologies are just thrift structures (link to storm.thrift) +- implementing spouts and bolts in another language is called a "multilang components" or "shelling" + - Here's a specification of the protocol: [Multilang protocol](Multilang-protocol.html) + - the thrift structure lets you define multilang components explicitly as a program and a script (e.g., python and the file implementing your bolt) + - In Java, you override ShellBolt or ShellSpout to create multilang components + - note that output fields declarations happens in the thrift structure, so in Java you create multilang components like the following: + - declare fields in java, processing code in the other language by specifying it in constructor of shellbolt + - multilang uses json messages over stdin/stdout to communicate with the subprocess + - storm comes with ruby, python, and fancy adapters that implement the protocol. show an example of python + - python supports emitting, anchoring, acking, and logging +- "storm shell" command makes constructing jar and uploading to nimbus easy + - makes jar and uploads it + - calls your program with host/port of nimbus and the jarfile id + +## Notes on implementing a DSL in a non-JVM language + +The right place to start is src/storm.thrift. Since Storm topologies are just Thrift structures, and Nimbus is a Thrift daemon, you can create and submit topologies in any language. + +When you create the Thrift structs for spouts and bolts, the code for the spout or bolt is specified in the ComponentObject struct: + +``` +union ComponentObject { + 1: binary serialized_java; + 2: ShellComponent shell; + 3: JavaObject java_object; +} +``` + +For a non-JVM DSL, you would want to make use of "2" and "3". ShellComponent lets you specify a script to run that component (e.g., your python code). And JavaObject lets you specify native java spouts and bolts for the component (and Storm will use reflection to create that spout or bolt). + +There's a "storm shell" command that will help with submitting a topology. Its usage is like this: + +``` +storm shell resources/ python topology.py arg1 arg2 +``` + +storm shell will then package resources/ into a jar, upload the jar to Nimbus, and call your topology.py script like this: + +``` +python topology.py arg1 arg2 {nimbus-host} {nimbus-port} {uploaded-jar-location} +``` + +Then you can connect to Nimbus using the Thrift API and submit the topology, passing {uploaded-jar-location} into the submitTopology method. For reference, here's the submitTopology definition: + +``` +void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) + throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite); +``` diff --git a/docs/_config.yml b/docs/_config.yml new file mode 100644 index 00000000000..b05bcefda67 --- /dev/null +++ b/docs/_config.yml @@ -0,0 +1,18 @@ +# Site settings +title: Apache Storm +baseurl: "" # the subpath of your site, e.g. /blog/ +url: "http://storm.apache.org" # the base hostname & protocol for your site +twitter_username: stormprocessor +github_username: apache/storm + +# Build settings +markdown: redcarpet +redcarpet: + extensions: ["no_intra_emphasis", "fenced_code_blocks", "autolink", "tables", "with_toc_data"] + +keep_files: [".git", ".svn"] +encoding: "utf-8" +exclude: + - READEME.md + +storm_release_only: true diff --git a/docs/_includes/footer.html b/docs/_includes/footer.html new file mode 100644 index 00000000000..1696720844a --- /dev/null +++ b/docs/_includes/footer.html @@ -0,0 +1,55 @@ +
+
+
+
+ +
+
+ +
+ +
+ +
+
+
+
+
+

Copyright © 2015 Apache Software Foundation. All Rights Reserved. +
Apache Storm, Apache, the Apache feather logo, and the Apache Storm project logos are trademarks of The Apache Software Foundation. +
All other marks mentioned may be trademarks or registered trademarks of their respective owners.

+
+
+
+
+ + + diff --git a/docs/_includes/head.html b/docs/_includes/head.html new file mode 100644 index 00000000000..8f51c94343a --- /dev/null +++ b/docs/_includes/head.html @@ -0,0 +1,34 @@ + + + + + + + + + {% if page.title %}{{ page.title }}{% else %}{{ site.title }}{% endif %} + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/_includes/header.html b/docs/_includes/header.html new file mode 100644 index 00000000000..b9a2b03d0d1 --- /dev/null +++ b/docs/_includes/header.html @@ -0,0 +1,59 @@ +
+
+
+
+ +
+
+ {% if page.version %} +

Version: {{page.version}}

+ {% endif %} +
+
+ Download +
+
+
+
+ + + + + diff --git a/docs/_layouts/about.html b/docs/_layouts/about.html new file mode 100644 index 00000000000..7ca3e79363b --- /dev/null +++ b/docs/_layouts/about.html @@ -0,0 +1,43 @@ +--- +layout: default +title: Project Information +items: + - + - "/about/simple-api.html" + - "Simple API" + - + - "/about/scalable.html" + - "Scalable" + - + - "/about/fault-tolerant.html" + - "Fault tolerant" + - + - "/about/guarantees-data-processing.html" + - "Guarantees data processing" + - + - "/about/multi-language.html" + - "Use with any language" + - + - "/about/deployment.html" + - "Easy to deploy and operate" + - + - "/about/free-and-open-source.html" + - "Free and open source" +--- +
+
+
+
    + {% for post in page.items %} +
  • + {{ post[1] }} +
  • + {% endfor %} +
+
+
+ {{ content }} + +
+
+
\ No newline at end of file diff --git a/docs/_layouts/default.html b/docs/_layouts/default.html new file mode 100644 index 00000000000..80b404e9f6b --- /dev/null +++ b/docs/_layouts/default.html @@ -0,0 +1,18 @@ + + + {% include head.html %} + + {% include header.html %} +
+

{{ page.title }}

+
+
+ {{ content }} +
+
+
+{% include footer.html %} + + + + diff --git a/docs/_layouts/documentation.html b/docs/_layouts/documentation.html new file mode 100644 index 00000000000..81cc09fa74a --- /dev/null +++ b/docs/_layouts/documentation.html @@ -0,0 +1,9 @@ +--- +layout: default +--- + + + + +{{ content }} + diff --git a/docs/_layouts/page.html b/docs/_layouts/page.html new file mode 100644 index 00000000000..e230861d3db --- /dev/null +++ b/docs/_layouts/page.html @@ -0,0 +1,5 @@ +--- +layout: default +--- + {{ content }} + diff --git a/docs/_layouts/post.html b/docs/_layouts/post.html new file mode 100644 index 00000000000..5080868a103 --- /dev/null +++ b/docs/_layouts/post.html @@ -0,0 +1,61 @@ + + + + {% include head.html %} + + + + {% include header.html %} +
+
+
+
+
+ +
+
+

+ {{ page.title }} +

+ +
+
+

Posted on {{ page.date | date: "%b %-d, %Y" }}{% if page.author %} by {{ page.author }}{% endif %}{% if page.meta %} • {{ page.meta }}{% endif %}

+
+ + +
+
+
+
+ {{ content }} +
+
+
+
+
+
+ {% include footer.html %} + + + + diff --git a/docs/_plugins/releases.rb b/docs/_plugins/releases.rb new file mode 100644 index 00000000000..f28ccd2db9e --- /dev/null +++ b/docs/_plugins/releases.rb @@ -0,0 +1,84 @@ +module Releases + class Generator < Jekyll::Generator + def dir_to_releasename(dir) + ret = nil + splitdir = dir.split("/").select{ |a| a != ""}; + if (splitdir[0] == 'releases') + ret = splitdir[1] + if (ret == 'current') + ret = File.readlink(splitdir.join("/")).split("/")[-1] + end + end + return ret + end + + def set_if_unset(hash, key, value) + hash[key] = hash[key] || value; + end + + def parse_version(version_string) + return version_string.split('.').map{|e| e.to_i} + end + + def release_from_pom() + text= `mvn -f ../pom.xml help:evaluate -Dexpression=project.version` + return text.split("\n").select{|a| !a.start_with?('[')}[0] + end + + def branch_from_git() + return `git rev-parse --abbrev-ref HEAD` + end + + def generate(site) + if site.config['storm_release_only'] + release_name = release_from_pom() + puts "release: #{release_name}" + git_branch = branch_from_git() + puts "branch: #{git_branch}" + for page in site.pages do + page.data['version'] = release_name; + page.data['git-tree-base'] = "http://github.com/apache/storm/tree/#{git_branch}" + page.data['git-blob-base'] = "http://github.com/apache/storm/blob/#{git_branch}" + end + return + end + + releases = Hash.new + if (site.data['releases']) + for rel_data in site.data['releases'] do + releases[rel_data['name']] = rel_data + end + end + + for page in site.pages do + release_name = dir_to_releasename(page.dir) + if (release_name != nil) + if !releases.has_key?(release_name) + releases[release_name] = {'name' => release_name}; + end + releases[release_name]['documented'] = true + end + end + + releases.each { |release_name, release_data| + set_if_unset(release_data, 'git-tag-or-branch', "v#{release_data['name']}") + set_if_unset(release_data, 'git-tree-base', "http://github.com/apache/storm/tree/#{release_data['git-tag-or-branch']}") + set_if_unset(release_data, 'git-blob-base', "http://github.com/apache/storm/blob/#{release_data['git-tag-or-branch']}") + set_if_unset(release_data, 'base-name', "apache-storm-#{release_data['name']}") + set_if_unset(release_data, 'has-download', !release_name.end_with?('-SNAPSHOT')) + } + + for page in site.pages do + release_name = dir_to_releasename(page.dir) + if (release_name != nil) + release_data = releases[release_name] + page.data['version'] = release_name; + page.data['git-tree-base'] = release_data['git-tree-base']; + page.data['git-blob-base'] = release_data['git-blob-base']; + end + end + site.data['releases'] = releases.values.sort{|x,y| parse_version(y['name']) <=> + parse_version(x['name'])}; + end + end +end diff --git a/docs/assets/css/bootstrap.css b/docs/assets/css/bootstrap.css new file mode 100644 index 00000000000..680e7687862 --- /dev/null +++ b/docs/assets/css/bootstrap.css @@ -0,0 +1,6800 @@ +/*! + * Bootstrap v3.3.5 (http://getbootstrap.com) + * Copyright 2011-2015 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */ +/*! normalize.css v3.0.3 | MIT License | github.com/necolas/normalize.css */ +html { + font-family: sans-serif; + -webkit-text-size-adjust: 100%; + -ms-text-size-adjust: 100%; +} +body { + margin: 0; +} +article, +aside, +details, +figcaption, +figure, +footer, +header, +hgroup, +main, +menu, +nav, +section, +summary { + display: block; +} +audio, +canvas, +progress, +video { + display: inline-block; + vertical-align: baseline; +} +audio:not([controls]) { + display: none; + height: 0; +} +[hidden], +template { + display: none; +} +a { + background-color: transparent; +} +a:active, +a:hover { + outline: 0; +} +abbr[title] { + border-bottom: 1px dotted; +} +b, +strong { + font-weight: bold; +} +dfn { + font-style: italic; +} +h1 { + margin: .67em 0; + font-size: 2em; +} +mark { + color: #000; + background: #ff0; +} +small { + font-size: 80%; +} +sub, +sup { + position: relative; + font-size: 75%; + line-height: 0; + vertical-align: baseline; +} +sup { + top: -.5em; +} +sub { + bottom: -.25em; +} +img { + border: 0; +} +svg:not(:root) { + overflow: hidden; +} +figure { + margin: 1em 40px; +} +hr { + height: 0; + -webkit-box-sizing: content-box; + -moz-box-sizing: content-box; + box-sizing: content-box; +} +pre { + overflow: auto; +} +code, +kbd, +pre, +samp { + font-family: monospace, monospace; + font-size: 1em; +} +button, +input, +optgroup, +select, +textarea { + margin: 0; + font: inherit; + color: inherit; +} +button { + overflow: visible; +} +button, +select { + text-transform: none; +} +button, +html input[type="button"], +input[type="reset"], +input[type="submit"] { + -webkit-appearance: button; + cursor: pointer; +} +button[disabled], +html input[disabled] { + cursor: default; +} +button::-moz-focus-inner, +input::-moz-focus-inner { + padding: 0; + border: 0; +} +input { + line-height: normal; +} +input[type="checkbox"], +input[type="radio"] { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; + padding: 0; +} +input[type="number"]::-webkit-inner-spin-button, +input[type="number"]::-webkit-outer-spin-button { + height: auto; +} +input[type="search"] { + -webkit-box-sizing: content-box; + -moz-box-sizing: content-box; + box-sizing: content-box; + -webkit-appearance: textfield; +} +input[type="search"]::-webkit-search-cancel-button, +input[type="search"]::-webkit-search-decoration { + -webkit-appearance: none; +} +fieldset { + padding: .35em .625em .75em; + margin: 0 2px; + border: 1px solid #c0c0c0; +} +legend { + padding: 0; + border: 0; +} +textarea { + overflow: auto; +} +optgroup { + font-weight: bold; +} +table { + border-spacing: 0; + border-collapse: collapse; +} +td, +th { + padding: 0; +} +/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */ +@media print { + *, + *:before, + *:after { + color: #000 !important; + text-shadow: none !important; + background: transparent !important; + -webkit-box-shadow: none !important; + box-shadow: none !important; + } + a, + a:visited { + text-decoration: underline; + } + a[href]:after { + content: " (" attr(href) ")"; + } + abbr[title]:after { + content: " (" attr(title) ")"; + } + a[href^="#"]:after, + a[href^="javascript:"]:after { + content: ""; + } + pre, + blockquote { + border: 1px solid #999; + + page-break-inside: avoid; + } + thead { + display: table-header-group; + } + tr, + img { + page-break-inside: avoid; + } + img { + max-width: 100% !important; + } + p, + h2, + h3 { + orphans: 3; + widows: 3; + } + h2, + h3 { + page-break-after: avoid; + } + .navbar { + display: none; + } + .btn > .caret, + .dropup > .btn > .caret { + border-top-color: #000 !important; + } + .label { + border: 1px solid #000; + } + .table { + border-collapse: collapse !important; + } + .table td, + .table th { + background-color: #fff !important; + } + .table-bordered th, + .table-bordered td { + border: 1px solid #ddd !important; + } +} +@font-face { + font-family: 'Glyphicons Halflings'; + + src: url('../fonts/glyphicons-halflings-regular.eot'); + src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff2') format('woff2'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg'); +} +.glyphicon { + position: relative; + top: 1px; + display: inline-block; + font-family: 'Glyphicons Halflings'; + font-style: normal; + font-weight: normal; + line-height: 1; + + -webkit-font-smoothing: antialiased; + -moz-osx-font-smoothing: grayscale; +} +.glyphicon-asterisk:before { + content: "\2a"; +} +.glyphicon-plus:before { + content: "\2b"; +} +.glyphicon-euro:before, +.glyphicon-eur:before { + content: "\20ac"; +} +.glyphicon-minus:before { + content: "\2212"; +} +.glyphicon-cloud:before { + content: "\2601"; +} +.glyphicon-envelope:before { + content: "\2709"; +} +.glyphicon-pencil:before { + content: "\270f"; +} +.glyphicon-glass:before { + content: "\e001"; +} +.glyphicon-music:before { + content: "\e002"; +} +.glyphicon-search:before { + content: "\e003"; +} +.glyphicon-heart:before { + content: "\e005"; +} +.glyphicon-star:before { + content: "\e006"; +} +.glyphicon-star-empty:before { + content: "\e007"; +} +.glyphicon-user:before { + content: "\e008"; +} +.glyphicon-film:before { + content: "\e009"; +} +.glyphicon-th-large:before { + content: "\e010"; +} +.glyphicon-th:before { + content: "\e011"; +} +.glyphicon-th-list:before { + content: "\e012"; +} +.glyphicon-ok:before { + content: "\e013"; +} +.glyphicon-remove:before { + content: "\e014"; +} +.glyphicon-zoom-in:before { + content: "\e015"; +} +.glyphicon-zoom-out:before { + content: "\e016"; +} +.glyphicon-off:before { + content: "\e017"; +} +.glyphicon-signal:before { + content: "\e018"; +} +.glyphicon-cog:before { + content: "\e019"; +} +.glyphicon-trash:before { + content: "\e020"; +} +.glyphicon-home:before { + content: "\e021"; +} +.glyphicon-file:before { + content: "\e022"; +} +.glyphicon-time:before { + content: "\e023"; +} +.glyphicon-road:before { + content: "\e024"; +} +.glyphicon-download-alt:before { + content: "\e025"; +} +.glyphicon-download:before { + content: "\e026"; +} +.glyphicon-upload:before { + content: "\e027"; +} +.glyphicon-inbox:before { + content: "\e028"; +} +.glyphicon-play-circle:before { + content: "\e029"; +} +.glyphicon-repeat:before { + content: "\e030"; +} +.glyphicon-refresh:before { + content: "\e031"; +} +.glyphicon-list-alt:before { + content: "\e032"; +} +.glyphicon-lock:before { + content: "\e033"; +} +.glyphicon-flag:before { + content: "\e034"; +} +.glyphicon-headphones:before { + content: "\e035"; +} +.glyphicon-volume-off:before { + content: "\e036"; +} +.glyphicon-volume-down:before { + content: "\e037"; +} +.glyphicon-volume-up:before { + content: "\e038"; +} +.glyphicon-qrcode:before { + content: "\e039"; +} +.glyphicon-barcode:before { + content: "\e040"; +} +.glyphicon-tag:before { + content: "\e041"; +} +.glyphicon-tags:before { + content: "\e042"; +} +.glyphicon-book:before { + content: "\e043"; +} +.glyphicon-bookmark:before { + content: "\e044"; +} +.glyphicon-print:before { + content: "\e045"; +} +.glyphicon-camera:before { + content: "\e046"; +} +.glyphicon-font:before { + content: "\e047"; +} +.glyphicon-bold:before { + content: "\e048"; +} +.glyphicon-italic:before { + content: "\e049"; +} +.glyphicon-text-height:before { + content: "\e050"; +} +.glyphicon-text-width:before { + content: "\e051"; +} +.glyphicon-align-left:before { + content: "\e052"; +} +.glyphicon-align-center:before { + content: "\e053"; +} +.glyphicon-align-right:before { + content: "\e054"; +} +.glyphicon-align-justify:before { + content: "\e055"; +} +.glyphicon-list:before { + content: "\e056"; +} +.glyphicon-indent-left:before { + content: "\e057"; +} +.glyphicon-indent-right:before { + content: "\e058"; +} +.glyphicon-facetime-video:before { + content: "\e059"; +} +.glyphicon-picture:before { + content: "\e060"; +} +.glyphicon-map-marker:before { + content: "\e062"; +} +.glyphicon-adjust:before { + content: "\e063"; +} +.glyphicon-tint:before { + content: "\e064"; +} +.glyphicon-edit:before { + content: "\e065"; +} +.glyphicon-share:before { + content: "\e066"; +} +.glyphicon-check:before { + content: "\e067"; +} +.glyphicon-move:before { + content: "\e068"; +} +.glyphicon-step-backward:before { + content: "\e069"; +} +.glyphicon-fast-backward:before { + content: "\e070"; +} +.glyphicon-backward:before { + content: "\e071"; +} +.glyphicon-play:before { + content: "\e072"; +} +.glyphicon-pause:before { + content: "\e073"; +} +.glyphicon-stop:before { + content: "\e074"; +} +.glyphicon-forward:before { + content: "\e075"; +} +.glyphicon-fast-forward:before { + content: "\e076"; +} +.glyphicon-step-forward:before { + content: "\e077"; +} +.glyphicon-eject:before { + content: "\e078"; +} +.glyphicon-chevron-left:before { + content: "\e079"; +} +.glyphicon-chevron-right:before { + content: "\e080"; +} +.glyphicon-plus-sign:before { + content: "\e081"; +} +.glyphicon-minus-sign:before { + content: "\e082"; +} +.glyphicon-remove-sign:before { + content: "\e083"; +} +.glyphicon-ok-sign:before { + content: "\e084"; +} +.glyphicon-question-sign:before { + content: "\e085"; +} +.glyphicon-info-sign:before { + content: "\e086"; +} +.glyphicon-screenshot:before { + content: "\e087"; +} +.glyphicon-remove-circle:before { + content: "\e088"; +} +.glyphicon-ok-circle:before { + content: "\e089"; +} +.glyphicon-ban-circle:before { + content: "\e090"; +} +.glyphicon-arrow-left:before { + content: "\e091"; +} +.glyphicon-arrow-right:before { + content: "\e092"; +} +.glyphicon-arrow-up:before { + content: "\e093"; +} +.glyphicon-arrow-down:before { + content: "\e094"; +} +.glyphicon-share-alt:before { + content: "\e095"; +} +.glyphicon-resize-full:before { + content: "\e096"; +} +.glyphicon-resize-small:before { + content: "\e097"; +} +.glyphicon-exclamation-sign:before { + content: "\e101"; +} +.glyphicon-gift:before { + content: "\e102"; +} +.glyphicon-leaf:before { + content: "\e103"; +} +.glyphicon-fire:before { + content: "\e104"; +} +.glyphicon-eye-open:before { + content: "\e105"; +} +.glyphicon-eye-close:before { + content: "\e106"; +} +.glyphicon-warning-sign:before { + content: "\e107"; +} +.glyphicon-plane:before { + content: "\e108"; +} +.glyphicon-calendar:before { + content: "\e109"; +} +.glyphicon-random:before { + content: "\e110"; +} +.glyphicon-comment:before { + content: "\e111"; +} +.glyphicon-magnet:before { + content: "\e112"; +} +.glyphicon-chevron-up:before { + content: "\e113"; +} +.glyphicon-chevron-down:before { + content: "\e114"; +} +.glyphicon-retweet:before { + content: "\e115"; +} +.glyphicon-shopping-cart:before { + content: "\e116"; +} +.glyphicon-folder-close:before { + content: "\e117"; +} +.glyphicon-folder-open:before { + content: "\e118"; +} +.glyphicon-resize-vertical:before { + content: "\e119"; +} +.glyphicon-resize-horizontal:before { + content: "\e120"; +} +.glyphicon-hdd:before { + content: "\e121"; +} +.glyphicon-bullhorn:before { + content: "\e122"; +} +.glyphicon-bell:before { + content: "\e123"; +} +.glyphicon-certificate:before { + content: "\e124"; +} +.glyphicon-thumbs-up:before { + content: "\e125"; +} +.glyphicon-thumbs-down:before { + content: "\e126"; +} +.glyphicon-hand-right:before { + content: "\e127"; +} +.glyphicon-hand-left:before { + content: "\e128"; +} +.glyphicon-hand-up:before { + content: "\e129"; +} +.glyphicon-hand-down:before { + content: "\e130"; +} +.glyphicon-circle-arrow-right:before { + content: "\e131"; +} +.glyphicon-circle-arrow-left:before { + content: "\e132"; +} +.glyphicon-circle-arrow-up:before { + content: "\e133"; +} +.glyphicon-circle-arrow-down:before { + content: "\e134"; +} +.glyphicon-globe:before { + content: "\e135"; +} +.glyphicon-wrench:before { + content: "\e136"; +} +.glyphicon-tasks:before { + content: "\e137"; +} +.glyphicon-filter:before { + content: "\e138"; +} +.glyphicon-briefcase:before { + content: "\e139"; +} +.glyphicon-fullscreen:before { + content: "\e140"; +} +.glyphicon-dashboard:before { + content: "\e141"; +} +.glyphicon-paperclip:before { + content: "\e142"; +} +.glyphicon-heart-empty:before { + content: "\e143"; +} +.glyphicon-link:before { + content: "\e144"; +} +.glyphicon-phone:before { + content: "\e145"; +} +.glyphicon-pushpin:before { + content: "\e146"; +} +.glyphicon-usd:before { + content: "\e148"; +} +.glyphicon-gbp:before { + content: "\e149"; +} +.glyphicon-sort:before { + content: "\e150"; +} +.glyphicon-sort-by-alphabet:before { + content: "\e151"; +} +.glyphicon-sort-by-alphabet-alt:before { + content: "\e152"; +} +.glyphicon-sort-by-order:before { + content: "\e153"; +} +.glyphicon-sort-by-order-alt:before { + content: "\e154"; +} +.glyphicon-sort-by-attributes:before { + content: "\e155"; +} +.glyphicon-sort-by-attributes-alt:before { + content: "\e156"; +} +.glyphicon-unchecked:before { + content: "\e157"; +} +.glyphicon-expand:before { + content: "\e158"; +} +.glyphicon-collapse-down:before { + content: "\e159"; +} +.glyphicon-collapse-up:before { + content: "\e160"; +} +.glyphicon-log-in:before { + content: "\e161"; +} +.glyphicon-flash:before { + content: "\e162"; +} +.glyphicon-log-out:before { + content: "\e163"; +} +.glyphicon-new-window:before { + content: "\e164"; +} +.glyphicon-record:before { + content: "\e165"; +} +.glyphicon-save:before { + content: "\e166"; +} +.glyphicon-open:before { + content: "\e167"; +} +.glyphicon-saved:before { + content: "\e168"; +} +.glyphicon-import:before { + content: "\e169"; +} +.glyphicon-export:before { + content: "\e170"; +} +.glyphicon-send:before { + content: "\e171"; +} +.glyphicon-floppy-disk:before { + content: "\e172"; +} +.glyphicon-floppy-saved:before { + content: "\e173"; +} +.glyphicon-floppy-remove:before { + content: "\e174"; +} +.glyphicon-floppy-save:before { + content: "\e175"; +} +.glyphicon-floppy-open:before { + content: "\e176"; +} +.glyphicon-credit-card:before { + content: "\e177"; +} +.glyphicon-transfer:before { + content: "\e178"; +} +.glyphicon-cutlery:before { + content: "\e179"; +} +.glyphicon-header:before { + content: "\e180"; +} +.glyphicon-compressed:before { + content: "\e181"; +} +.glyphicon-earphone:before { + content: "\e182"; +} +.glyphicon-phone-alt:before { + content: "\e183"; +} +.glyphicon-tower:before { + content: "\e184"; +} +.glyphicon-stats:before { + content: "\e185"; +} +.glyphicon-sd-video:before { + content: "\e186"; +} +.glyphicon-hd-video:before { + content: "\e187"; +} +.glyphicon-subtitles:before { + content: "\e188"; +} +.glyphicon-sound-stereo:before { + content: "\e189"; +} +.glyphicon-sound-dolby:before { + content: "\e190"; +} +.glyphicon-sound-5-1:before { + content: "\e191"; +} +.glyphicon-sound-6-1:before { + content: "\e192"; +} +.glyphicon-sound-7-1:before { + content: "\e193"; +} +.glyphicon-copyright-mark:before { + content: "\e194"; +} +.glyphicon-registration-mark:before { + content: "\e195"; +} +.glyphicon-cloud-download:before { + content: "\e197"; +} +.glyphicon-cloud-upload:before { + content: "\e198"; +} +.glyphicon-tree-conifer:before { + content: "\e199"; +} +.glyphicon-tree-deciduous:before { + content: "\e200"; +} +.glyphicon-cd:before { + content: "\e201"; +} +.glyphicon-save-file:before { + content: "\e202"; +} +.glyphicon-open-file:before { + content: "\e203"; +} +.glyphicon-level-up:before { + content: "\e204"; +} +.glyphicon-copy:before { + content: "\e205"; +} +.glyphicon-paste:before { + content: "\e206"; +} +.glyphicon-alert:before { + content: "\e209"; +} +.glyphicon-equalizer:before { + content: "\e210"; +} +.glyphicon-king:before { + content: "\e211"; +} +.glyphicon-queen:before { + content: "\e212"; +} +.glyphicon-pawn:before { + content: "\e213"; +} +.glyphicon-bishop:before { + content: "\e214"; +} +.glyphicon-knight:before { + content: "\e215"; +} +.glyphicon-baby-formula:before { + content: "\e216"; +} +.glyphicon-tent:before { + content: "\26fa"; +} +.glyphicon-blackboard:before { + content: "\e218"; +} +.glyphicon-bed:before { + content: "\e219"; +} +.glyphicon-apple:before { + content: "\f8ff"; +} +.glyphicon-erase:before { + content: "\e221"; +} +.glyphicon-hourglass:before { + content: "\231b"; +} +.glyphicon-lamp:before { + content: "\e223"; +} +.glyphicon-duplicate:before { + content: "\e224"; +} +.glyphicon-piggy-bank:before { + content: "\e225"; +} +.glyphicon-scissors:before { + content: "\e226"; +} +.glyphicon-bitcoin:before { + content: "\e227"; +} +.glyphicon-btc:before { + content: "\e227"; +} +.glyphicon-xbt:before { + content: "\e227"; +} +.glyphicon-yen:before { + content: "\00a5"; +} +.glyphicon-jpy:before { + content: "\00a5"; +} +.glyphicon-ruble:before { + content: "\20bd"; +} +.glyphicon-rub:before { + content: "\20bd"; +} +.glyphicon-scale:before { + content: "\e230"; +} +.glyphicon-ice-lolly:before { + content: "\e231"; +} +.glyphicon-ice-lolly-tasted:before { + content: "\e232"; +} +.glyphicon-education:before { + content: "\e233"; +} +.glyphicon-option-horizontal:before { + content: "\e234"; +} +.glyphicon-option-vertical:before { + content: "\e235"; +} +.glyphicon-menu-hamburger:before { + content: "\e236"; +} +.glyphicon-modal-window:before { + content: "\e237"; +} +.glyphicon-oil:before { + content: "\e238"; +} +.glyphicon-grain:before { + content: "\e239"; +} +.glyphicon-sunglasses:before { + content: "\e240"; +} +.glyphicon-text-size:before { + content: "\e241"; +} +.glyphicon-text-color:before { + content: "\e242"; +} +.glyphicon-text-background:before { + content: "\e243"; +} +.glyphicon-object-align-top:before { + content: "\e244"; +} +.glyphicon-object-align-bottom:before { + content: "\e245"; +} +.glyphicon-object-align-horizontal:before { + content: "\e246"; +} +.glyphicon-object-align-left:before { + content: "\e247"; +} +.glyphicon-object-align-vertical:before { + content: "\e248"; +} +.glyphicon-object-align-right:before { + content: "\e249"; +} +.glyphicon-triangle-right:before { + content: "\e250"; +} +.glyphicon-triangle-left:before { + content: "\e251"; +} +.glyphicon-triangle-bottom:before { + content: "\e252"; +} +.glyphicon-triangle-top:before { + content: "\e253"; +} +.glyphicon-console:before { + content: "\e254"; +} +.glyphicon-superscript:before { + content: "\e255"; +} +.glyphicon-subscript:before { + content: "\e256"; +} +.glyphicon-menu-left:before { + content: "\e257"; +} +.glyphicon-menu-right:before { + content: "\e258"; +} +.glyphicon-menu-down:before { + content: "\e259"; +} +.glyphicon-menu-up:before { + content: "\e260"; +} +* { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +*:before, +*:after { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +html { + font-size: 10px; + + -webkit-tap-highlight-color: rgba(0, 0, 0, 0); +} +body { + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 14px; + line-height: 1.42857143; + color: #333; + background-color: #fff; +} +input, +button, +select, +textarea { + font-family: inherit; + font-size: inherit; + line-height: inherit; +} +a { + color: #337ab7; + text-decoration: none; +} +a:hover, +a:focus { + color: #23527c; + text-decoration: underline; +} +a:focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +figure { + margin: 0; +} +img { + vertical-align: middle; +} +.img-responsive, +.thumbnail > img, +.thumbnail a > img, +.carousel-inner > .item > img, +.carousel-inner > .item > a > img { + display: block; + max-width: 100%; + height: auto; +} +.img-rounded { + border-radius: 6px; +} +.img-thumbnail { + display: inline-block; + max-width: 100%; + height: auto; + padding: 4px; + line-height: 1.42857143; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 4px; + -webkit-transition: all .2s ease-in-out; + -o-transition: all .2s ease-in-out; + transition: all .2s ease-in-out; +} +.img-circle { + border-radius: 50%; +} +hr { + margin-top: 20px; + margin-bottom: 20px; + border: 0; + border-top: 1px solid #eee; +} +.sr-only { + position: absolute; + width: 1px; + height: 1px; + padding: 0; + margin: -1px; + overflow: hidden; + clip: rect(0, 0, 0, 0); + border: 0; +} +.sr-only-focusable:active, +.sr-only-focusable:focus { + position: static; + width: auto; + height: auto; + margin: 0; + overflow: visible; + clip: auto; +} +[role="button"] { + cursor: pointer; +} +h1, +h2, +h3, +h4, +h5, +h6, +.h1, +.h2, +.h3, +.h4, +.h5, +.h6 { + font-family: inherit; + font-weight: 500; + line-height: 1.1; + color: inherit; +} +h1 small, +h2 small, +h3 small, +h4 small, +h5 small, +h6 small, +.h1 small, +.h2 small, +.h3 small, +.h4 small, +.h5 small, +.h6 small, +h1 .small, +h2 .small, +h3 .small, +h4 .small, +h5 .small, +h6 .small, +.h1 .small, +.h2 .small, +.h3 .small, +.h4 .small, +.h5 .small, +.h6 .small { + font-weight: normal; + line-height: 1; + color: #777; +} +h1, +.h1, +h2, +.h2, +h3, +.h3 { + margin-top: 20px; + margin-bottom: 10px; +} +h1 small, +.h1 small, +h2 small, +.h2 small, +h3 small, +.h3 small, +h1 .small, +.h1 .small, +h2 .small, +.h2 .small, +h3 .small, +.h3 .small { + font-size: 65%; +} +h4, +.h4, +h5, +.h5, +h6, +.h6 { + margin-top: 10px; + margin-bottom: 10px; +} +h4 small, +.h4 small, +h5 small, +.h5 small, +h6 small, +.h6 small, +h4 .small, +.h4 .small, +h5 .small, +.h5 .small, +h6 .small, +.h6 .small { + font-size: 75%; +} +h1, +.h1 { + font-size: 36px; +} +h2, +.h2 { + font-size: 30px; +} +h3, +.h3 { + font-size: 24px; +} +h4, +.h4 { + font-size: 18px; +} +h5, +.h5 { + font-size: 14px; +} +h6, +.h6 { + font-size: 12px; +} +p { + margin: 0 0 10px; +} +.lead { + margin-bottom: 20px; + font-size: 16px; + font-weight: 300; + line-height: 1.4; +} +@media (min-width: 768px) { + .lead { + font-size: 21px; + } +} +small, +.small { + font-size: 85%; +} +mark, +.mark { + padding: .2em; + background-color: #fcf8e3; +} +.text-left { + text-align: left; +} +.text-right { + text-align: right; +} +.text-center { + text-align: center; +} +.text-justify { + text-align: justify; +} +.text-nowrap { + white-space: nowrap; +} +.text-lowercase { + text-transform: lowercase; +} +.text-uppercase { + text-transform: uppercase; +} +.text-capitalize { + text-transform: capitalize; +} +.text-muted { + color: #777; +} +.text-primary { + color: #337ab7; +} +a.text-primary:hover, +a.text-primary:focus { + color: #286090; +} +.text-success { + color: #3c763d; +} +a.text-success:hover, +a.text-success:focus { + color: #2b542c; +} +.text-info { + color: #31708f; +} +a.text-info:hover, +a.text-info:focus { + color: #245269; +} +.text-warning { + color: #8a6d3b; +} +a.text-warning:hover, +a.text-warning:focus { + color: #66512c; +} +.text-danger { + color: #a94442; +} +a.text-danger:hover, +a.text-danger:focus { + color: #843534; +} +.bg-primary { + color: #fff; + background-color: #337ab7; +} +a.bg-primary:hover, +a.bg-primary:focus { + background-color: #286090; +} +.bg-success { + background-color: #dff0d8; +} +a.bg-success:hover, +a.bg-success:focus { + background-color: #c1e2b3; +} +.bg-info { + background-color: #d9edf7; +} +a.bg-info:hover, +a.bg-info:focus { + background-color: #afd9ee; +} +.bg-warning { + background-color: #fcf8e3; +} +a.bg-warning:hover, +a.bg-warning:focus { + background-color: #f7ecb5; +} +.bg-danger { + background-color: #f2dede; +} +a.bg-danger:hover, +a.bg-danger:focus { + background-color: #e4b9b9; +} +.page-header { + padding-bottom: 9px; + margin: 40px 0 20px; + border-bottom: 1px solid #eee; +} +ul, +ol { + margin-top: 0; + margin-bottom: 10px; +} +ul ul, +ol ul, +ul ol, +ol ol { + margin-bottom: 0; +} +.list-unstyled { + padding-left: 0; + list-style: none; +} +.list-inline { + padding-left: 0; + margin-left: -5px; + list-style: none; +} +.list-inline > li { + display: inline-block; + padding-right: 5px; + padding-left: 5px; +} +dl { + margin-top: 0; + margin-bottom: 20px; +} +dt, +dd { + line-height: 1.42857143; +} +dt { + font-weight: bold; +} +dd { + margin-left: 0; +} +@media (min-width: 768px) { + .dl-horizontal dt { + float: left; + width: 160px; + overflow: hidden; + clear: left; + text-align: right; + text-overflow: ellipsis; + white-space: nowrap; + } + .dl-horizontal dd { + margin-left: 180px; + } +} +abbr[title], +abbr[data-original-title] { + cursor: help; + border-bottom: 1px dotted #777; +} +.initialism { + font-size: 90%; + text-transform: uppercase; +} +blockquote { + padding: 10px 20px; + margin: 0 0 20px; + font-size: 17.5px; + border-left: 5px solid #eee; +} +blockquote p:last-child, +blockquote ul:last-child, +blockquote ol:last-child { + margin-bottom: 0; +} +blockquote footer, +blockquote small, +blockquote .small { + display: block; + font-size: 80%; + line-height: 1.42857143; + color: #777; +} +blockquote footer:before, +blockquote small:before, +blockquote .small:before { + content: '\2014 \00A0'; +} +.blockquote-reverse, +blockquote.pull-right { + padding-right: 15px; + padding-left: 0; + text-align: right; + border-right: 5px solid #eee; + border-left: 0; +} +.blockquote-reverse footer:before, +blockquote.pull-right footer:before, +.blockquote-reverse small:before, +blockquote.pull-right small:before, +.blockquote-reverse .small:before, +blockquote.pull-right .small:before { + content: ''; +} +.blockquote-reverse footer:after, +blockquote.pull-right footer:after, +.blockquote-reverse small:after, +blockquote.pull-right small:after, +.blockquote-reverse .small:after, +blockquote.pull-right .small:after { + content: '\00A0 \2014'; +} +address { + margin-bottom: 20px; + font-style: normal; + line-height: 1.42857143; +} +code, +kbd, +pre, +samp { + font-family: Menlo, Monaco, Consolas, "Courier New", monospace; +} +code { + padding: 2px 4px; + font-size: 90%; + color: #c7254e; + background-color: #f9f2f4; + border-radius: 4px; +} +kbd { + padding: 2px 4px; + font-size: 90%; + color: #fff; + background-color: #333; + border-radius: 3px; + -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .25); + box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .25); +} +kbd kbd { + padding: 0; + font-size: 100%; + font-weight: bold; + -webkit-box-shadow: none; + box-shadow: none; +} +pre { + display: block; + padding: 9.5px; + margin: 0 0 10px; + font-size: 13px; + line-height: 1.42857143; + color: #333; + word-break: break-all; + word-wrap: break-word; + background-color: #f5f5f5; + border: 1px solid #ccc; + border-radius: 4px; +} +pre code { + padding: 0; + font-size: inherit; + color: inherit; + white-space: pre-wrap; + background-color: transparent; + border-radius: 0; +} +.pre-scrollable { + max-height: 340px; + overflow-y: scroll; +} +.container { + padding-right: 15px; + padding-left: 15px; + margin-right: auto; + margin-left: auto; +} +@media (min-width: 768px) { + .container { + width: 750px; + } +} +@media (min-width: 992px) { + .container { + width: 970px; + } +} +@media (min-width: 1200px) { + .container { + width: 1170px; + } +} +.container-fluid { + padding-right: 15px; + padding-left: 15px; + margin-right: auto; + margin-left: auto; +} +.row { + margin-right: -15px; + margin-left: -15px; +} +.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 { + position: relative; + min-height: 1px; + padding-right: 15px; + padding-left: 15px; +} +.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 { + float: left; +} +.col-xs-12 { + width: 100%; +} +.col-xs-11 { + width: 91.66666667%; +} +.col-xs-10 { + width: 83.33333333%; +} +.col-xs-9 { + width: 75%; +} +.col-xs-8 { + width: 66.66666667%; +} +.col-xs-7 { + width: 58.33333333%; +} +.col-xs-6 { + width: 50%; +} +.col-xs-5 { + width: 41.66666667%; +} +.col-xs-4 { + width: 33.33333333%; +} +.col-xs-3 { + width: 25%; +} +.col-xs-2 { + width: 16.66666667%; +} +.col-xs-1 { + width: 8.33333333%; +} +.col-xs-pull-12 { + right: 100%; +} +.col-xs-pull-11 { + right: 91.66666667%; +} +.col-xs-pull-10 { + right: 83.33333333%; +} +.col-xs-pull-9 { + right: 75%; +} +.col-xs-pull-8 { + right: 66.66666667%; +} +.col-xs-pull-7 { + right: 58.33333333%; +} +.col-xs-pull-6 { + right: 50%; +} +.col-xs-pull-5 { + right: 41.66666667%; +} +.col-xs-pull-4 { + right: 33.33333333%; +} +.col-xs-pull-3 { + right: 25%; +} +.col-xs-pull-2 { + right: 16.66666667%; +} +.col-xs-pull-1 { + right: 8.33333333%; +} +.col-xs-pull-0 { + right: auto; +} +.col-xs-push-12 { + left: 100%; +} +.col-xs-push-11 { + left: 91.66666667%; +} +.col-xs-push-10 { + left: 83.33333333%; +} +.col-xs-push-9 { + left: 75%; +} +.col-xs-push-8 { + left: 66.66666667%; +} +.col-xs-push-7 { + left: 58.33333333%; +} +.col-xs-push-6 { + left: 50%; +} +.col-xs-push-5 { + left: 41.66666667%; +} +.col-xs-push-4 { + left: 33.33333333%; +} +.col-xs-push-3 { + left: 25%; +} +.col-xs-push-2 { + left: 16.66666667%; +} +.col-xs-push-1 { + left: 8.33333333%; +} +.col-xs-push-0 { + left: auto; +} +.col-xs-offset-12 { + margin-left: 100%; +} +.col-xs-offset-11 { + margin-left: 91.66666667%; +} +.col-xs-offset-10 { + margin-left: 83.33333333%; +} +.col-xs-offset-9 { + margin-left: 75%; +} +.col-xs-offset-8 { + margin-left: 66.66666667%; +} +.col-xs-offset-7 { + margin-left: 58.33333333%; +} +.col-xs-offset-6 { + margin-left: 50%; +} +.col-xs-offset-5 { + margin-left: 41.66666667%; +} +.col-xs-offset-4 { + margin-left: 33.33333333%; +} +.col-xs-offset-3 { + margin-left: 25%; +} +.col-xs-offset-2 { + margin-left: 16.66666667%; +} +.col-xs-offset-1 { + margin-left: 8.33333333%; +} +.col-xs-offset-0 { + margin-left: 0; +} +@media (min-width: 768px) { + .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 { + float: left; + } + .col-sm-12 { + width: 100%; + } + .col-sm-11 { + width: 91.66666667%; + } + .col-sm-10 { + width: 83.33333333%; + } + .col-sm-9 { + width: 75%; + } + .col-sm-8 { + width: 66.66666667%; + } + .col-sm-7 { + width: 58.33333333%; + } + .col-sm-6 { + width: 50%; + } + .col-sm-5 { + width: 41.66666667%; + } + .col-sm-4 { + width: 33.33333333%; + } + .col-sm-3 { + width: 25%; + } + .col-sm-2 { + width: 16.66666667%; + } + .col-sm-1 { + width: 8.33333333%; + } + .col-sm-pull-12 { + right: 100%; + } + .col-sm-pull-11 { + right: 91.66666667%; + } + .col-sm-pull-10 { + right: 83.33333333%; + } + .col-sm-pull-9 { + right: 75%; + } + .col-sm-pull-8 { + right: 66.66666667%; + } + .col-sm-pull-7 { + right: 58.33333333%; + } + .col-sm-pull-6 { + right: 50%; + } + .col-sm-pull-5 { + right: 41.66666667%; + } + .col-sm-pull-4 { + right: 33.33333333%; + } + .col-sm-pull-3 { + right: 25%; + } + .col-sm-pull-2 { + right: 16.66666667%; + } + .col-sm-pull-1 { + right: 8.33333333%; + } + .col-sm-pull-0 { + right: auto; + } + .col-sm-push-12 { + left: 100%; + } + .col-sm-push-11 { + left: 91.66666667%; + } + .col-sm-push-10 { + left: 83.33333333%; + } + .col-sm-push-9 { + left: 75%; + } + .col-sm-push-8 { + left: 66.66666667%; + } + .col-sm-push-7 { + left: 58.33333333%; + } + .col-sm-push-6 { + left: 50%; + } + .col-sm-push-5 { + left: 41.66666667%; + } + .col-sm-push-4 { + left: 33.33333333%; + } + .col-sm-push-3 { + left: 25%; + } + .col-sm-push-2 { + left: 16.66666667%; + } + .col-sm-push-1 { + left: 8.33333333%; + } + .col-sm-push-0 { + left: auto; + } + .col-sm-offset-12 { + margin-left: 100%; + } + .col-sm-offset-11 { + margin-left: 91.66666667%; + } + .col-sm-offset-10 { + margin-left: 83.33333333%; + } + .col-sm-offset-9 { + margin-left: 75%; + } + .col-sm-offset-8 { + margin-left: 66.66666667%; + } + .col-sm-offset-7 { + margin-left: 58.33333333%; + } + .col-sm-offset-6 { + margin-left: 50%; + } + .col-sm-offset-5 { + margin-left: 41.66666667%; + } + .col-sm-offset-4 { + margin-left: 33.33333333%; + } + .col-sm-offset-3 { + margin-left: 25%; + } + .col-sm-offset-2 { + margin-left: 16.66666667%; + } + .col-sm-offset-1 { + margin-left: 8.33333333%; + } + .col-sm-offset-0 { + margin-left: 0; + } +} +@media (min-width: 992px) { + .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 { + float: left; + } + .col-md-12 { + width: 100%; + } + .col-md-11 { + width: 91.66666667%; + } + .col-md-10 { + width: 83.33333333%; + } + .col-md-9 { + width: 75%; + } + .col-md-8 { + width: 66.66666667%; + } + .col-md-7 { + width: 58.33333333%; + } + .col-md-6 { + width: 50%; + } + .col-md-5 { + width: 41.66666667%; + } + .col-md-4 { + width: 33.33333333%; + } + .col-md-3 { + width: 25%; + } + .col-md-2 { + width: 16.66666667%; + } + .col-md-1 { + width: 8.33333333%; + } + .col-md-pull-12 { + right: 100%; + } + .col-md-pull-11 { + right: 91.66666667%; + } + .col-md-pull-10 { + right: 83.33333333%; + } + .col-md-pull-9 { + right: 75%; + } + .col-md-pull-8 { + right: 66.66666667%; + } + .col-md-pull-7 { + right: 58.33333333%; + } + .col-md-pull-6 { + right: 50%; + } + .col-md-pull-5 { + right: 41.66666667%; + } + .col-md-pull-4 { + right: 33.33333333%; + } + .col-md-pull-3 { + right: 25%; + } + .col-md-pull-2 { + right: 16.66666667%; + } + .col-md-pull-1 { + right: 8.33333333%; + } + .col-md-pull-0 { + right: auto; + } + .col-md-push-12 { + left: 100%; + } + .col-md-push-11 { + left: 91.66666667%; + } + .col-md-push-10 { + left: 83.33333333%; + } + .col-md-push-9 { + left: 75%; + } + .col-md-push-8 { + left: 66.66666667%; + } + .col-md-push-7 { + left: 58.33333333%; + } + .col-md-push-6 { + left: 50%; + } + .col-md-push-5 { + left: 41.66666667%; + } + .col-md-push-4 { + left: 33.33333333%; + } + .col-md-push-3 { + left: 25%; + } + .col-md-push-2 { + left: 16.66666667%; + } + .col-md-push-1 { + left: 8.33333333%; + } + .col-md-push-0 { + left: auto; + } + .col-md-offset-12 { + margin-left: 100%; + } + .col-md-offset-11 { + margin-left: 91.66666667%; + } + .col-md-offset-10 { + margin-left: 83.33333333%; + } + .col-md-offset-9 { + margin-left: 75%; + } + .col-md-offset-8 { + margin-left: 66.66666667%; + } + .col-md-offset-7 { + margin-left: 58.33333333%; + } + .col-md-offset-6 { + margin-left: 50%; + } + .col-md-offset-5 { + margin-left: 41.66666667%; + } + .col-md-offset-4 { + margin-left: 33.33333333%; + } + .col-md-offset-3 { + margin-left: 25%; + } + .col-md-offset-2 { + margin-left: 16.66666667%; + } + .col-md-offset-1 { + margin-left: 8.33333333%; + } + .col-md-offset-0 { + margin-left: 0; + } +} +@media (min-width: 1200px) { + .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 { + float: left; + } + .col-lg-12 { + width: 100%; + } + .col-lg-11 { + width: 91.66666667%; + } + .col-lg-10 { + width: 83.33333333%; + } + .col-lg-9 { + width: 75%; + } + .col-lg-8 { + width: 66.66666667%; + } + .col-lg-7 { + width: 58.33333333%; + } + .col-lg-6 { + width: 50%; + } + .col-lg-5 { + width: 41.66666667%; + } + .col-lg-4 { + width: 33.33333333%; + } + .col-lg-3 { + width: 25%; + } + .col-lg-2 { + width: 16.66666667%; + } + .col-lg-1 { + width: 8.33333333%; + } + .col-lg-pull-12 { + right: 100%; + } + .col-lg-pull-11 { + right: 91.66666667%; + } + .col-lg-pull-10 { + right: 83.33333333%; + } + .col-lg-pull-9 { + right: 75%; + } + .col-lg-pull-8 { + right: 66.66666667%; + } + .col-lg-pull-7 { + right: 58.33333333%; + } + .col-lg-pull-6 { + right: 50%; + } + .col-lg-pull-5 { + right: 41.66666667%; + } + .col-lg-pull-4 { + right: 33.33333333%; + } + .col-lg-pull-3 { + right: 25%; + } + .col-lg-pull-2 { + right: 16.66666667%; + } + .col-lg-pull-1 { + right: 8.33333333%; + } + .col-lg-pull-0 { + right: auto; + } + .col-lg-push-12 { + left: 100%; + } + .col-lg-push-11 { + left: 91.66666667%; + } + .col-lg-push-10 { + left: 83.33333333%; + } + .col-lg-push-9 { + left: 75%; + } + .col-lg-push-8 { + left: 66.66666667%; + } + .col-lg-push-7 { + left: 58.33333333%; + } + .col-lg-push-6 { + left: 50%; + } + .col-lg-push-5 { + left: 41.66666667%; + } + .col-lg-push-4 { + left: 33.33333333%; + } + .col-lg-push-3 { + left: 25%; + } + .col-lg-push-2 { + left: 16.66666667%; + } + .col-lg-push-1 { + left: 8.33333333%; + } + .col-lg-push-0 { + left: auto; + } + .col-lg-offset-12 { + margin-left: 100%; + } + .col-lg-offset-11 { + margin-left: 91.66666667%; + } + .col-lg-offset-10 { + margin-left: 83.33333333%; + } + .col-lg-offset-9 { + margin-left: 75%; + } + .col-lg-offset-8 { + margin-left: 66.66666667%; + } + .col-lg-offset-7 { + margin-left: 58.33333333%; + } + .col-lg-offset-6 { + margin-left: 50%; + } + .col-lg-offset-5 { + margin-left: 41.66666667%; + } + .col-lg-offset-4 { + margin-left: 33.33333333%; + } + .col-lg-offset-3 { + margin-left: 25%; + } + .col-lg-offset-2 { + margin-left: 16.66666667%; + } + .col-lg-offset-1 { + margin-left: 8.33333333%; + } + .col-lg-offset-0 { + margin-left: 0; + } +} +table { + background-color: transparent; +} +caption { + padding-top: 8px; + padding-bottom: 8px; + color: #777; + text-align: left; +} +th { + text-align: left; +} +.table { + width: 100%; + max-width: 100%; + margin-bottom: 20px; +} +.table > thead > tr > th, +.table > tbody > tr > th, +.table > tfoot > tr > th, +.table > thead > tr > td, +.table > tbody > tr > td, +.table > tfoot > tr > td { + padding: 8px; + line-height: 1.42857143; + vertical-align: top; + border-top: 1px solid #ddd; +} +.table > thead > tr > th { + vertical-align: bottom; + border-bottom: 2px solid #ddd; +} +.table > caption + thead > tr:first-child > th, +.table > colgroup + thead > tr:first-child > th, +.table > thead:first-child > tr:first-child > th, +.table > caption + thead > tr:first-child > td, +.table > colgroup + thead > tr:first-child > td, +.table > thead:first-child > tr:first-child > td { + border-top: 0; +} +.table > tbody + tbody { + border-top: 2px solid #ddd; +} +.table .table { + background-color: #fff; +} +.table-condensed > thead > tr > th, +.table-condensed > tbody > tr > th, +.table-condensed > tfoot > tr > th, +.table-condensed > thead > tr > td, +.table-condensed > tbody > tr > td, +.table-condensed > tfoot > tr > td { + padding: 5px; +} +.table-bordered { + border: 1px solid #ddd; +} +.table-bordered > thead > tr > th, +.table-bordered > tbody > tr > th, +.table-bordered > tfoot > tr > th, +.table-bordered > thead > tr > td, +.table-bordered > tbody > tr > td, +.table-bordered > tfoot > tr > td { + border: 1px solid #ddd; +} +.table-bordered > thead > tr > th, +.table-bordered > thead > tr > td { + border-bottom-width: 2px; +} +.table-striped > tbody > tr:nth-of-type(odd) { + background-color: #f9f9f9; +} +.table-hover > tbody > tr:hover { + background-color: #f5f5f5; +} +table col[class*="col-"] { + position: static; + display: table-column; + float: none; +} +table td[class*="col-"], +table th[class*="col-"] { + position: static; + display: table-cell; + float: none; +} +.table > thead > tr > td.active, +.table > tbody > tr > td.active, +.table > tfoot > tr > td.active, +.table > thead > tr > th.active, +.table > tbody > tr > th.active, +.table > tfoot > tr > th.active, +.table > thead > tr.active > td, +.table > tbody > tr.active > td, +.table > tfoot > tr.active > td, +.table > thead > tr.active > th, +.table > tbody > tr.active > th, +.table > tfoot > tr.active > th { + background-color: #f5f5f5; +} +.table-hover > tbody > tr > td.active:hover, +.table-hover > tbody > tr > th.active:hover, +.table-hover > tbody > tr.active:hover > td, +.table-hover > tbody > tr:hover > .active, +.table-hover > tbody > tr.active:hover > th { + background-color: #e8e8e8; +} +.table > thead > tr > td.success, +.table > tbody > tr > td.success, +.table > tfoot > tr > td.success, +.table > thead > tr > th.success, +.table > tbody > tr > th.success, +.table > tfoot > tr > th.success, +.table > thead > tr.success > td, +.table > tbody > tr.success > td, +.table > tfoot > tr.success > td, +.table > thead > tr.success > th, +.table > tbody > tr.success > th, +.table > tfoot > tr.success > th { + background-color: #dff0d8; +} +.table-hover > tbody > tr > td.success:hover, +.table-hover > tbody > tr > th.success:hover, +.table-hover > tbody > tr.success:hover > td, +.table-hover > tbody > tr:hover > .success, +.table-hover > tbody > tr.success:hover > th { + background-color: #d0e9c6; +} +.table > thead > tr > td.info, +.table > tbody > tr > td.info, +.table > tfoot > tr > td.info, +.table > thead > tr > th.info, +.table > tbody > tr > th.info, +.table > tfoot > tr > th.info, +.table > thead > tr.info > td, +.table > tbody > tr.info > td, +.table > tfoot > tr.info > td, +.table > thead > tr.info > th, +.table > tbody > tr.info > th, +.table > tfoot > tr.info > th { + background-color: #d9edf7; +} +.table-hover > tbody > tr > td.info:hover, +.table-hover > tbody > tr > th.info:hover, +.table-hover > tbody > tr.info:hover > td, +.table-hover > tbody > tr:hover > .info, +.table-hover > tbody > tr.info:hover > th { + background-color: #c4e3f3; +} +.table > thead > tr > td.warning, +.table > tbody > tr > td.warning, +.table > tfoot > tr > td.warning, +.table > thead > tr > th.warning, +.table > tbody > tr > th.warning, +.table > tfoot > tr > th.warning, +.table > thead > tr.warning > td, +.table > tbody > tr.warning > td, +.table > tfoot > tr.warning > td, +.table > thead > tr.warning > th, +.table > tbody > tr.warning > th, +.table > tfoot > tr.warning > th { + background-color: #fcf8e3; +} +.table-hover > tbody > tr > td.warning:hover, +.table-hover > tbody > tr > th.warning:hover, +.table-hover > tbody > tr.warning:hover > td, +.table-hover > tbody > tr:hover > .warning, +.table-hover > tbody > tr.warning:hover > th { + background-color: #faf2cc; +} +.table > thead > tr > td.danger, +.table > tbody > tr > td.danger, +.table > tfoot > tr > td.danger, +.table > thead > tr > th.danger, +.table > tbody > tr > th.danger, +.table > tfoot > tr > th.danger, +.table > thead > tr.danger > td, +.table > tbody > tr.danger > td, +.table > tfoot > tr.danger > td, +.table > thead > tr.danger > th, +.table > tbody > tr.danger > th, +.table > tfoot > tr.danger > th { + background-color: #f2dede; +} +.table-hover > tbody > tr > td.danger:hover, +.table-hover > tbody > tr > th.danger:hover, +.table-hover > tbody > tr.danger:hover > td, +.table-hover > tbody > tr:hover > .danger, +.table-hover > tbody > tr.danger:hover > th { + background-color: #ebcccc; +} +.table-responsive { + min-height: .01%; + overflow-x: auto; +} +@media screen and (max-width: 767px) { + .table-responsive { + width: 100%; + margin-bottom: 15px; + overflow-y: hidden; + -ms-overflow-style: -ms-autohiding-scrollbar; + border: 1px solid #ddd; + } + .table-responsive > .table { + margin-bottom: 0; + } + .table-responsive > .table > thead > tr > th, + .table-responsive > .table > tbody > tr > th, + .table-responsive > .table > tfoot > tr > th, + .table-responsive > .table > thead > tr > td, + .table-responsive > .table > tbody > tr > td, + .table-responsive > .table > tfoot > tr > td { + white-space: nowrap; + } + .table-responsive > .table-bordered { + border: 0; + } + .table-responsive > .table-bordered > thead > tr > th:first-child, + .table-responsive > .table-bordered > tbody > tr > th:first-child, + .table-responsive > .table-bordered > tfoot > tr > th:first-child, + .table-responsive > .table-bordered > thead > tr > td:first-child, + .table-responsive > .table-bordered > tbody > tr > td:first-child, + .table-responsive > .table-bordered > tfoot > tr > td:first-child { + border-left: 0; + } + .table-responsive > .table-bordered > thead > tr > th:last-child, + .table-responsive > .table-bordered > tbody > tr > th:last-child, + .table-responsive > .table-bordered > tfoot > tr > th:last-child, + .table-responsive > .table-bordered > thead > tr > td:last-child, + .table-responsive > .table-bordered > tbody > tr > td:last-child, + .table-responsive > .table-bordered > tfoot > tr > td:last-child { + border-right: 0; + } + .table-responsive > .table-bordered > tbody > tr:last-child > th, + .table-responsive > .table-bordered > tfoot > tr:last-child > th, + .table-responsive > .table-bordered > tbody > tr:last-child > td, + .table-responsive > .table-bordered > tfoot > tr:last-child > td { + border-bottom: 0; + } +} +fieldset { + min-width: 0; + padding: 0; + margin: 0; + border: 0; +} +legend { + display: block; + width: 100%; + padding: 0; + margin-bottom: 20px; + font-size: 21px; + line-height: inherit; + color: #333; + border: 0; + border-bottom: 1px solid #e5e5e5; +} +label { + display: inline-block; + max-width: 100%; + margin-bottom: 5px; + font-weight: bold; +} +input[type="search"] { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +input[type="radio"], +input[type="checkbox"] { + margin: 4px 0 0; + margin-top: 1px \9; + line-height: normal; +} +input[type="file"] { + display: block; +} +input[type="range"] { + display: block; + width: 100%; +} +select[multiple], +select[size] { + height: auto; +} +input[type="file"]:focus, +input[type="radio"]:focus, +input[type="checkbox"]:focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +output { + display: block; + padding-top: 7px; + font-size: 14px; + line-height: 1.42857143; + color: #555; +} +.form-control { + display: block; + width: 100%; + height: 34px; + padding: 6px 12px; + font-size: 14px; + line-height: 1.42857143; + color: #555; + background-color: #fff; + background-image: none; + border: 1px solid #ccc; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + -webkit-transition: border-color ease-in-out .15s, -webkit-box-shadow ease-in-out .15s; + -o-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s; + transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s; +} +.form-control:focus { + border-color: #66afe9; + outline: 0; + -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, .6); + box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, .6); +} +.form-control::-moz-placeholder { + color: #999; + opacity: 1; +} +.form-control:-ms-input-placeholder { + color: #999; +} +.form-control::-webkit-input-placeholder { + color: #999; +} +.form-control[disabled], +.form-control[readonly], +fieldset[disabled] .form-control { + background-color: #eee; + opacity: 1; +} +.form-control[disabled], +fieldset[disabled] .form-control { + cursor: not-allowed; +} +textarea.form-control { + height: auto; +} +input[type="search"] { + -webkit-appearance: none; +} +@media screen and (-webkit-min-device-pixel-ratio: 0) { + input[type="date"].form-control, + input[type="time"].form-control, + input[type="datetime-local"].form-control, + input[type="month"].form-control { + line-height: 34px; + } + input[type="date"].input-sm, + input[type="time"].input-sm, + input[type="datetime-local"].input-sm, + input[type="month"].input-sm, + .input-group-sm input[type="date"], + .input-group-sm input[type="time"], + .input-group-sm input[type="datetime-local"], + .input-group-sm input[type="month"] { + line-height: 30px; + } + input[type="date"].input-lg, + input[type="time"].input-lg, + input[type="datetime-local"].input-lg, + input[type="month"].input-lg, + .input-group-lg input[type="date"], + .input-group-lg input[type="time"], + .input-group-lg input[type="datetime-local"], + .input-group-lg input[type="month"] { + line-height: 46px; + } +} +.form-group { + margin-bottom: 15px; +} +.radio, +.checkbox { + position: relative; + display: block; + margin-top: 10px; + margin-bottom: 10px; +} +.radio label, +.checkbox label { + min-height: 20px; + padding-left: 20px; + margin-bottom: 0; + font-weight: normal; + cursor: pointer; +} +.radio input[type="radio"], +.radio-inline input[type="radio"], +.checkbox input[type="checkbox"], +.checkbox-inline input[type="checkbox"] { + position: absolute; + margin-top: 4px \9; + margin-left: -20px; +} +.radio + .radio, +.checkbox + .checkbox { + margin-top: -5px; +} +.radio-inline, +.checkbox-inline { + position: relative; + display: inline-block; + padding-left: 20px; + margin-bottom: 0; + font-weight: normal; + vertical-align: middle; + cursor: pointer; +} +.radio-inline + .radio-inline, +.checkbox-inline + .checkbox-inline { + margin-top: 0; + margin-left: 10px; +} +input[type="radio"][disabled], +input[type="checkbox"][disabled], +input[type="radio"].disabled, +input[type="checkbox"].disabled, +fieldset[disabled] input[type="radio"], +fieldset[disabled] input[type="checkbox"] { + cursor: not-allowed; +} +.radio-inline.disabled, +.checkbox-inline.disabled, +fieldset[disabled] .radio-inline, +fieldset[disabled] .checkbox-inline { + cursor: not-allowed; +} +.radio.disabled label, +.checkbox.disabled label, +fieldset[disabled] .radio label, +fieldset[disabled] .checkbox label { + cursor: not-allowed; +} +.form-control-static { + min-height: 34px; + padding-top: 7px; + padding-bottom: 7px; + margin-bottom: 0; +} +.form-control-static.input-lg, +.form-control-static.input-sm { + padding-right: 0; + padding-left: 0; +} +.input-sm { + height: 30px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +select.input-sm { + height: 30px; + line-height: 30px; +} +textarea.input-sm, +select[multiple].input-sm { + height: auto; +} +.form-group-sm .form-control { + height: 30px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +.form-group-sm select.form-control { + height: 30px; + line-height: 30px; +} +.form-group-sm textarea.form-control, +.form-group-sm select[multiple].form-control { + height: auto; +} +.form-group-sm .form-control-static { + height: 30px; + min-height: 32px; + padding: 6px 10px; + font-size: 12px; + line-height: 1.5; +} +.input-lg { + height: 46px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; + border-radius: 6px; +} +select.input-lg { + height: 46px; + line-height: 46px; +} +textarea.input-lg, +select[multiple].input-lg { + height: auto; +} +.form-group-lg .form-control { + height: 46px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; + border-radius: 6px; +} +.form-group-lg select.form-control { + height: 46px; + line-height: 46px; +} +.form-group-lg textarea.form-control, +.form-group-lg select[multiple].form-control { + height: auto; +} +.form-group-lg .form-control-static { + height: 46px; + min-height: 38px; + padding: 11px 16px; + font-size: 18px; + line-height: 1.3333333; +} +.has-feedback { + position: relative; +} +.has-feedback .form-control { + padding-right: 42.5px; +} +.form-control-feedback { + position: absolute; + top: 0; + right: 0; + z-index: 2; + display: block; + width: 34px; + height: 34px; + line-height: 34px; + text-align: center; + pointer-events: none; +} +.input-lg + .form-control-feedback, +.input-group-lg + .form-control-feedback, +.form-group-lg .form-control + .form-control-feedback { + width: 46px; + height: 46px; + line-height: 46px; +} +.input-sm + .form-control-feedback, +.input-group-sm + .form-control-feedback, +.form-group-sm .form-control + .form-control-feedback { + width: 30px; + height: 30px; + line-height: 30px; +} +.has-success .help-block, +.has-success .control-label, +.has-success .radio, +.has-success .checkbox, +.has-success .radio-inline, +.has-success .checkbox-inline, +.has-success.radio label, +.has-success.checkbox label, +.has-success.radio-inline label, +.has-success.checkbox-inline label { + color: #3c763d; +} +.has-success .form-control { + border-color: #3c763d; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-success .form-control:focus { + border-color: #2b542c; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #67b168; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #67b168; +} +.has-success .input-group-addon { + color: #3c763d; + background-color: #dff0d8; + border-color: #3c763d; +} +.has-success .form-control-feedback { + color: #3c763d; +} +.has-warning .help-block, +.has-warning .control-label, +.has-warning .radio, +.has-warning .checkbox, +.has-warning .radio-inline, +.has-warning .checkbox-inline, +.has-warning.radio label, +.has-warning.checkbox label, +.has-warning.radio-inline label, +.has-warning.checkbox-inline label { + color: #8a6d3b; +} +.has-warning .form-control { + border-color: #8a6d3b; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-warning .form-control:focus { + border-color: #66512c; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #c0a16b; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #c0a16b; +} +.has-warning .input-group-addon { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #8a6d3b; +} +.has-warning .form-control-feedback { + color: #8a6d3b; +} +.has-error .help-block, +.has-error .control-label, +.has-error .radio, +.has-error .checkbox, +.has-error .radio-inline, +.has-error .checkbox-inline, +.has-error.radio label, +.has-error.checkbox label, +.has-error.radio-inline label, +.has-error.checkbox-inline label { + color: #a94442; +} +.has-error .form-control { + border-color: #a94442; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-error .form-control:focus { + border-color: #843534; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #ce8483; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #ce8483; +} +.has-error .input-group-addon { + color: #a94442; + background-color: #f2dede; + border-color: #a94442; +} +.has-error .form-control-feedback { + color: #a94442; +} +.has-feedback label ~ .form-control-feedback { + top: 25px; +} +.has-feedback label.sr-only ~ .form-control-feedback { + top: 0; +} +.help-block { + display: block; + margin-top: 5px; + margin-bottom: 10px; + color: #737373; +} +@media (min-width: 768px) { + .form-inline .form-group { + display: inline-block; + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .form-control { + display: inline-block; + width: auto; + vertical-align: middle; + } + .form-inline .form-control-static { + display: inline-block; + } + .form-inline .input-group { + display: inline-table; + vertical-align: middle; + } + .form-inline .input-group .input-group-addon, + .form-inline .input-group .input-group-btn, + .form-inline .input-group .form-control { + width: auto; + } + .form-inline .input-group > .form-control { + width: 100%; + } + .form-inline .control-label { + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .radio, + .form-inline .checkbox { + display: inline-block; + margin-top: 0; + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .radio label, + .form-inline .checkbox label { + padding-left: 0; + } + .form-inline .radio input[type="radio"], + .form-inline .checkbox input[type="checkbox"] { + position: relative; + margin-left: 0; + } + .form-inline .has-feedback .form-control-feedback { + top: 0; + } +} +.form-horizontal .radio, +.form-horizontal .checkbox, +.form-horizontal .radio-inline, +.form-horizontal .checkbox-inline { + padding-top: 7px; + margin-top: 0; + margin-bottom: 0; +} +.form-horizontal .radio, +.form-horizontal .checkbox { + min-height: 27px; +} +.form-horizontal .form-group { + margin-right: -15px; + margin-left: -15px; +} +@media (min-width: 768px) { + .form-horizontal .control-label { + padding-top: 7px; + margin-bottom: 0; + text-align: right; + } +} +.form-horizontal .has-feedback .form-control-feedback { + right: 15px; +} +@media (min-width: 768px) { + .form-horizontal .form-group-lg .control-label { + padding-top: 14.333333px; + font-size: 18px; + } +} +@media (min-width: 768px) { + .form-horizontal .form-group-sm .control-label { + padding-top: 6px; + font-size: 12px; + } +} +.btn { + display: inline-block; + padding: 6px 12px; + margin-bottom: 0; + font-size: 14px; + font-weight: normal; + line-height: 1.42857143; + text-align: center; + white-space: nowrap; + vertical-align: middle; + -ms-touch-action: manipulation; + touch-action: manipulation; + cursor: pointer; + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + user-select: none; + background-image: none; + border: 1px solid transparent; + border-radius: 4px; +} +.btn:focus, +.btn:active:focus, +.btn.active:focus, +.btn.focus, +.btn:active.focus, +.btn.active.focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +.btn:hover, +.btn:focus, +.btn.focus { + color: #333; + text-decoration: none; +} +.btn:active, +.btn.active { + background-image: none; + outline: 0; + -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); + box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); +} +.btn.disabled, +.btn[disabled], +fieldset[disabled] .btn { + cursor: not-allowed; + filter: alpha(opacity=65); + -webkit-box-shadow: none; + box-shadow: none; + opacity: .65; +} +a.btn.disabled, +fieldset[disabled] a.btn { + pointer-events: none; +} +.btn-default { + color: #333; + background-color: #fff; + border-color: #ccc; +} +.btn-default:focus, +.btn-default.focus { + color: #333; + background-color: #e6e6e6; + border-color: #8c8c8c; +} +.btn-default:hover { + color: #333; + background-color: #e6e6e6; + border-color: #adadad; +} +.btn-default:active, +.btn-default.active, +.open > .dropdown-toggle.btn-default { + color: #333; + background-color: #e6e6e6; + border-color: #adadad; +} +.btn-default:active:hover, +.btn-default.active:hover, +.open > .dropdown-toggle.btn-default:hover, +.btn-default:active:focus, +.btn-default.active:focus, +.open > .dropdown-toggle.btn-default:focus, +.btn-default:active.focus, +.btn-default.active.focus, +.open > .dropdown-toggle.btn-default.focus { + color: #333; + background-color: #d4d4d4; + border-color: #8c8c8c; +} +.btn-default:active, +.btn-default.active, +.open > .dropdown-toggle.btn-default { + background-image: none; +} +.btn-default.disabled, +.btn-default[disabled], +fieldset[disabled] .btn-default, +.btn-default.disabled:hover, +.btn-default[disabled]:hover, +fieldset[disabled] .btn-default:hover, +.btn-default.disabled:focus, +.btn-default[disabled]:focus, +fieldset[disabled] .btn-default:focus, +.btn-default.disabled.focus, +.btn-default[disabled].focus, +fieldset[disabled] .btn-default.focus, +.btn-default.disabled:active, +.btn-default[disabled]:active, +fieldset[disabled] .btn-default:active, +.btn-default.disabled.active, +.btn-default[disabled].active, +fieldset[disabled] .btn-default.active { + background-color: #fff; + border-color: #ccc; +} +.btn-default .badge { + color: #fff; + background-color: #333; +} +.btn-primary { + color: #fff; + background-color: #337ab7; + border-color: #2e6da4; +} +.btn-primary:focus, +.btn-primary.focus { + color: #fff; + background-color: #286090; + border-color: #122b40; +} +.btn-primary:hover { + color: #fff; + background-color: #286090; + border-color: #204d74; +} +.btn-primary:active, +.btn-primary.active, +.open > .dropdown-toggle.btn-primary { + color: #fff; + background-color: #286090; + border-color: #204d74; +} +.btn-primary:active:hover, +.btn-primary.active:hover, +.open > .dropdown-toggle.btn-primary:hover, +.btn-primary:active:focus, +.btn-primary.active:focus, +.open > .dropdown-toggle.btn-primary:focus, +.btn-primary:active.focus, +.btn-primary.active.focus, +.open > .dropdown-toggle.btn-primary.focus { + color: #fff; + background-color: #204d74; + border-color: #122b40; +} +.btn-primary:active, +.btn-primary.active, +.open > .dropdown-toggle.btn-primary { + background-image: none; +} +.btn-primary.disabled, +.btn-primary[disabled], +fieldset[disabled] .btn-primary, +.btn-primary.disabled:hover, +.btn-primary[disabled]:hover, +fieldset[disabled] .btn-primary:hover, +.btn-primary.disabled:focus, +.btn-primary[disabled]:focus, +fieldset[disabled] .btn-primary:focus, +.btn-primary.disabled.focus, +.btn-primary[disabled].focus, +fieldset[disabled] .btn-primary.focus, +.btn-primary.disabled:active, +.btn-primary[disabled]:active, +fieldset[disabled] .btn-primary:active, +.btn-primary.disabled.active, +.btn-primary[disabled].active, +fieldset[disabled] .btn-primary.active { + background-color: #337ab7; + border-color: #2e6da4; +} +.btn-primary .badge { + color: #337ab7; + background-color: #fff; +} +.btn-success { + color: #fff; + background-color: #5cb85c; + border-color: #4cae4c; +} +.btn-success:focus, +.btn-success.focus { + color: #fff; + background-color: #449d44; + border-color: #255625; +} +.btn-success:hover { + color: #fff; + background-color: #449d44; + border-color: #398439; +} +.btn-success:active, +.btn-success.active, +.open > .dropdown-toggle.btn-success { + color: #fff; + background-color: #449d44; + border-color: #398439; +} +.btn-success:active:hover, +.btn-success.active:hover, +.open > .dropdown-toggle.btn-success:hover, +.btn-success:active:focus, +.btn-success.active:focus, +.open > .dropdown-toggle.btn-success:focus, +.btn-success:active.focus, +.btn-success.active.focus, +.open > .dropdown-toggle.btn-success.focus { + color: #fff; + background-color: #398439; + border-color: #255625; +} +.btn-success:active, +.btn-success.active, +.open > .dropdown-toggle.btn-success { + background-image: none; +} +.btn-success.disabled, +.btn-success[disabled], +fieldset[disabled] .btn-success, +.btn-success.disabled:hover, +.btn-success[disabled]:hover, +fieldset[disabled] .btn-success:hover, +.btn-success.disabled:focus, +.btn-success[disabled]:focus, +fieldset[disabled] .btn-success:focus, +.btn-success.disabled.focus, +.btn-success[disabled].focus, +fieldset[disabled] .btn-success.focus, +.btn-success.disabled:active, +.btn-success[disabled]:active, +fieldset[disabled] .btn-success:active, +.btn-success.disabled.active, +.btn-success[disabled].active, +fieldset[disabled] .btn-success.active { + background-color: #5cb85c; + border-color: #4cae4c; +} +.btn-success .badge { + color: #5cb85c; + background-color: #fff; +} +.btn-info { + color: #fff; + background-color: #5bc0de; + border-color: #46b8da; +} +.btn-info:focus, +.btn-info.focus { + color: #fff; + background-color: #31b0d5; + border-color: #1b6d85; +} +.btn-info:hover { + color: #fff; + background-color: #31b0d5; + border-color: #269abc; +} +.btn-info:active, +.btn-info.active, +.open > .dropdown-toggle.btn-info { + color: #fff; + background-color: #31b0d5; + border-color: #269abc; +} +.btn-info:active:hover, +.btn-info.active:hover, +.open > .dropdown-toggle.btn-info:hover, +.btn-info:active:focus, +.btn-info.active:focus, +.open > .dropdown-toggle.btn-info:focus, +.btn-info:active.focus, +.btn-info.active.focus, +.open > .dropdown-toggle.btn-info.focus { + color: #fff; + background-color: #269abc; + border-color: #1b6d85; +} +.btn-info:active, +.btn-info.active, +.open > .dropdown-toggle.btn-info { + background-image: none; +} +.btn-info.disabled, +.btn-info[disabled], +fieldset[disabled] .btn-info, +.btn-info.disabled:hover, +.btn-info[disabled]:hover, +fieldset[disabled] .btn-info:hover, +.btn-info.disabled:focus, +.btn-info[disabled]:focus, +fieldset[disabled] .btn-info:focus, +.btn-info.disabled.focus, +.btn-info[disabled].focus, +fieldset[disabled] .btn-info.focus, +.btn-info.disabled:active, +.btn-info[disabled]:active, +fieldset[disabled] .btn-info:active, +.btn-info.disabled.active, +.btn-info[disabled].active, +fieldset[disabled] .btn-info.active { + background-color: #5bc0de; + border-color: #46b8da; +} +.btn-info .badge { + color: #5bc0de; + background-color: #fff; +} +.btn-warning { + color: #fff; + background-color: #f0ad4e; + border-color: #eea236; +} +.btn-warning:focus, +.btn-warning.focus { + color: #fff; + background-color: #ec971f; + border-color: #985f0d; +} +.btn-warning:hover { + color: #fff; + background-color: #ec971f; + border-color: #d58512; +} +.btn-warning:active, +.btn-warning.active, +.open > .dropdown-toggle.btn-warning { + color: #fff; + background-color: #ec971f; + border-color: #d58512; +} +.btn-warning:active:hover, +.btn-warning.active:hover, +.open > .dropdown-toggle.btn-warning:hover, +.btn-warning:active:focus, +.btn-warning.active:focus, +.open > .dropdown-toggle.btn-warning:focus, +.btn-warning:active.focus, +.btn-warning.active.focus, +.open > .dropdown-toggle.btn-warning.focus { + color: #fff; + background-color: #d58512; + border-color: #985f0d; +} +.btn-warning:active, +.btn-warning.active, +.open > .dropdown-toggle.btn-warning { + background-image: none; +} +.btn-warning.disabled, +.btn-warning[disabled], +fieldset[disabled] .btn-warning, +.btn-warning.disabled:hover, +.btn-warning[disabled]:hover, +fieldset[disabled] .btn-warning:hover, +.btn-warning.disabled:focus, +.btn-warning[disabled]:focus, +fieldset[disabled] .btn-warning:focus, +.btn-warning.disabled.focus, +.btn-warning[disabled].focus, +fieldset[disabled] .btn-warning.focus, +.btn-warning.disabled:active, +.btn-warning[disabled]:active, +fieldset[disabled] .btn-warning:active, +.btn-warning.disabled.active, +.btn-warning[disabled].active, +fieldset[disabled] .btn-warning.active { + background-color: #f0ad4e; + border-color: #eea236; +} +.btn-warning .badge { + color: #f0ad4e; + background-color: #fff; +} +.btn-danger { + color: #fff; + background-color: #d9534f; + border-color: #d43f3a; +} +.btn-danger:focus, +.btn-danger.focus { + color: #fff; + background-color: #c9302c; + border-color: #761c19; +} +.btn-danger:hover { + color: #fff; + background-color: #c9302c; + border-color: #ac2925; +} +.btn-danger:active, +.btn-danger.active, +.open > .dropdown-toggle.btn-danger { + color: #fff; + background-color: #c9302c; + border-color: #ac2925; +} +.btn-danger:active:hover, +.btn-danger.active:hover, +.open > .dropdown-toggle.btn-danger:hover, +.btn-danger:active:focus, +.btn-danger.active:focus, +.open > .dropdown-toggle.btn-danger:focus, +.btn-danger:active.focus, +.btn-danger.active.focus, +.open > .dropdown-toggle.btn-danger.focus { + color: #fff; + background-color: #ac2925; + border-color: #761c19; +} +.btn-danger:active, +.btn-danger.active, +.open > .dropdown-toggle.btn-danger { + background-image: none; +} +.btn-danger.disabled, +.btn-danger[disabled], +fieldset[disabled] .btn-danger, +.btn-danger.disabled:hover, +.btn-danger[disabled]:hover, +fieldset[disabled] .btn-danger:hover, +.btn-danger.disabled:focus, +.btn-danger[disabled]:focus, +fieldset[disabled] .btn-danger:focus, +.btn-danger.disabled.focus, +.btn-danger[disabled].focus, +fieldset[disabled] .btn-danger.focus, +.btn-danger.disabled:active, +.btn-danger[disabled]:active, +fieldset[disabled] .btn-danger:active, +.btn-danger.disabled.active, +.btn-danger[disabled].active, +fieldset[disabled] .btn-danger.active { + background-color: #d9534f; + border-color: #d43f3a; +} +.btn-danger .badge { + color: #d9534f; + background-color: #fff; +} +.btn-link { + font-weight: normal; + color: #337ab7; + border-radius: 0; +} +.btn-link, +.btn-link:active, +.btn-link.active, +.btn-link[disabled], +fieldset[disabled] .btn-link { + background-color: transparent; + -webkit-box-shadow: none; + box-shadow: none; +} +.btn-link, +.btn-link:hover, +.btn-link:focus, +.btn-link:active { + border-color: transparent; +} +.btn-link:hover, +.btn-link:focus { + color: #23527c; + text-decoration: underline; + background-color: transparent; +} +.btn-link[disabled]:hover, +fieldset[disabled] .btn-link:hover, +.btn-link[disabled]:focus, +fieldset[disabled] .btn-link:focus { + color: #777; + text-decoration: none; +} +.btn-lg, +.btn-group-lg > .btn { + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; + border-radius: 6px; +} +.btn-sm, +.btn-group-sm > .btn { + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +.btn-xs, +.btn-group-xs > .btn { + padding: 1px 5px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +.btn-block { + display: block; + width: 100%; +} +.btn-block + .btn-block { + margin-top: 5px; +} +input[type="submit"].btn-block, +input[type="reset"].btn-block, +input[type="button"].btn-block { + width: 100%; +} +.fade { + opacity: 0; + -webkit-transition: opacity .15s linear; + -o-transition: opacity .15s linear; + transition: opacity .15s linear; +} +.fade.in { + opacity: 1; +} +.collapse { + display: none; +} +.collapse.in { + display: block; +} +tr.collapse.in { + display: table-row; +} +tbody.collapse.in { + display: table-row-group; +} +.collapsing { + position: relative; + height: 0; + overflow: hidden; + -webkit-transition-timing-function: ease; + -o-transition-timing-function: ease; + transition-timing-function: ease; + -webkit-transition-duration: .35s; + -o-transition-duration: .35s; + transition-duration: .35s; + -webkit-transition-property: height, visibility; + -o-transition-property: height, visibility; + transition-property: height, visibility; +} +.caret { + display: inline-block; + width: 0; + height: 0; + margin-left: 2px; + vertical-align: middle; + border-top: 4px dashed; + border-top: 4px solid \9; + border-right: 4px solid transparent; + border-left: 4px solid transparent; +} +.dropup, +.dropdown { + position: relative; +} +.dropdown-toggle:focus { + outline: 0; +} +.dropdown-menu { + position: absolute; + top: 100%; + left: 0; + z-index: 1000; + display: none; + float: left; + min-width: 160px; + padding: 5px 0; + margin: 2px 0 0; + font-size: 14px; + text-align: left; + list-style: none; + background-color: #fff; + -webkit-background-clip: padding-box; + background-clip: padding-box; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, .15); + border-radius: 4px; + -webkit-box-shadow: 0 6px 12px rgba(0, 0, 0, .175); + box-shadow: 0 6px 12px rgba(0, 0, 0, .175); +} +.dropdown-menu.pull-right { + right: 0; + left: auto; +} +.dropdown-menu .divider { + height: 1px; + margin: 9px 0; + overflow: hidden; + background-color: #e5e5e5; +} +.dropdown-menu > li > a { + display: block; + padding: 3px 20px; + clear: both; + font-weight: normal; + line-height: 1.42857143; + color: #333; + white-space: nowrap; +} +.dropdown-menu > li > a:hover, +.dropdown-menu > li > a:focus { + color: #262626; + text-decoration: none; + background-color: #f5f5f5; +} +.dropdown-menu > .active > a, +.dropdown-menu > .active > a:hover, +.dropdown-menu > .active > a:focus { + color: #fff; + text-decoration: none; + background-color: #337ab7; + outline: 0; +} +.dropdown-menu > .disabled > a, +.dropdown-menu > .disabled > a:hover, +.dropdown-menu > .disabled > a:focus { + color: #777; +} +.dropdown-menu > .disabled > a:hover, +.dropdown-menu > .disabled > a:focus { + text-decoration: none; + cursor: not-allowed; + background-color: transparent; + background-image: none; + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); +} +.open > .dropdown-menu { + display: block; +} +.open > a { + outline: 0; +} +.dropdown-menu-right { + right: 0; + left: auto; +} +.dropdown-menu-left { + right: auto; + left: 0; +} +.dropdown-header { + display: block; + padding: 3px 20px; + font-size: 12px; + line-height: 1.42857143; + color: #777; + white-space: nowrap; +} +.dropdown-backdrop { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 990; +} +.pull-right > .dropdown-menu { + right: 0; + left: auto; +} +.dropup .caret, +.navbar-fixed-bottom .dropdown .caret { + content: ""; + border-top: 0; + border-bottom: 4px dashed; + border-bottom: 4px solid \9; +} +.dropup .dropdown-menu, +.navbar-fixed-bottom .dropdown .dropdown-menu { + top: auto; + bottom: 100%; + margin-bottom: 2px; +} +@media (min-width: 768px) { + .navbar-right .dropdown-menu { + right: 0; + left: auto; + } + .navbar-right .dropdown-menu-left { + right: auto; + left: 0; + } +} +.btn-group, +.btn-group-vertical { + position: relative; + display: inline-block; + vertical-align: middle; +} +.btn-group > .btn, +.btn-group-vertical > .btn { + position: relative; + float: left; +} +.btn-group > .btn:hover, +.btn-group-vertical > .btn:hover, +.btn-group > .btn:focus, +.btn-group-vertical > .btn:focus, +.btn-group > .btn:active, +.btn-group-vertical > .btn:active, +.btn-group > .btn.active, +.btn-group-vertical > .btn.active { + z-index: 2; +} +.btn-group .btn + .btn, +.btn-group .btn + .btn-group, +.btn-group .btn-group + .btn, +.btn-group .btn-group + .btn-group { + margin-left: -1px; +} +.btn-toolbar { + margin-left: -5px; +} +.btn-toolbar .btn, +.btn-toolbar .btn-group, +.btn-toolbar .input-group { + float: left; +} +.btn-toolbar > .btn, +.btn-toolbar > .btn-group, +.btn-toolbar > .input-group { + margin-left: 5px; +} +.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) { + border-radius: 0; +} +.btn-group > .btn:first-child { + margin-left: 0; +} +.btn-group > .btn:first-child:not(:last-child):not(.dropdown-toggle) { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.btn-group > .btn:last-child:not(:first-child), +.btn-group > .dropdown-toggle:not(:first-child) { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group > .btn-group { + float: left; +} +.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn { + border-radius: 0; +} +.btn-group > .btn-group:first-child:not(:last-child) > .btn:last-child, +.btn-group > .btn-group:first-child:not(:last-child) > .dropdown-toggle { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.btn-group > .btn-group:last-child:not(:first-child) > .btn:first-child { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group .dropdown-toggle:active, +.btn-group.open .dropdown-toggle { + outline: 0; +} +.btn-group > .btn + .dropdown-toggle { + padding-right: 8px; + padding-left: 8px; +} +.btn-group > .btn-lg + .dropdown-toggle { + padding-right: 12px; + padding-left: 12px; +} +.btn-group.open .dropdown-toggle { + -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); + box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); +} +.btn-group.open .dropdown-toggle.btn-link { + -webkit-box-shadow: none; + box-shadow: none; +} +.btn .caret { + margin-left: 0; +} +.btn-lg .caret { + border-width: 5px 5px 0; + border-bottom-width: 0; +} +.dropup .btn-lg .caret { + border-width: 0 5px 5px; +} +.btn-group-vertical > .btn, +.btn-group-vertical > .btn-group, +.btn-group-vertical > .btn-group > .btn { + display: block; + float: none; + width: 100%; + max-width: 100%; +} +.btn-group-vertical > .btn-group > .btn { + float: none; +} +.btn-group-vertical > .btn + .btn, +.btn-group-vertical > .btn + .btn-group, +.btn-group-vertical > .btn-group + .btn, +.btn-group-vertical > .btn-group + .btn-group { + margin-top: -1px; + margin-left: 0; +} +.btn-group-vertical > .btn:not(:first-child):not(:last-child) { + border-radius: 0; +} +.btn-group-vertical > .btn:first-child:not(:last-child) { + border-top-right-radius: 4px; + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group-vertical > .btn:last-child:not(:first-child) { + border-top-left-radius: 0; + border-top-right-radius: 0; + border-bottom-left-radius: 4px; +} +.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn { + border-radius: 0; +} +.btn-group-vertical > .btn-group:first-child:not(:last-child) > .btn:last-child, +.btn-group-vertical > .btn-group:first-child:not(:last-child) > .dropdown-toggle { + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group-vertical > .btn-group:last-child:not(:first-child) > .btn:first-child { + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.btn-group-justified { + display: table; + width: 100%; + table-layout: fixed; + border-collapse: separate; +} +.btn-group-justified > .btn, +.btn-group-justified > .btn-group { + display: table-cell; + float: none; + width: 1%; +} +.btn-group-justified > .btn-group .btn { + width: 100%; +} +.btn-group-justified > .btn-group .dropdown-menu { + left: auto; +} +[data-toggle="buttons"] > .btn input[type="radio"], +[data-toggle="buttons"] > .btn-group > .btn input[type="radio"], +[data-toggle="buttons"] > .btn input[type="checkbox"], +[data-toggle="buttons"] > .btn-group > .btn input[type="checkbox"] { + position: absolute; + clip: rect(0, 0, 0, 0); + pointer-events: none; +} +.input-group { + position: relative; + display: table; + border-collapse: separate; +} +.input-group[class*="col-"] { + float: none; + padding-right: 0; + padding-left: 0; +} +.input-group .form-control { + position: relative; + z-index: 2; + float: left; + width: 100%; + margin-bottom: 0; +} +.input-group-lg > .form-control, +.input-group-lg > .input-group-addon, +.input-group-lg > .input-group-btn > .btn { + height: 46px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; + border-radius: 6px; +} +select.input-group-lg > .form-control, +select.input-group-lg > .input-group-addon, +select.input-group-lg > .input-group-btn > .btn { + height: 46px; + line-height: 46px; +} +textarea.input-group-lg > .form-control, +textarea.input-group-lg > .input-group-addon, +textarea.input-group-lg > .input-group-btn > .btn, +select[multiple].input-group-lg > .form-control, +select[multiple].input-group-lg > .input-group-addon, +select[multiple].input-group-lg > .input-group-btn > .btn { + height: auto; +} +.input-group-sm > .form-control, +.input-group-sm > .input-group-addon, +.input-group-sm > .input-group-btn > .btn { + height: 30px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +select.input-group-sm > .form-control, +select.input-group-sm > .input-group-addon, +select.input-group-sm > .input-group-btn > .btn { + height: 30px; + line-height: 30px; +} +textarea.input-group-sm > .form-control, +textarea.input-group-sm > .input-group-addon, +textarea.input-group-sm > .input-group-btn > .btn, +select[multiple].input-group-sm > .form-control, +select[multiple].input-group-sm > .input-group-addon, +select[multiple].input-group-sm > .input-group-btn > .btn { + height: auto; +} +.input-group-addon, +.input-group-btn, +.input-group .form-control { + display: table-cell; +} +.input-group-addon:not(:first-child):not(:last-child), +.input-group-btn:not(:first-child):not(:last-child), +.input-group .form-control:not(:first-child):not(:last-child) { + border-radius: 0; +} +.input-group-addon, +.input-group-btn { + width: 1%; + white-space: nowrap; + vertical-align: middle; +} +.input-group-addon { + padding: 6px 12px; + font-size: 14px; + font-weight: normal; + line-height: 1; + color: #555; + text-align: center; + background-color: #eee; + border: 1px solid #ccc; + border-radius: 4px; +} +.input-group-addon.input-sm { + padding: 5px 10px; + font-size: 12px; + border-radius: 3px; +} +.input-group-addon.input-lg { + padding: 10px 16px; + font-size: 18px; + border-radius: 6px; +} +.input-group-addon input[type="radio"], +.input-group-addon input[type="checkbox"] { + margin-top: 0; +} +.input-group .form-control:first-child, +.input-group-addon:first-child, +.input-group-btn:first-child > .btn, +.input-group-btn:first-child > .btn-group > .btn, +.input-group-btn:first-child > .dropdown-toggle, +.input-group-btn:last-child > .btn:not(:last-child):not(.dropdown-toggle), +.input-group-btn:last-child > .btn-group:not(:last-child) > .btn { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.input-group-addon:first-child { + border-right: 0; +} +.input-group .form-control:last-child, +.input-group-addon:last-child, +.input-group-btn:last-child > .btn, +.input-group-btn:last-child > .btn-group > .btn, +.input-group-btn:last-child > .dropdown-toggle, +.input-group-btn:first-child > .btn:not(:first-child), +.input-group-btn:first-child > .btn-group:not(:first-child) > .btn { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.input-group-addon:last-child { + border-left: 0; +} +.input-group-btn { + position: relative; + font-size: 0; + white-space: nowrap; +} +.input-group-btn > .btn { + position: relative; +} +.input-group-btn > .btn + .btn { + margin-left: -1px; +} +.input-group-btn > .btn:hover, +.input-group-btn > .btn:focus, +.input-group-btn > .btn:active { + z-index: 2; +} +.input-group-btn:first-child > .btn, +.input-group-btn:first-child > .btn-group { + margin-right: -1px; +} +.input-group-btn:last-child > .btn, +.input-group-btn:last-child > .btn-group { + z-index: 2; + margin-left: -1px; +} +.nav { + padding-left: 0; + margin-bottom: 0; + list-style: none; +} +.nav > li { + position: relative; + display: block; +} +.nav > li > a { + position: relative; + display: block; + padding: 10px 15px; +} +.nav > li > a:hover, +.nav > li > a:focus { + text-decoration: none; + background-color: #eee; +} +.nav > li.disabled > a { + color: #777; +} +.nav > li.disabled > a:hover, +.nav > li.disabled > a:focus { + color: #777; + text-decoration: none; + cursor: not-allowed; + background-color: transparent; +} +.nav .open > a, +.nav .open > a:hover, +.nav .open > a:focus { + background-color: #eee; + border-color: #337ab7; +} +.nav .nav-divider { + height: 1px; + margin: 9px 0; + overflow: hidden; + background-color: #e5e5e5; +} +.nav > li > a > img { + max-width: none; +} +.nav-tabs { + border-bottom: 1px solid #ddd; +} +.nav-tabs > li { + float: left; + margin-bottom: -1px; +} +.nav-tabs > li > a { + margin-right: 2px; + line-height: 1.42857143; + border: 1px solid transparent; + border-radius: 4px 4px 0 0; +} +.nav-tabs > li > a:hover { + border-color: #eee #eee #ddd; +} +.nav-tabs > li.active > a, +.nav-tabs > li.active > a:hover, +.nav-tabs > li.active > a:focus { + color: #555; + cursor: default; + background-color: #fff; + border: 1px solid #ddd; + border-bottom-color: transparent; +} +.nav-tabs.nav-justified { + width: 100%; + border-bottom: 0; +} +.nav-tabs.nav-justified > li { + float: none; +} +.nav-tabs.nav-justified > li > a { + margin-bottom: 5px; + text-align: center; +} +.nav-tabs.nav-justified > .dropdown .dropdown-menu { + top: auto; + left: auto; +} +@media (min-width: 768px) { + .nav-tabs.nav-justified > li { + display: table-cell; + width: 1%; + } + .nav-tabs.nav-justified > li > a { + margin-bottom: 0; + } +} +.nav-tabs.nav-justified > li > a { + margin-right: 0; + border-radius: 4px; +} +.nav-tabs.nav-justified > .active > a, +.nav-tabs.nav-justified > .active > a:hover, +.nav-tabs.nav-justified > .active > a:focus { + border: 1px solid #ddd; +} +@media (min-width: 768px) { + .nav-tabs.nav-justified > li > a { + border-bottom: 1px solid #ddd; + border-radius: 4px 4px 0 0; + } + .nav-tabs.nav-justified > .active > a, + .nav-tabs.nav-justified > .active > a:hover, + .nav-tabs.nav-justified > .active > a:focus { + border-bottom-color: #fff; + } +} +.nav-pills > li { + float: left; +} +.nav-pills > li > a { + border-radius: 4px; +} +.nav-pills > li + li { + margin-left: 2px; +} +.nav-pills > li.active > a, +.nav-pills > li.active > a:hover, +.nav-pills > li.active > a:focus { + color: #fff; + background-color: #337ab7; +} +.nav-stacked > li { + float: none; +} +.nav-stacked > li + li { + margin-top: 2px; + margin-left: 0; +} +.nav-justified { + width: 100%; +} +.nav-justified > li { + float: none; +} +.nav-justified > li > a { + margin-bottom: 5px; + text-align: center; +} +.nav-justified > .dropdown .dropdown-menu { + top: auto; + left: auto; +} +@media (min-width: 768px) { + .nav-justified > li { + display: table-cell; + width: 1%; + } + .nav-justified > li > a { + margin-bottom: 0; + } +} +.nav-tabs-justified { + border-bottom: 0; +} +.nav-tabs-justified > li > a { + margin-right: 0; + border-radius: 4px; +} +.nav-tabs-justified > .active > a, +.nav-tabs-justified > .active > a:hover, +.nav-tabs-justified > .active > a:focus { + border: 1px solid #ddd; +} +@media (min-width: 768px) { + .nav-tabs-justified > li > a { + border-bottom: 1px solid #ddd; + border-radius: 4px 4px 0 0; + } + .nav-tabs-justified > .active > a, + .nav-tabs-justified > .active > a:hover, + .nav-tabs-justified > .active > a:focus { + border-bottom-color: #fff; + } +} +.tab-content > .tab-pane { + display: none; +} +.tab-content > .active { + display: block; +} +.nav-tabs .dropdown-menu { + margin-top: -1px; + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.navbar { + position: relative; + min-height: 50px; + margin-bottom: 20px; + border: 1px solid transparent; +} +@media (min-width: 768px) { + .navbar { + border-radius: 4px; + } +} +@media (min-width: 768px) { + .navbar-header { + float: left; + } +} +.navbar-collapse { + padding-right: 15px; + padding-left: 15px; + overflow-x: visible; + -webkit-overflow-scrolling: touch; + border-top: 1px solid transparent; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1); +} +.navbar-collapse.in { + overflow-y: auto; +} +@media (min-width: 768px) { + .navbar-collapse { + width: auto; + border-top: 0; + -webkit-box-shadow: none; + box-shadow: none; + } + .navbar-collapse.collapse { + display: block !important; + height: auto !important; + padding-bottom: 0; + overflow: visible !important; + } + .navbar-collapse.in { + overflow-y: visible; + } + .navbar-fixed-top .navbar-collapse, + .navbar-static-top .navbar-collapse, + .navbar-fixed-bottom .navbar-collapse { + padding-right: 0; + padding-left: 0; + } +} +.navbar-fixed-top .navbar-collapse, +.navbar-fixed-bottom .navbar-collapse { + max-height: 340px; +} +@media (max-device-width: 480px) and (orientation: landscape) { + .navbar-fixed-top .navbar-collapse, + .navbar-fixed-bottom .navbar-collapse { + max-height: 200px; + } +} +.container > .navbar-header, +.container-fluid > .navbar-header, +.container > .navbar-collapse, +.container-fluid > .navbar-collapse { + margin-right: -15px; + margin-left: -15px; +} +@media (min-width: 768px) { + .container > .navbar-header, + .container-fluid > .navbar-header, + .container > .navbar-collapse, + .container-fluid > .navbar-collapse { + margin-right: 0; + margin-left: 0; + } +} +.navbar-static-top { + z-index: 1000; + border-width: 0 0 1px; +} +@media (min-width: 768px) { + .navbar-static-top { + border-radius: 0; + } +} +.navbar-fixed-top, +.navbar-fixed-bottom { + position: fixed; + right: 0; + left: 0; + z-index: 1030; +} +@media (min-width: 768px) { + .navbar-fixed-top, + .navbar-fixed-bottom { + border-radius: 0; + } +} +.navbar-fixed-top { + top: 0; + border-width: 0 0 1px; +} +.navbar-fixed-bottom { + bottom: 0; + margin-bottom: 0; + border-width: 1px 0 0; +} +.navbar-brand { + float: left; + height: 50px; + padding: 15px 15px; + font-size: 18px; + line-height: 20px; +} +.navbar-brand:hover, +.navbar-brand:focus { + text-decoration: none; +} +.navbar-brand > img { + display: block; +} +@media (min-width: 768px) { + .navbar > .container .navbar-brand, + .navbar > .container-fluid .navbar-brand { + margin-left: -15px; + } +} +.navbar-toggle { + position: relative; + float: right; + padding: 9px 10px; + margin-top: 8px; + margin-right: 15px; + margin-bottom: 8px; + background-color: transparent; + background-image: none; + border: 1px solid transparent; + border-radius: 4px; +} +.navbar-toggle:focus { + outline: 0; +} +.navbar-toggle .icon-bar { + display: block; + width: 22px; + height: 2px; + border-radius: 1px; +} +.navbar-toggle .icon-bar + .icon-bar { + margin-top: 4px; +} +@media (min-width: 768px) { + .navbar-toggle { + display: none; + } +} +.navbar-nav { + margin: 7.5px -15px; +} +.navbar-nav > li > a { + padding-top: 10px; + padding-bottom: 10px; + line-height: 20px; +} +@media (max-width: 767px) { + .navbar-nav .open .dropdown-menu { + position: static; + float: none; + width: auto; + margin-top: 0; + background-color: transparent; + border: 0; + -webkit-box-shadow: none; + box-shadow: none; + } + .navbar-nav .open .dropdown-menu > li > a, + .navbar-nav .open .dropdown-menu .dropdown-header { + padding: 5px 15px 5px 25px; + } + .navbar-nav .open .dropdown-menu > li > a { + line-height: 20px; + } + .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-nav .open .dropdown-menu > li > a:focus { + background-image: none; + } +} +@media (min-width: 768px) { + .navbar-nav { + float: left; + margin: 0; + } + .navbar-nav > li { + float: left; + } + .navbar-nav > li > a { + padding-top: 15px; + padding-bottom: 15px; + } +} +.navbar-form { + padding: 10px 15px; + margin-top: 8px; + margin-right: -15px; + margin-bottom: 8px; + margin-left: -15px; + border-top: 1px solid transparent; + border-bottom: 1px solid transparent; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1), 0 1px 0 rgba(255, 255, 255, .1); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1), 0 1px 0 rgba(255, 255, 255, .1); +} +@media (min-width: 768px) { + .navbar-form .form-group { + display: inline-block; + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .form-control { + display: inline-block; + width: auto; + vertical-align: middle; + } + .navbar-form .form-control-static { + display: inline-block; + } + .navbar-form .input-group { + display: inline-table; + vertical-align: middle; + } + .navbar-form .input-group .input-group-addon, + .navbar-form .input-group .input-group-btn, + .navbar-form .input-group .form-control { + width: auto; + } + .navbar-form .input-group > .form-control { + width: 100%; + } + .navbar-form .control-label { + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .radio, + .navbar-form .checkbox { + display: inline-block; + margin-top: 0; + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .radio label, + .navbar-form .checkbox label { + padding-left: 0; + } + .navbar-form .radio input[type="radio"], + .navbar-form .checkbox input[type="checkbox"] { + position: relative; + margin-left: 0; + } + .navbar-form .has-feedback .form-control-feedback { + top: 0; + } +} +@media (max-width: 767px) { + .navbar-form .form-group { + margin-bottom: 5px; + } + .navbar-form .form-group:last-child { + margin-bottom: 0; + } +} +@media (min-width: 768px) { + .navbar-form { + width: auto; + padding-top: 0; + padding-bottom: 0; + margin-right: 0; + margin-left: 0; + border: 0; + -webkit-box-shadow: none; + box-shadow: none; + } +} +.navbar-nav > li > .dropdown-menu { + margin-top: 0; + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.navbar-fixed-bottom .navbar-nav > li > .dropdown-menu { + margin-bottom: 0; + border-top-left-radius: 4px; + border-top-right-radius: 4px; + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.navbar-btn { + margin-top: 8px; + margin-bottom: 8px; +} +.navbar-btn.btn-sm { + margin-top: 10px; + margin-bottom: 10px; +} +.navbar-btn.btn-xs { + margin-top: 14px; + margin-bottom: 14px; +} +.navbar-text { + margin-top: 15px; + margin-bottom: 15px; +} +@media (min-width: 768px) { + .navbar-text { + float: left; + margin-right: 15px; + margin-left: 15px; + } +} +@media (min-width: 768px) { + .navbar-left { + float: left !important; + } + .navbar-right { + float: right !important; + margin-right: -15px; + } + .navbar-right ~ .navbar-right { + margin-right: 0; + } +} +.navbar-default { + background-color: #f8f8f8; + border-color: #e7e7e7; +} +.navbar-default .navbar-brand { + color: #777; +} +.navbar-default .navbar-brand:hover, +.navbar-default .navbar-brand:focus { + color: #5e5e5e; + background-color: transparent; +} +.navbar-default .navbar-text { + color: #777; +} +.navbar-default .navbar-nav > li > a { + color: #777; +} +.navbar-default .navbar-nav > li > a:hover, +.navbar-default .navbar-nav > li > a:focus { + color: #333; + background-color: transparent; +} +.navbar-default .navbar-nav > .active > a, +.navbar-default .navbar-nav > .active > a:hover, +.navbar-default .navbar-nav > .active > a:focus { + color: #555; + background-color: #e7e7e7; +} +.navbar-default .navbar-nav > .disabled > a, +.navbar-default .navbar-nav > .disabled > a:hover, +.navbar-default .navbar-nav > .disabled > a:focus { + color: #ccc; + background-color: transparent; +} +.navbar-default .navbar-toggle { + border-color: #ddd; +} +.navbar-default .navbar-toggle:hover, +.navbar-default .navbar-toggle:focus { + background-color: #ddd; +} +.navbar-default .navbar-toggle .icon-bar { + background-color: #888; +} +.navbar-default .navbar-collapse, +.navbar-default .navbar-form { + border-color: #e7e7e7; +} +.navbar-default .navbar-nav > .open > a, +.navbar-default .navbar-nav > .open > a:hover, +.navbar-default .navbar-nav > .open > a:focus { + color: #555; + background-color: #e7e7e7; +} +@media (max-width: 767px) { + .navbar-default .navbar-nav .open .dropdown-menu > li > a { + color: #777; + } + .navbar-default .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > li > a:focus { + color: #333; + background-color: transparent; + } + .navbar-default .navbar-nav .open .dropdown-menu > .active > a, + .navbar-default .navbar-nav .open .dropdown-menu > .active > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > .active > a:focus { + color: #555; + background-color: #e7e7e7; + } + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a, + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:focus { + color: #ccc; + background-color: transparent; + } +} +.navbar-default .navbar-link { + color: #777; +} +.navbar-default .navbar-link:hover { + color: #333; +} +.navbar-default .btn-link { + color: #777; +} +.navbar-default .btn-link:hover, +.navbar-default .btn-link:focus { + color: #333; +} +.navbar-default .btn-link[disabled]:hover, +fieldset[disabled] .navbar-default .btn-link:hover, +.navbar-default .btn-link[disabled]:focus, +fieldset[disabled] .navbar-default .btn-link:focus { + color: #ccc; +} +.navbar-inverse { + background-color: #222; + border-color: #080808; +} +.navbar-inverse .navbar-brand { + color: #9d9d9d; +} +.navbar-inverse .navbar-brand:hover, +.navbar-inverse .navbar-brand:focus { + color: #fff; + background-color: transparent; +} +.navbar-inverse .navbar-text { + color: #9d9d9d; +} +.navbar-inverse .navbar-nav > li > a { + color: #9d9d9d; +} +.navbar-inverse .navbar-nav > li > a:hover, +.navbar-inverse .navbar-nav > li > a:focus { + color: #fff; + background-color: transparent; +} +.navbar-inverse .navbar-nav > .active > a, +.navbar-inverse .navbar-nav > .active > a:hover, +.navbar-inverse .navbar-nav > .active > a:focus { + color: #fff; + background-color: #080808; +} +.navbar-inverse .navbar-nav > .disabled > a, +.navbar-inverse .navbar-nav > .disabled > a:hover, +.navbar-inverse .navbar-nav > .disabled > a:focus { + color: #444; + background-color: transparent; +} +.navbar-inverse .navbar-toggle { + border-color: #333; +} +.navbar-inverse .navbar-toggle:hover, +.navbar-inverse .navbar-toggle:focus { + background-color: #333; +} +.navbar-inverse .navbar-toggle .icon-bar { + background-color: #fff; +} +.navbar-inverse .navbar-collapse, +.navbar-inverse .navbar-form { + border-color: #101010; +} +.navbar-inverse .navbar-nav > .open > a, +.navbar-inverse .navbar-nav > .open > a:hover, +.navbar-inverse .navbar-nav > .open > a:focus { + color: #fff; + background-color: #080808; +} +@media (max-width: 767px) { + .navbar-inverse .navbar-nav .open .dropdown-menu > .dropdown-header { + border-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu .divider { + background-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a { + color: #9d9d9d; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:focus { + color: #fff; + background-color: transparent; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a, + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:focus { + color: #fff; + background-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a, + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:focus { + color: #444; + background-color: transparent; + } +} +.navbar-inverse .navbar-link { + color: #9d9d9d; +} +.navbar-inverse .navbar-link:hover { + color: #fff; +} +.navbar-inverse .btn-link { + color: #9d9d9d; +} +.navbar-inverse .btn-link:hover, +.navbar-inverse .btn-link:focus { + color: #fff; +} +.navbar-inverse .btn-link[disabled]:hover, +fieldset[disabled] .navbar-inverse .btn-link:hover, +.navbar-inverse .btn-link[disabled]:focus, +fieldset[disabled] .navbar-inverse .btn-link:focus { + color: #444; +} +.breadcrumb { + padding: 8px 15px; + margin-bottom: 20px; + list-style: none; + background-color: #f5f5f5; + border-radius: 4px; +} +.breadcrumb > li { + display: inline-block; +} +.breadcrumb > li + li:before { + padding: 0 5px; + color: #ccc; + content: "/\00a0"; +} +.breadcrumb > .active { + color: #777; +} +.pagination { + display: inline-block; + padding-left: 0; + margin: 20px 0; + border-radius: 4px; +} +.pagination > li { + display: inline; +} +.pagination > li > a, +.pagination > li > span { + position: relative; + float: left; + padding: 6px 12px; + margin-left: -1px; + line-height: 1.42857143; + color: #337ab7; + text-decoration: none; + background-color: #fff; + border: 1px solid #ddd; +} +.pagination > li:first-child > a, +.pagination > li:first-child > span { + margin-left: 0; + border-top-left-radius: 4px; + border-bottom-left-radius: 4px; +} +.pagination > li:last-child > a, +.pagination > li:last-child > span { + border-top-right-radius: 4px; + border-bottom-right-radius: 4px; +} +.pagination > li > a:hover, +.pagination > li > span:hover, +.pagination > li > a:focus, +.pagination > li > span:focus { + z-index: 3; + color: #23527c; + background-color: #eee; + border-color: #ddd; +} +.pagination > .active > a, +.pagination > .active > span, +.pagination > .active > a:hover, +.pagination > .active > span:hover, +.pagination > .active > a:focus, +.pagination > .active > span:focus { + z-index: 2; + color: #fff; + cursor: default; + background-color: #337ab7; + border-color: #337ab7; +} +.pagination > .disabled > span, +.pagination > .disabled > span:hover, +.pagination > .disabled > span:focus, +.pagination > .disabled > a, +.pagination > .disabled > a:hover, +.pagination > .disabled > a:focus { + color: #777; + cursor: not-allowed; + background-color: #fff; + border-color: #ddd; +} +.pagination-lg > li > a, +.pagination-lg > li > span { + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; +} +.pagination-lg > li:first-child > a, +.pagination-lg > li:first-child > span { + border-top-left-radius: 6px; + border-bottom-left-radius: 6px; +} +.pagination-lg > li:last-child > a, +.pagination-lg > li:last-child > span { + border-top-right-radius: 6px; + border-bottom-right-radius: 6px; +} +.pagination-sm > li > a, +.pagination-sm > li > span { + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; +} +.pagination-sm > li:first-child > a, +.pagination-sm > li:first-child > span { + border-top-left-radius: 3px; + border-bottom-left-radius: 3px; +} +.pagination-sm > li:last-child > a, +.pagination-sm > li:last-child > span { + border-top-right-radius: 3px; + border-bottom-right-radius: 3px; +} +.pager { + padding-left: 0; + margin: 20px 0; + text-align: center; + list-style: none; +} +.pager li { + display: inline; +} +.pager li > a, +.pager li > span { + display: inline-block; + padding: 5px 14px; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 15px; +} +.pager li > a:hover, +.pager li > a:focus { + text-decoration: none; + background-color: #eee; +} +.pager .next > a, +.pager .next > span { + float: right; +} +.pager .previous > a, +.pager .previous > span { + float: left; +} +.pager .disabled > a, +.pager .disabled > a:hover, +.pager .disabled > a:focus, +.pager .disabled > span { + color: #777; + cursor: not-allowed; + background-color: #fff; +} +.label { + display: inline; + padding: .2em .6em .3em; + font-size: 75%; + font-weight: bold; + line-height: 1; + color: #fff; + text-align: center; + white-space: nowrap; + vertical-align: baseline; + border-radius: .25em; +} +a.label:hover, +a.label:focus { + color: #fff; + text-decoration: none; + cursor: pointer; +} +.label:empty { + display: none; +} +.btn .label { + position: relative; + top: -1px; +} +.label-default { + background-color: #777; +} +.label-default[href]:hover, +.label-default[href]:focus { + background-color: #5e5e5e; +} +.label-primary { + background-color: #337ab7; +} +.label-primary[href]:hover, +.label-primary[href]:focus { + background-color: #286090; +} +.label-success { + background-color: #5cb85c; +} +.label-success[href]:hover, +.label-success[href]:focus { + background-color: #449d44; +} +.label-info { + background-color: #5bc0de; +} +.label-info[href]:hover, +.label-info[href]:focus { + background-color: #31b0d5; +} +.label-warning { + background-color: #f0ad4e; +} +.label-warning[href]:hover, +.label-warning[href]:focus { + background-color: #ec971f; +} +.label-danger { + background-color: #d9534f; +} +.label-danger[href]:hover, +.label-danger[href]:focus { + background-color: #c9302c; +} +.badge { + display: inline-block; + min-width: 10px; + padding: 3px 7px; + font-size: 12px; + font-weight: bold; + line-height: 1; + color: #fff; + text-align: center; + white-space: nowrap; + vertical-align: middle; + background-color: #777; + border-radius: 10px; +} +.badge:empty { + display: none; +} +.btn .badge { + position: relative; + top: -1px; +} +.btn-xs .badge, +.btn-group-xs > .btn .badge { + top: 0; + padding: 1px 5px; +} +a.badge:hover, +a.badge:focus { + color: #fff; + text-decoration: none; + cursor: pointer; +} +.list-group-item.active > .badge, +.nav-pills > .active > a > .badge { + color: #337ab7; + background-color: #fff; +} +.list-group-item > .badge { + float: right; +} +.list-group-item > .badge + .badge { + margin-right: 5px; +} +.nav-pills > li > a > .badge { + margin-left: 3px; +} +.jumbotron { + padding-top: 30px; + padding-bottom: 30px; + margin-bottom: 30px; + color: inherit; + background-color: #eee; +} +.jumbotron h1, +.jumbotron .h1 { + color: inherit; +} +.jumbotron p { + margin-bottom: 15px; + font-size: 21px; + font-weight: 200; +} +.jumbotron > hr { + border-top-color: #d5d5d5; +} +.container .jumbotron, +.container-fluid .jumbotron { + border-radius: 6px; +} +.jumbotron .container { + max-width: 100%; +} +@media screen and (min-width: 768px) { + .jumbotron { + padding-top: 48px; + padding-bottom: 48px; + } + .container .jumbotron, + .container-fluid .jumbotron { + padding-right: 60px; + padding-left: 60px; + } + .jumbotron h1, + .jumbotron .h1 { + font-size: 63px; + } +} +.thumbnail { + display: block; + padding: 4px; + margin-bottom: 20px; + line-height: 1.42857143; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 4px; + -webkit-transition: border .2s ease-in-out; + -o-transition: border .2s ease-in-out; + transition: border .2s ease-in-out; +} +.thumbnail > img, +.thumbnail a > img { + margin-right: auto; + margin-left: auto; +} +a.thumbnail:hover, +a.thumbnail:focus, +a.thumbnail.active { + border-color: #337ab7; +} +.thumbnail .caption { + padding: 9px; + color: #333; +} +.alert { + padding: 15px; + margin-bottom: 20px; + border: 1px solid transparent; + border-radius: 4px; +} +.alert h4 { + margin-top: 0; + color: inherit; +} +.alert .alert-link { + font-weight: bold; +} +.alert > p, +.alert > ul { + margin-bottom: 0; +} +.alert > p + p { + margin-top: 5px; +} +.alert-dismissable, +.alert-dismissible { + padding-right: 35px; +} +.alert-dismissable .close, +.alert-dismissible .close { + position: relative; + top: -2px; + right: -21px; + color: inherit; +} +.alert-success { + color: #3c763d; + background-color: #dff0d8; + border-color: #d6e9c6; +} +.alert-success hr { + border-top-color: #c9e2b3; +} +.alert-success .alert-link { + color: #2b542c; +} +.alert-info { + color: #31708f; + background-color: #d9edf7; + border-color: #bce8f1; +} +.alert-info hr { + border-top-color: #a6e1ec; +} +.alert-info .alert-link { + color: #245269; +} +.alert-warning { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #faebcc; +} +.alert-warning hr { + border-top-color: #f7e1b5; +} +.alert-warning .alert-link { + color: #66512c; +} +.alert-danger { + color: #a94442; + background-color: #f2dede; + border-color: #ebccd1; +} +.alert-danger hr { + border-top-color: #e4b9c0; +} +.alert-danger .alert-link { + color: #843534; +} +@-webkit-keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} +@-o-keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} +@keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} +.progress { + height: 20px; + margin-bottom: 20px; + overflow: hidden; + background-color: #f5f5f5; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, .1); + box-shadow: inset 0 1px 2px rgba(0, 0, 0, .1); +} +.progress-bar { + float: left; + width: 0; + height: 100%; + font-size: 12px; + line-height: 20px; + color: #fff; + text-align: center; + background-color: #337ab7; + -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .15); + box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .15); + -webkit-transition: width .6s ease; + -o-transition: width .6s ease; + transition: width .6s ease; +} +.progress-striped .progress-bar, +.progress-bar-striped { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + -webkit-background-size: 40px 40px; + background-size: 40px 40px; +} +.progress.active .progress-bar, +.progress-bar.active { + -webkit-animation: progress-bar-stripes 2s linear infinite; + -o-animation: progress-bar-stripes 2s linear infinite; + animation: progress-bar-stripes 2s linear infinite; +} +.progress-bar-success { + background-color: #5cb85c; +} +.progress-striped .progress-bar-success { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-info { + background-color: #5bc0de; +} +.progress-striped .progress-bar-info { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-warning { + background-color: #f0ad4e; +} +.progress-striped .progress-bar-warning { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-danger { + background-color: #d9534f; +} +.progress-striped .progress-bar-danger { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.media { + margin-top: 15px; +} +.media:first-child { + margin-top: 0; +} +.media, +.media-body { + overflow: hidden; + zoom: 1; +} +.media-body { + width: 10000px; +} +.media-object { + display: block; +} +.media-object.img-thumbnail { + max-width: none; +} +.media-right, +.media > .pull-right { + padding-left: 10px; +} +.media-left, +.media > .pull-left { + padding-right: 10px; +} +.media-left, +.media-right, +.media-body { + display: table-cell; + vertical-align: top; +} +.media-middle { + vertical-align: middle; +} +.media-bottom { + vertical-align: bottom; +} +.media-heading { + margin-top: 0; + margin-bottom: 5px; +} +.media-list { + padding-left: 0; + list-style: none; +} +.list-group { + padding-left: 0; + margin-bottom: 20px; +} +.list-group-item { + position: relative; + display: block; + padding: 10px 15px; + margin-bottom: -1px; + background-color: #fff; + border: 1px solid #ddd; +} +.list-group-item:first-child { + border-top-left-radius: 4px; + border-top-right-radius: 4px; +} +.list-group-item:last-child { + margin-bottom: 0; + border-bottom-right-radius: 4px; + border-bottom-left-radius: 4px; +} +a.list-group-item, +button.list-group-item { + color: #555; +} +a.list-group-item .list-group-item-heading, +button.list-group-item .list-group-item-heading { + color: #333; +} +a.list-group-item:hover, +button.list-group-item:hover, +a.list-group-item:focus, +button.list-group-item:focus { + color: #555; + text-decoration: none; + background-color: #f5f5f5; +} +button.list-group-item { + width: 100%; + text-align: left; +} +.list-group-item.disabled, +.list-group-item.disabled:hover, +.list-group-item.disabled:focus { + color: #777; + cursor: not-allowed; + background-color: #eee; +} +.list-group-item.disabled .list-group-item-heading, +.list-group-item.disabled:hover .list-group-item-heading, +.list-group-item.disabled:focus .list-group-item-heading { + color: inherit; +} +.list-group-item.disabled .list-group-item-text, +.list-group-item.disabled:hover .list-group-item-text, +.list-group-item.disabled:focus .list-group-item-text { + color: #777; +} +.list-group-item.active, +.list-group-item.active:hover, +.list-group-item.active:focus { + z-index: 2; + color: #fff; + background-color: #337ab7; + border-color: #337ab7; +} +.list-group-item.active .list-group-item-heading, +.list-group-item.active:hover .list-group-item-heading, +.list-group-item.active:focus .list-group-item-heading, +.list-group-item.active .list-group-item-heading > small, +.list-group-item.active:hover .list-group-item-heading > small, +.list-group-item.active:focus .list-group-item-heading > small, +.list-group-item.active .list-group-item-heading > .small, +.list-group-item.active:hover .list-group-item-heading > .small, +.list-group-item.active:focus .list-group-item-heading > .small { + color: inherit; +} +.list-group-item.active .list-group-item-text, +.list-group-item.active:hover .list-group-item-text, +.list-group-item.active:focus .list-group-item-text { + color: #c7ddef; +} +.list-group-item-success { + color: #3c763d; + background-color: #dff0d8; +} +a.list-group-item-success, +button.list-group-item-success { + color: #3c763d; +} +a.list-group-item-success .list-group-item-heading, +button.list-group-item-success .list-group-item-heading { + color: inherit; +} +a.list-group-item-success:hover, +button.list-group-item-success:hover, +a.list-group-item-success:focus, +button.list-group-item-success:focus { + color: #3c763d; + background-color: #d0e9c6; +} +a.list-group-item-success.active, +button.list-group-item-success.active, +a.list-group-item-success.active:hover, +button.list-group-item-success.active:hover, +a.list-group-item-success.active:focus, +button.list-group-item-success.active:focus { + color: #fff; + background-color: #3c763d; + border-color: #3c763d; +} +.list-group-item-info { + color: #31708f; + background-color: #d9edf7; +} +a.list-group-item-info, +button.list-group-item-info { + color: #31708f; +} +a.list-group-item-info .list-group-item-heading, +button.list-group-item-info .list-group-item-heading { + color: inherit; +} +a.list-group-item-info:hover, +button.list-group-item-info:hover, +a.list-group-item-info:focus, +button.list-group-item-info:focus { + color: #31708f; + background-color: #c4e3f3; +} +a.list-group-item-info.active, +button.list-group-item-info.active, +a.list-group-item-info.active:hover, +button.list-group-item-info.active:hover, +a.list-group-item-info.active:focus, +button.list-group-item-info.active:focus { + color: #fff; + background-color: #31708f; + border-color: #31708f; +} +.list-group-item-warning { + color: #8a6d3b; + background-color: #fcf8e3; +} +a.list-group-item-warning, +button.list-group-item-warning { + color: #8a6d3b; +} +a.list-group-item-warning .list-group-item-heading, +button.list-group-item-warning .list-group-item-heading { + color: inherit; +} +a.list-group-item-warning:hover, +button.list-group-item-warning:hover, +a.list-group-item-warning:focus, +button.list-group-item-warning:focus { + color: #8a6d3b; + background-color: #faf2cc; +} +a.list-group-item-warning.active, +button.list-group-item-warning.active, +a.list-group-item-warning.active:hover, +button.list-group-item-warning.active:hover, +a.list-group-item-warning.active:focus, +button.list-group-item-warning.active:focus { + color: #fff; + background-color: #8a6d3b; + border-color: #8a6d3b; +} +.list-group-item-danger { + color: #a94442; + background-color: #f2dede; +} +a.list-group-item-danger, +button.list-group-item-danger { + color: #a94442; +} +a.list-group-item-danger .list-group-item-heading, +button.list-group-item-danger .list-group-item-heading { + color: inherit; +} +a.list-group-item-danger:hover, +button.list-group-item-danger:hover, +a.list-group-item-danger:focus, +button.list-group-item-danger:focus { + color: #a94442; + background-color: #ebcccc; +} +a.list-group-item-danger.active, +button.list-group-item-danger.active, +a.list-group-item-danger.active:hover, +button.list-group-item-danger.active:hover, +a.list-group-item-danger.active:focus, +button.list-group-item-danger.active:focus { + color: #fff; + background-color: #a94442; + border-color: #a94442; +} +.list-group-item-heading { + margin-top: 0; + margin-bottom: 5px; +} +.list-group-item-text { + margin-bottom: 0; + line-height: 1.3; +} +.panel { + margin-bottom: 20px; + background-color: #fff; + border: 1px solid transparent; + border-radius: 4px; + -webkit-box-shadow: 0 1px 1px rgba(0, 0, 0, .05); + box-shadow: 0 1px 1px rgba(0, 0, 0, .05); +} +.panel-body { + padding: 15px; +} +.panel-heading { + padding: 10px 15px; + border-bottom: 1px solid transparent; + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel-heading > .dropdown .dropdown-toggle { + color: inherit; +} +.panel-title { + margin-top: 0; + margin-bottom: 0; + font-size: 16px; + color: inherit; +} +.panel-title > a, +.panel-title > small, +.panel-title > .small, +.panel-title > small > a, +.panel-title > .small > a { + color: inherit; +} +.panel-footer { + padding: 10px 15px; + background-color: #f5f5f5; + border-top: 1px solid #ddd; + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .list-group, +.panel > .panel-collapse > .list-group { + margin-bottom: 0; +} +.panel > .list-group .list-group-item, +.panel > .panel-collapse > .list-group .list-group-item { + border-width: 1px 0; + border-radius: 0; +} +.panel > .list-group:first-child .list-group-item:first-child, +.panel > .panel-collapse > .list-group:first-child .list-group-item:first-child { + border-top: 0; + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel > .list-group:last-child .list-group-item:last-child, +.panel > .panel-collapse > .list-group:last-child .list-group-item:last-child { + border-bottom: 0; + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .panel-heading + .panel-collapse > .list-group .list-group-item:first-child { + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.panel-heading + .list-group .list-group-item:first-child { + border-top-width: 0; +} +.list-group + .panel-footer { + border-top-width: 0; +} +.panel > .table, +.panel > .table-responsive > .table, +.panel > .panel-collapse > .table { + margin-bottom: 0; +} +.panel > .table caption, +.panel > .table-responsive > .table caption, +.panel > .panel-collapse > .table caption { + padding-right: 15px; + padding-left: 15px; +} +.panel > .table:first-child, +.panel > .table-responsive:first-child > .table:first-child { + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel > .table:first-child > thead:first-child > tr:first-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child, +.panel > .table:first-child > tbody:first-child > tr:first-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child { + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel > .table:first-child > thead:first-child > tr:first-child td:first-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:first-child, +.panel > .table:first-child > tbody:first-child > tr:first-child td:first-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:first-child, +.panel > .table:first-child > thead:first-child > tr:first-child th:first-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:first-child, +.panel > .table:first-child > tbody:first-child > tr:first-child th:first-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:first-child { + border-top-left-radius: 3px; +} +.panel > .table:first-child > thead:first-child > tr:first-child td:last-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:last-child, +.panel > .table:first-child > tbody:first-child > tr:first-child td:last-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:last-child, +.panel > .table:first-child > thead:first-child > tr:first-child th:last-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:last-child, +.panel > .table:first-child > tbody:first-child > tr:first-child th:last-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:last-child { + border-top-right-radius: 3px; +} +.panel > .table:last-child, +.panel > .table-responsive:last-child > .table:last-child { + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .table:last-child > tbody:last-child > tr:last-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child { + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .table:last-child > tbody:last-child > tr:last-child td:first-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:first-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child td:first-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:first-child, +.panel > .table:last-child > tbody:last-child > tr:last-child th:first-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:first-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child th:first-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:first-child { + border-bottom-left-radius: 3px; +} +.panel > .table:last-child > tbody:last-child > tr:last-child td:last-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:last-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child td:last-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:last-child, +.panel > .table:last-child > tbody:last-child > tr:last-child th:last-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:last-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child th:last-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:last-child { + border-bottom-right-radius: 3px; +} +.panel > .panel-body + .table, +.panel > .panel-body + .table-responsive, +.panel > .table + .panel-body, +.panel > .table-responsive + .panel-body { + border-top: 1px solid #ddd; +} +.panel > .table > tbody:first-child > tr:first-child th, +.panel > .table > tbody:first-child > tr:first-child td { + border-top: 0; +} +.panel > .table-bordered, +.panel > .table-responsive > .table-bordered { + border: 0; +} +.panel > .table-bordered > thead > tr > th:first-child, +.panel > .table-responsive > .table-bordered > thead > tr > th:first-child, +.panel > .table-bordered > tbody > tr > th:first-child, +.panel > .table-responsive > .table-bordered > tbody > tr > th:first-child, +.panel > .table-bordered > tfoot > tr > th:first-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > th:first-child, +.panel > .table-bordered > thead > tr > td:first-child, +.panel > .table-responsive > .table-bordered > thead > tr > td:first-child, +.panel > .table-bordered > tbody > tr > td:first-child, +.panel > .table-responsive > .table-bordered > tbody > tr > td:first-child, +.panel > .table-bordered > tfoot > tr > td:first-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > td:first-child { + border-left: 0; +} +.panel > .table-bordered > thead > tr > th:last-child, +.panel > .table-responsive > .table-bordered > thead > tr > th:last-child, +.panel > .table-bordered > tbody > tr > th:last-child, +.panel > .table-responsive > .table-bordered > tbody > tr > th:last-child, +.panel > .table-bordered > tfoot > tr > th:last-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > th:last-child, +.panel > .table-bordered > thead > tr > td:last-child, +.panel > .table-responsive > .table-bordered > thead > tr > td:last-child, +.panel > .table-bordered > tbody > tr > td:last-child, +.panel > .table-responsive > .table-bordered > tbody > tr > td:last-child, +.panel > .table-bordered > tfoot > tr > td:last-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > td:last-child { + border-right: 0; +} +.panel > .table-bordered > thead > tr:first-child > td, +.panel > .table-responsive > .table-bordered > thead > tr:first-child > td, +.panel > .table-bordered > tbody > tr:first-child > td, +.panel > .table-responsive > .table-bordered > tbody > tr:first-child > td, +.panel > .table-bordered > thead > tr:first-child > th, +.panel > .table-responsive > .table-bordered > thead > tr:first-child > th, +.panel > .table-bordered > tbody > tr:first-child > th, +.panel > .table-responsive > .table-bordered > tbody > tr:first-child > th { + border-bottom: 0; +} +.panel > .table-bordered > tbody > tr:last-child > td, +.panel > .table-responsive > .table-bordered > tbody > tr:last-child > td, +.panel > .table-bordered > tfoot > tr:last-child > td, +.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > td, +.panel > .table-bordered > tbody > tr:last-child > th, +.panel > .table-responsive > .table-bordered > tbody > tr:last-child > th, +.panel > .table-bordered > tfoot > tr:last-child > th, +.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > th { + border-bottom: 0; +} +.panel > .table-responsive { + margin-bottom: 0; + border: 0; +} +.panel-group { + margin-bottom: 20px; +} +.panel-group .panel { + margin-bottom: 0; + border-radius: 4px; +} +.panel-group .panel + .panel { + margin-top: 5px; +} +.panel-group .panel-heading { + border-bottom: 0; +} +.panel-group .panel-heading + .panel-collapse > .panel-body, +.panel-group .panel-heading + .panel-collapse > .list-group { + border-top: 1px solid #ddd; +} +.panel-group .panel-footer { + border-top: 0; +} +.panel-group .panel-footer + .panel-collapse .panel-body { + border-bottom: 1px solid #ddd; +} +.panel-default { + border-color: #ddd; +} +.panel-default > .panel-heading { + color: #333; + background-color: #f5f5f5; + border-color: #ddd; +} +.panel-default > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #ddd; +} +.panel-default > .panel-heading .badge { + color: #f5f5f5; + background-color: #333; +} +.panel-default > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #ddd; +} +.panel-primary { + border-color: #337ab7; +} +.panel-primary > .panel-heading { + color: #fff; + background-color: #337ab7; + border-color: #337ab7; +} +.panel-primary > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #337ab7; +} +.panel-primary > .panel-heading .badge { + color: #337ab7; + background-color: #fff; +} +.panel-primary > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #337ab7; +} +.panel-success { + border-color: #d6e9c6; +} +.panel-success > .panel-heading { + color: #3c763d; + background-color: #dff0d8; + border-color: #d6e9c6; +} +.panel-success > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #d6e9c6; +} +.panel-success > .panel-heading .badge { + color: #dff0d8; + background-color: #3c763d; +} +.panel-success > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #d6e9c6; +} +.panel-info { + border-color: #bce8f1; +} +.panel-info > .panel-heading { + color: #31708f; + background-color: #d9edf7; + border-color: #bce8f1; +} +.panel-info > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #bce8f1; +} +.panel-info > .panel-heading .badge { + color: #d9edf7; + background-color: #31708f; +} +.panel-info > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #bce8f1; +} +.panel-warning { + border-color: #faebcc; +} +.panel-warning > .panel-heading { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #faebcc; +} +.panel-warning > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #faebcc; +} +.panel-warning > .panel-heading .badge { + color: #fcf8e3; + background-color: #8a6d3b; +} +.panel-warning > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #faebcc; +} +.panel-danger { + border-color: #ebccd1; +} +.panel-danger > .panel-heading { + color: #a94442; + background-color: #f2dede; + border-color: #ebccd1; +} +.panel-danger > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #ebccd1; +} +.panel-danger > .panel-heading .badge { + color: #f2dede; + background-color: #a94442; +} +.panel-danger > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #ebccd1; +} +.embed-responsive { + position: relative; + display: block; + height: 0; + padding: 0; + overflow: hidden; +} +.embed-responsive .embed-responsive-item, +.embed-responsive iframe, +.embed-responsive embed, +.embed-responsive object, +.embed-responsive video { + position: absolute; + top: 0; + bottom: 0; + left: 0; + width: 100%; + height: 100%; + border: 0; +} +.embed-responsive-16by9 { + padding-bottom: 56.25%; +} +.embed-responsive-4by3 { + padding-bottom: 75%; +} +.well { + min-height: 20px; + padding: 19px; + margin-bottom: 20px; + background-color: #f5f5f5; + border: 1px solid #e3e3e3; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .05); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .05); +} +.well blockquote { + border-color: #ddd; + border-color: rgba(0, 0, 0, .15); +} +.well-lg { + padding: 24px; + border-radius: 6px; +} +.well-sm { + padding: 9px; + border-radius: 3px; +} +.close { + float: right; + font-size: 21px; + font-weight: bold; + line-height: 1; + color: #000; + text-shadow: 0 1px 0 #fff; + filter: alpha(opacity=20); + opacity: .2; +} +.close:hover, +.close:focus { + color: #000; + text-decoration: none; + cursor: pointer; + filter: alpha(opacity=50); + opacity: .5; +} +button.close { + -webkit-appearance: none; + padding: 0; + cursor: pointer; + background: transparent; + border: 0; +} +.modal-open { + overflow: hidden; +} +.modal { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 1050; + display: none; + overflow: hidden; + -webkit-overflow-scrolling: touch; + outline: 0; +} +.modal.fade .modal-dialog { + -webkit-transition: -webkit-transform .3s ease-out; + -o-transition: -o-transform .3s ease-out; + transition: transform .3s ease-out; + -webkit-transform: translate(0, -25%); + -ms-transform: translate(0, -25%); + -o-transform: translate(0, -25%); + transform: translate(0, -25%); +} +.modal.in .modal-dialog { + -webkit-transform: translate(0, 0); + -ms-transform: translate(0, 0); + -o-transform: translate(0, 0); + transform: translate(0, 0); +} +.modal-open .modal { + overflow-x: hidden; + overflow-y: auto; +} +.modal-dialog { + position: relative; + width: auto; + margin: 10px; +} +.modal-content { + position: relative; + background-color: #fff; + -webkit-background-clip: padding-box; + background-clip: padding-box; + border: 1px solid #999; + border: 1px solid rgba(0, 0, 0, .2); + border-radius: 6px; + outline: 0; + -webkit-box-shadow: 0 3px 9px rgba(0, 0, 0, .5); + box-shadow: 0 3px 9px rgba(0, 0, 0, .5); +} +.modal-backdrop { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 1040; + background-color: #000; +} +.modal-backdrop.fade { + filter: alpha(opacity=0); + opacity: 0; +} +.modal-backdrop.in { + filter: alpha(opacity=50); + opacity: .5; +} +.modal-header { + min-height: 16.42857143px; + padding: 15px; + border-bottom: 1px solid #e5e5e5; +} +.modal-header .close { + margin-top: -2px; +} +.modal-title { + margin: 0; + line-height: 1.42857143; +} +.modal-body { + position: relative; + padding: 15px; +} +.modal-footer { + padding: 15px; + text-align: right; + border-top: 1px solid #e5e5e5; +} +.modal-footer .btn + .btn { + margin-bottom: 0; + margin-left: 5px; +} +.modal-footer .btn-group .btn + .btn { + margin-left: -1px; +} +.modal-footer .btn-block + .btn-block { + margin-left: 0; +} +.modal-scrollbar-measure { + position: absolute; + top: -9999px; + width: 50px; + height: 50px; + overflow: scroll; +} +@media (min-width: 768px) { + .modal-dialog { + width: 600px; + margin: 30px auto; + } + .modal-content { + -webkit-box-shadow: 0 5px 15px rgba(0, 0, 0, .5); + box-shadow: 0 5px 15px rgba(0, 0, 0, .5); + } + .modal-sm { + width: 300px; + } +} +@media (min-width: 992px) { + .modal-lg { + width: 900px; + } +} +.tooltip { + position: absolute; + z-index: 1070; + display: block; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 12px; + font-style: normal; + font-weight: normal; + line-height: 1.42857143; + text-align: left; + text-align: start; + text-decoration: none; + text-shadow: none; + text-transform: none; + letter-spacing: normal; + word-break: normal; + word-spacing: normal; + word-wrap: normal; + white-space: normal; + filter: alpha(opacity=0); + opacity: 0; + + line-break: auto; +} +.tooltip.in { + filter: alpha(opacity=90); + opacity: .9; +} +.tooltip.top { + padding: 5px 0; + margin-top: -3px; +} +.tooltip.right { + padding: 0 5px; + margin-left: 3px; +} +.tooltip.bottom { + padding: 5px 0; + margin-top: 3px; +} +.tooltip.left { + padding: 0 5px; + margin-left: -3px; +} +.tooltip-inner { + max-width: 200px; + padding: 3px 8px; + color: #fff; + text-align: center; + background-color: #000; + border-radius: 4px; +} +.tooltip-arrow { + position: absolute; + width: 0; + height: 0; + border-color: transparent; + border-style: solid; +} +.tooltip.top .tooltip-arrow { + bottom: 0; + left: 50%; + margin-left: -5px; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.top-left .tooltip-arrow { + right: 5px; + bottom: 0; + margin-bottom: -5px; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.top-right .tooltip-arrow { + bottom: 0; + left: 5px; + margin-bottom: -5px; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.right .tooltip-arrow { + top: 50%; + left: 0; + margin-top: -5px; + border-width: 5px 5px 5px 0; + border-right-color: #000; +} +.tooltip.left .tooltip-arrow { + top: 50%; + right: 0; + margin-top: -5px; + border-width: 5px 0 5px 5px; + border-left-color: #000; +} +.tooltip.bottom .tooltip-arrow { + top: 0; + left: 50%; + margin-left: -5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.tooltip.bottom-left .tooltip-arrow { + top: 0; + right: 5px; + margin-top: -5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.tooltip.bottom-right .tooltip-arrow { + top: 0; + left: 5px; + margin-top: -5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.popover { + position: absolute; + top: 0; + left: 0; + z-index: 1060; + display: none; + max-width: 276px; + padding: 1px; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 14px; + font-style: normal; + font-weight: normal; + line-height: 1.42857143; + text-align: left; + text-align: start; + text-decoration: none; + text-shadow: none; + text-transform: none; + letter-spacing: normal; + word-break: normal; + word-spacing: normal; + word-wrap: normal; + white-space: normal; + background-color: #fff; + -webkit-background-clip: padding-box; + background-clip: padding-box; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, .2); + border-radius: 6px; + -webkit-box-shadow: 0 5px 10px rgba(0, 0, 0, .2); + box-shadow: 0 5px 10px rgba(0, 0, 0, .2); + + line-break: auto; +} +.popover.top { + margin-top: -10px; +} +.popover.right { + margin-left: 10px; +} +.popover.bottom { + margin-top: 10px; +} +.popover.left { + margin-left: -10px; +} +.popover-title { + padding: 8px 14px; + margin: 0; + font-size: 14px; + background-color: #f7f7f7; + border-bottom: 1px solid #ebebeb; + border-radius: 5px 5px 0 0; +} +.popover-content { + padding: 9px 14px; +} +.popover > .arrow, +.popover > .arrow:after { + position: absolute; + display: block; + width: 0; + height: 0; + border-color: transparent; + border-style: solid; +} +.popover > .arrow { + border-width: 11px; +} +.popover > .arrow:after { + content: ""; + border-width: 10px; +} +.popover.top > .arrow { + bottom: -11px; + left: 50%; + margin-left: -11px; + border-top-color: #999; + border-top-color: rgba(0, 0, 0, .25); + border-bottom-width: 0; +} +.popover.top > .arrow:after { + bottom: 1px; + margin-left: -10px; + content: " "; + border-top-color: #fff; + border-bottom-width: 0; +} +.popover.right > .arrow { + top: 50%; + left: -11px; + margin-top: -11px; + border-right-color: #999; + border-right-color: rgba(0, 0, 0, .25); + border-left-width: 0; +} +.popover.right > .arrow:after { + bottom: -10px; + left: 1px; + content: " "; + border-right-color: #fff; + border-left-width: 0; +} +.popover.bottom > .arrow { + top: -11px; + left: 50%; + margin-left: -11px; + border-top-width: 0; + border-bottom-color: #999; + border-bottom-color: rgba(0, 0, 0, .25); +} +.popover.bottom > .arrow:after { + top: 1px; + margin-left: -10px; + content: " "; + border-top-width: 0; + border-bottom-color: #fff; +} +.popover.left > .arrow { + top: 50%; + right: -11px; + margin-top: -11px; + border-right-width: 0; + border-left-color: #999; + border-left-color: rgba(0, 0, 0, .25); +} +.popover.left > .arrow:after { + right: 1px; + bottom: -10px; + content: " "; + border-right-width: 0; + border-left-color: #fff; +} +.carousel { + position: relative; +} +.carousel-inner { + position: relative; + width: 100%; + overflow: hidden; +} +.carousel-inner > .item { + position: relative; + display: none; + -webkit-transition: .6s ease-in-out left; + -o-transition: .6s ease-in-out left; + transition: .6s ease-in-out left; +} +.carousel-inner > .item > img, +.carousel-inner > .item > a > img { + line-height: 1; +} +@media all and (transform-3d), (-webkit-transform-3d) { + .carousel-inner > .item { + -webkit-transition: -webkit-transform .6s ease-in-out; + -o-transition: -o-transform .6s ease-in-out; + transition: transform .6s ease-in-out; + + -webkit-backface-visibility: hidden; + backface-visibility: hidden; + -webkit-perspective: 1000px; + perspective: 1000px; + } + .carousel-inner > .item.next, + .carousel-inner > .item.active.right { + left: 0; + -webkit-transform: translate3d(100%, 0, 0); + transform: translate3d(100%, 0, 0); + } + .carousel-inner > .item.prev, + .carousel-inner > .item.active.left { + left: 0; + -webkit-transform: translate3d(-100%, 0, 0); + transform: translate3d(-100%, 0, 0); + } + .carousel-inner > .item.next.left, + .carousel-inner > .item.prev.right, + .carousel-inner > .item.active { + left: 0; + -webkit-transform: translate3d(0, 0, 0); + transform: translate3d(0, 0, 0); + } +} +.carousel-inner > .active, +.carousel-inner > .next, +.carousel-inner > .prev { + display: block; +} +.carousel-inner > .active { + left: 0; +} +.carousel-inner > .next, +.carousel-inner > .prev { + position: absolute; + top: 0; + width: 100%; +} +.carousel-inner > .next { + left: 100%; +} +.carousel-inner > .prev { + left: -100%; +} +.carousel-inner > .next.left, +.carousel-inner > .prev.right { + left: 0; +} +.carousel-inner > .active.left { + left: -100%; +} +.carousel-inner > .active.right { + left: 100%; +} +.carousel-control { + position: absolute; + top: 0; + bottom: 0; + left: 0; + width: 15%; + font-size: 20px; + color: #fff; + text-align: center; + text-shadow: 0 1px 2px rgba(0, 0, 0, .6); + filter: alpha(opacity=50); + opacity: .5; +} +.carousel-control.left { + background-image: -webkit-linear-gradient(left, rgba(0, 0, 0, .5) 0%, rgba(0, 0, 0, .0001) 100%); + background-image: -o-linear-gradient(left, rgba(0, 0, 0, .5) 0%, rgba(0, 0, 0, .0001) 100%); + background-image: -webkit-gradient(linear, left top, right top, from(rgba(0, 0, 0, .5)), to(rgba(0, 0, 0, .0001))); + background-image: linear-gradient(to right, rgba(0, 0, 0, .5) 0%, rgba(0, 0, 0, .0001) 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#80000000', endColorstr='#00000000', GradientType=1); + background-repeat: repeat-x; +} +.carousel-control.right { + right: 0; + left: auto; + background-image: -webkit-linear-gradient(left, rgba(0, 0, 0, .0001) 0%, rgba(0, 0, 0, .5) 100%); + background-image: -o-linear-gradient(left, rgba(0, 0, 0, .0001) 0%, rgba(0, 0, 0, .5) 100%); + background-image: -webkit-gradient(linear, left top, right top, from(rgba(0, 0, 0, .0001)), to(rgba(0, 0, 0, .5))); + background-image: linear-gradient(to right, rgba(0, 0, 0, .0001) 0%, rgba(0, 0, 0, .5) 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#80000000', GradientType=1); + background-repeat: repeat-x; +} +.carousel-control:hover, +.carousel-control:focus { + color: #fff; + text-decoration: none; + filter: alpha(opacity=90); + outline: 0; + opacity: .9; +} +.carousel-control .icon-prev, +.carousel-control .icon-next, +.carousel-control .glyphicon-chevron-left, +.carousel-control .glyphicon-chevron-right { + position: absolute; + top: 50%; + z-index: 5; + display: inline-block; + margin-top: -10px; +} +.carousel-control .icon-prev, +.carousel-control .glyphicon-chevron-left { + left: 50%; + margin-left: -10px; +} +.carousel-control .icon-next, +.carousel-control .glyphicon-chevron-right { + right: 50%; + margin-right: -10px; +} +.carousel-control .icon-prev, +.carousel-control .icon-next { + width: 20px; + height: 20px; + font-family: serif; + line-height: 1; +} +.carousel-control .icon-prev:before { + content: '\2039'; +} +.carousel-control .icon-next:before { + content: '\203a'; +} +.carousel-indicators { + position: absolute; + bottom: 10px; + left: 50%; + z-index: 15; + width: 60%; + padding-left: 0; + margin-left: -30%; + text-align: center; + list-style: none; +} +.carousel-indicators li { + display: inline-block; + width: 10px; + height: 10px; + margin: 1px; + text-indent: -999px; + cursor: pointer; + background-color: #000 \9; + background-color: rgba(0, 0, 0, 0); + border: 1px solid #fff; + border-radius: 10px; +} +.carousel-indicators .active { + width: 12px; + height: 12px; + margin: 0; + background-color: #fff; +} +.carousel-caption { + position: absolute; + right: 15%; + bottom: 20px; + left: 15%; + z-index: 10; + padding-top: 20px; + padding-bottom: 20px; + color: #fff; + text-align: center; + text-shadow: 0 1px 2px rgba(0, 0, 0, .6); +} +.carousel-caption .btn { + text-shadow: none; +} +@media screen and (min-width: 768px) { + .carousel-control .glyphicon-chevron-left, + .carousel-control .glyphicon-chevron-right, + .carousel-control .icon-prev, + .carousel-control .icon-next { + width: 30px; + height: 30px; + margin-top: -15px; + font-size: 30px; + } + .carousel-control .glyphicon-chevron-left, + .carousel-control .icon-prev { + margin-left: -15px; + } + .carousel-control .glyphicon-chevron-right, + .carousel-control .icon-next { + margin-right: -15px; + } + .carousel-caption { + right: 20%; + left: 20%; + padding-bottom: 30px; + } + .carousel-indicators { + bottom: 20px; + } +} +.clearfix:before, +.clearfix:after, +.dl-horizontal dd:before, +.dl-horizontal dd:after, +.container:before, +.container:after, +.container-fluid:before, +.container-fluid:after, +.row:before, +.row:after, +.form-horizontal .form-group:before, +.form-horizontal .form-group:after, +.btn-toolbar:before, +.btn-toolbar:after, +.btn-group-vertical > .btn-group:before, +.btn-group-vertical > .btn-group:after, +.nav:before, +.nav:after, +.navbar:before, +.navbar:after, +.navbar-header:before, +.navbar-header:after, +.navbar-collapse:before, +.navbar-collapse:after, +.pager:before, +.pager:after, +.panel-body:before, +.panel-body:after, +.modal-footer:before, +.modal-footer:after { + display: table; + content: " "; +} +.clearfix:after, +.dl-horizontal dd:after, +.container:after, +.container-fluid:after, +.row:after, +.form-horizontal .form-group:after, +.btn-toolbar:after, +.btn-group-vertical > .btn-group:after, +.nav:after, +.navbar:after, +.navbar-header:after, +.navbar-collapse:after, +.pager:after, +.panel-body:after, +.modal-footer:after { + clear: both; +} +.center-block { + display: block; + margin-right: auto; + margin-left: auto; +} +.pull-right { + float: right !important; +} +.pull-left { + float: left !important; +} +.hide { + display: none !important; +} +.show { + display: block !important; +} +.invisible { + visibility: hidden; +} +.text-hide { + font: 0/0 a; + color: transparent; + text-shadow: none; + background-color: transparent; + border: 0; +} +.hidden { + display: none !important; +} +.affix { + position: fixed; +} +@-ms-viewport { + width: device-width; +} +.visible-xs, +.visible-sm, +.visible-md, +.visible-lg { + display: none !important; +} +.visible-xs-block, +.visible-xs-inline, +.visible-xs-inline-block, +.visible-sm-block, +.visible-sm-inline, +.visible-sm-inline-block, +.visible-md-block, +.visible-md-inline, +.visible-md-inline-block, +.visible-lg-block, +.visible-lg-inline, +.visible-lg-inline-block { + display: none !important; +} +@media (max-width: 767px) { + .visible-xs { + display: block !important; + } + table.visible-xs { + display: table !important; + } + tr.visible-xs { + display: table-row !important; + } + th.visible-xs, + td.visible-xs { + display: table-cell !important; + } +} +@media (max-width: 767px) { + .visible-xs-block { + display: block !important; + } +} +@media (max-width: 767px) { + .visible-xs-inline { + display: inline !important; + } +} +@media (max-width: 767px) { + .visible-xs-inline-block { + display: inline-block !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm { + display: block !important; + } + table.visible-sm { + display: table !important; + } + tr.visible-sm { + display: table-row !important; + } + th.visible-sm, + td.visible-sm { + display: table-cell !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm-block { + display: block !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm-inline { + display: inline !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm-inline-block { + display: inline-block !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md { + display: block !important; + } + table.visible-md { + display: table !important; + } + tr.visible-md { + display: table-row !important; + } + th.visible-md, + td.visible-md { + display: table-cell !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md-block { + display: block !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md-inline { + display: inline !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md-inline-block { + display: inline-block !important; + } +} +@media (min-width: 1200px) { + .visible-lg { + display: block !important; + } + table.visible-lg { + display: table !important; + } + tr.visible-lg { + display: table-row !important; + } + th.visible-lg, + td.visible-lg { + display: table-cell !important; + } +} +@media (min-width: 1200px) { + .visible-lg-block { + display: block !important; + } +} +@media (min-width: 1200px) { + .visible-lg-inline { + display: inline !important; + } +} +@media (min-width: 1200px) { + .visible-lg-inline-block { + display: inline-block !important; + } +} +@media (max-width: 767px) { + .hidden-xs { + display: none !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .hidden-sm { + display: none !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .hidden-md { + display: none !important; + } +} +@media (min-width: 1200px) { + .hidden-lg { + display: none !important; + } +} +.visible-print { + display: none !important; +} +@media print { + .visible-print { + display: block !important; + } + table.visible-print { + display: table !important; + } + tr.visible-print { + display: table-row !important; + } + th.visible-print, + td.visible-print { + display: table-cell !important; + } +} +.visible-print-block { + display: none !important; +} +@media print { + .visible-print-block { + display: block !important; + } +} +.visible-print-inline { + display: none !important; +} +@media print { + .visible-print-inline { + display: inline !important; + } +} +.visible-print-inline-block { + display: none !important; +} +@media print { + .visible-print-inline-block { + display: inline-block !important; + } +} +@media print { + .hidden-print { + display: none !important; + } +} +/*# sourceMappingURL=bootstrap.css.map */ diff --git a/docs/assets/css/bootstrap.css.map b/docs/assets/css/bootstrap.css.map new file mode 100644 index 00000000000..9f60ed2b1bd --- /dev/null +++ b/docs/assets/css/bootstrap.css.map @@ -0,0 +1 @@ +{"version":3,"sources":["bootstrap.css","less/normalize.less","less/print.less","less/glyphicons.less","less/scaffolding.less","less/mixins/vendor-prefixes.less","less/mixins/tab-focus.less","less/mixins/image.less","less/type.less","less/mixins/text-emphasis.less","less/mixins/background-variant.less","less/mixins/text-overflow.less","less/code.less","less/grid.less","less/mixins/grid.less","less/mixins/grid-framework.less","less/tables.less","less/mixins/table-row.less","less/forms.less","less/mixins/forms.less","less/buttons.less","less/mixins/buttons.less","less/mixins/opacity.less","less/component-animations.less","less/dropdowns.less","less/mixins/nav-divider.less","less/mixins/reset-filter.less","less/button-groups.less","less/mixins/border-radius.less","less/input-groups.less","less/navs.less","less/navbar.less","less/mixins/nav-vertical-align.less","less/utilities.less","less/breadcrumbs.less","less/pagination.less","less/mixins/pagination.less","less/pager.less","less/labels.less","less/mixins/labels.less","less/badges.less","less/jumbotron.less","less/thumbnails.less","less/alerts.less","less/mixins/alerts.less","less/progress-bars.less","less/mixins/gradients.less","less/mixins/progress-bar.less","less/media.less","less/list-group.less","less/mixins/list-group.less","less/panels.less","less/mixins/panels.less","less/responsive-embed.less","less/wells.less","less/close.less","less/modals.less","less/tooltip.less","less/mixins/reset-text.less","less/popovers.less","less/carousel.less","less/mixins/clearfix.less","less/mixins/center-block.less","less/mixins/hide-text.less","less/responsive-utilities.less","less/mixins/responsive-visibility.less"],"names":[],"mappings":"AAAA;;;;GAIG;AACH,4EAA4E;ACG5E;EACE,wBAAA;EACA,2BAAA;EACA,+BAAA;CDDD;ACQD;EACE,UAAA;CDND;ACmBD;;;;;;;;;;;;;EAaE,eAAA;CDjBD;ACyBD;;;;EAIE,sBAAA;EACA,yBAAA;CDvBD;AC+BD;EACE,cAAA;EACA,UAAA;CD7BD;ACqCD;;EAEE,cAAA;CDnCD;AC6CD;EACE,8BAAA;CD3CD;ACmDD;;EAEE,WAAA;CDjDD;AC2DD;EACE,0BAAA;CDzDD;ACgED;;EAEE,kBAAA;CD9DD;ACqED;EACE,mBAAA;CDnED;AC2ED;EACE,eAAA;EACA,iBAAA;CDzED;ACgFD;EACE,iBAAA;EACA,YAAA;CD9ED;ACqFD;EACE,eAAA;CDnFD;AC0FD;;EAEE,eAAA;EACA,eAAA;EACA,mBAAA;EACA,yBAAA;CDxFD;AC2FD;EACE,YAAA;CDzFD;AC4FD;EACE,gBAAA;CD1FD;ACoGD;EACE,UAAA;CDlGD;ACyGD;EACE,iBAAA;CDvGD;ACiHD;EACE,iBAAA;CD/GD;ACsHD;EACE,gCAAA;KAAA,6BAAA;UAAA,wBAAA;EACA,UAAA;CDpHD;AC2HD;EACE,eAAA;CDzHD;ACgID;;;;EAIE,kCAAA;EACA,eAAA;CD9HD;ACgJD;;;;;EAKE,eAAA;EACA,cAAA;EACA,UAAA;CD9ID;ACqJD;EACE,kBAAA;CDnJD;AC6JD;;EAEE,qBAAA;CD3JD;ACsKD;;;;EAIE,2BAAA;EACA,gBAAA;CDpKD;AC2KD;;EAEE,gBAAA;CDzKD;ACgLD;;EAEE,UAAA;EACA,WAAA;CD9KD;ACsLD;EACE,oBAAA;CDpLD;AC+LD;;EAEE,+BAAA;KAAA,4BAAA;UAAA,uBAAA;EACA,WAAA;CD7LD;ACsMD;;EAEE,aAAA;CDpMD;AC4MD;EACE,8BAAA;EACA,gCAAA;KAAA,6BAAA;UAAA,wBAAA;CD1MD;ACmND;;EAEE,yBAAA;CDjND;ACwND;EACE,0BAAA;EACA,cAAA;EACA,+BAAA;CDtND;AC8ND;EACE,UAAA;EACA,WAAA;CD5ND;ACmOD;EACE,eAAA;CDjOD;ACyOD;EACE,kBAAA;CDvOD;ACiPD;EACE,0BAAA;EACA,kBAAA;CD/OD;ACkPD;;EAEE,WAAA;CDhPD;AACD,qFAAqF;AElFrF;EA7FI;;;IAGI,mCAAA;IACA,uBAAA;IACA,oCAAA;YAAA,4BAAA;IACA,6BAAA;GFkLL;EE/KC;;IAEI,2BAAA;GFiLL;EE9KC;IACI,6BAAA;GFgLL;EE7KC;IACI,8BAAA;GF+KL;EE1KC;;IAEI,YAAA;GF4KL;EEzKC;;IAEI,uBAAA;IACA,yBAAA;GF2KL;EExKC;IACI,4BAAA;GF0KL;EEvKC;;IAEI,yBAAA;GFyKL;EEtKC;IACI,2BAAA;GFwKL;EErKC;;;IAGI,WAAA;IACA,UAAA;GFuKL;EEpKC;;IAEI,wBAAA;GFsKL;EEhKC;IACI,cAAA;GFkKL;EEhKC;;IAGQ,kCAAA;GFiKT;EE9JC;IACI,uBAAA;GFgKL;EE7JC;IACI,qCAAA;GF+JL;EEhKC;;IAKQ,kCAAA;GF+JT;EE5JC;;IAGQ,kCAAA;GF6JT;CACF;AGnPD;EACE,oCAAA;EACA,sDAAA;EACA,gYAAA;CHqPD;AG7OD;EACE,mBAAA;EACA,SAAA;EACA,sBAAA;EACA,oCAAA;EACA,mBAAA;EACA,oBAAA;EACA,eAAA;EACA,oCAAA;EACA,mCAAA;CH+OD;AG3OmC;EAAW,eAAA;CH8O9C;AG7OmC;EAAW,eAAA;CHgP9C;AG9OmC;;EAAW,iBAAA;CHkP9C;AGjPmC;EAAW,iBAAA;CHoP9C;AGnPmC;EAAW,iBAAA;CHsP9C;AGrPmC;EAAW,iBAAA;CHwP9C;AGvPmC;EAAW,iBAAA;CH0P9C;AGzPmC;EAAW,iBAAA;CH4P9C;AG3PmC;EAAW,iBAAA;CH8P9C;AG7PmC;EAAW,iBAAA;CHgQ9C;AG/PmC;EAAW,iBAAA;CHkQ9C;AGjQmC;EAAW,iBAAA;CHoQ9C;AGnQmC;EAAW,iBAAA;CHsQ9C;AGrQmC;EAAW,iBAAA;CHwQ9C;AGvQmC;EAAW,iBAAA;CH0Q9C;AGzQmC;EAAW,iBAAA;CH4Q9C;AG3QmC;EAAW,iBAAA;CH8Q9C;AG7QmC;EAAW,iBAAA;CHgR9C;AG/QmC;EAAW,iBAAA;CHkR9C;AGjRmC;EAAW,iBAAA;CHoR9C;AGnRmC;EAAW,iBAAA;CHsR9C;AGrRmC;EAAW,iBAAA;CHwR9C;AGvRmC;EAAW,iBAAA;CH0R9C;AGzRmC;EAAW,iBAAA;CH4R9C;AG3RmC;EAAW,iBAAA;CH8R9C;AG7RmC;EAAW,iBAAA;CHgS9C;AG/RmC;EAAW,iBAAA;CHkS9C;AGjSmC;EAAW,iBAAA;CHoS9C;AGnSmC;EAAW,iBAAA;CHsS9C;AGrSmC;EAAW,iBAAA;CHwS9C;AGvSmC;EAAW,iBAAA;CH0S9C;AGzSmC;EAAW,iBAAA;CH4S9C;AG3SmC;EAAW,iBAAA;CH8S9C;AG7SmC;EAAW,iBAAA;CHgT9C;AG/SmC;EAAW,iBAAA;CHkT9C;AGjTmC;EAAW,iBAAA;CHoT9C;AGnTmC;EAAW,iBAAA;CHsT9C;AGrTmC;EAAW,iBAAA;CHwT9C;AGvTmC;EAAW,iBAAA;CH0T9C;AGzTmC;EAAW,iBAAA;CH4T9C;AG3TmC;EAAW,iBAAA;CH8T9C;AG7TmC;EAAW,iBAAA;CHgU9C;AG/TmC;EAAW,iBAAA;CHkU9C;AGjUmC;EAAW,iBAAA;CHoU9C;AGnUmC;EAAW,iBAAA;CHsU9C;AGrUmC;EAAW,iBAAA;CHwU9C;AGvUmC;EAAW,iBAAA;CH0U9C;AGzUmC;EAAW,iBAAA;CH4U9C;AG3UmC;EAAW,iBAAA;CH8U9C;AG7UmC;EAAW,iBAAA;CHgV9C;AG/UmC;EAAW,iBAAA;CHkV9C;AGjVmC;EAAW,iBAAA;CHoV9C;AGnVmC;EAAW,iBAAA;CHsV9C;AGrVmC;EAAW,iBAAA;CHwV9C;AGvVmC;EAAW,iBAAA;CH0V9C;AGzVmC;EAAW,iBAAA;CH4V9C;AG3VmC;EAAW,iBAAA;CH8V9C;AG7VmC;EAAW,iBAAA;CHgW9C;AG/VmC;EAAW,iBAAA;CHkW9C;AGjWmC;EAAW,iBAAA;CHoW9C;AGnWmC;EAAW,iBAAA;CHsW9C;AGrWmC;EAAW,iBAAA;CHwW9C;AGvWmC;EAAW,iBAAA;CH0W9C;AGzWmC;EAAW,iBAAA;CH4W9C;AG3WmC;EAAW,iBAAA;CH8W9C;AG7WmC;EAAW,iBAAA;CHgX9C;AG/WmC;EAAW,iBAAA;CHkX9C;AGjXmC;EAAW,iBAAA;CHoX9C;AGnXmC;EAAW,iBAAA;CHsX9C;AGrXmC;EAAW,iBAAA;CHwX9C;AGvXmC;EAAW,iBAAA;CH0X9C;AGzXmC;EAAW,iBAAA;CH4X9C;AG3XmC;EAAW,iBAAA;CH8X9C;AG7XmC;EAAW,iBAAA;CHgY9C;AG/XmC;EAAW,iBAAA;CHkY9C;AGjYmC;EAAW,iBAAA;CHoY9C;AGnYmC;EAAW,iBAAA;CHsY9C;AGrYmC;EAAW,iBAAA;CHwY9C;AGvYmC;EAAW,iBAAA;CH0Y9C;AGzYmC;EAAW,iBAAA;CH4Y9C;AG3YmC;EAAW,iBAAA;CH8Y9C;AG7YmC;EAAW,iBAAA;CHgZ9C;AG/YmC;EAAW,iBAAA;CHkZ9C;AGjZmC;EAAW,iBAAA;CHoZ9C;AGnZmC;EAAW,iBAAA;CHsZ9C;AGrZmC;EAAW,iBAAA;CHwZ9C;AGvZmC;EAAW,iBAAA;CH0Z9C;AGzZmC;EAAW,iBAAA;CH4Z9C;AG3ZmC;EAAW,iBAAA;CH8Z9C;AG7ZmC;EAAW,iBAAA;CHga9C;AG/ZmC;EAAW,iBAAA;CHka9C;AGjamC;EAAW,iBAAA;CHoa9C;AGnamC;EAAW,iBAAA;CHsa9C;AGramC;EAAW,iBAAA;CHwa9C;AGvamC;EAAW,iBAAA;CH0a9C;AGzamC;EAAW,iBAAA;CH4a9C;AG3amC;EAAW,iBAAA;CH8a9C;AG7amC;EAAW,iBAAA;CHgb9C;AG/amC;EAAW,iBAAA;CHkb9C;AGjbmC;EAAW,iBAAA;CHob9C;AGnbmC;EAAW,iBAAA;CHsb9C;AGrbmC;EAAW,iBAAA;CHwb9C;AGvbmC;EAAW,iBAAA;CH0b9C;AGzbmC;EAAW,iBAAA;CH4b9C;AG3bmC;EAAW,iBAAA;CH8b9C;AG7bmC;EAAW,iBAAA;CHgc9C;AG/bmC;EAAW,iBAAA;CHkc9C;AGjcmC;EAAW,iBAAA;CHoc9C;AGncmC;EAAW,iBAAA;CHsc9C;AGrcmC;EAAW,iBAAA;CHwc9C;AGvcmC;EAAW,iBAAA;CH0c9C;AGzcmC;EAAW,iBAAA;CH4c9C;AG3cmC;EAAW,iBAAA;CH8c9C;AG7cmC;EAAW,iBAAA;CHgd9C;AG/cmC;EAAW,iBAAA;CHkd9C;AGjdmC;EAAW,iBAAA;CHod9C;AGndmC;EAAW,iBAAA;CHsd9C;AGrdmC;EAAW,iBAAA;CHwd9C;AGvdmC;EAAW,iBAAA;CH0d9C;AGzdmC;EAAW,iBAAA;CH4d9C;AG3dmC;EAAW,iBAAA;CH8d9C;AG7dmC;EAAW,iBAAA;CHge9C;AG/dmC;EAAW,iBAAA;CHke9C;AGjemC;EAAW,iBAAA;CHoe9C;AGnemC;EAAW,iBAAA;CHse9C;AGremC;EAAW,iBAAA;CHwe9C;AGvemC;EAAW,iBAAA;CH0e9C;AGzemC;EAAW,iBAAA;CH4e9C;AG3emC;EAAW,iBAAA;CH8e9C;AG7emC;EAAW,iBAAA;CHgf9C;AG/emC;EAAW,iBAAA;CHkf9C;AGjfmC;EAAW,iBAAA;CHof9C;AGnfmC;EAAW,iBAAA;CHsf9C;AGrfmC;EAAW,iBAAA;CHwf9C;AGvfmC;EAAW,iBAAA;CH0f9C;AGzfmC;EAAW,iBAAA;CH4f9C;AG3fmC;EAAW,iBAAA;CH8f9C;AG7fmC;EAAW,iBAAA;CHggB9C;AG/fmC;EAAW,iBAAA;CHkgB9C;AGjgBmC;EAAW,iBAAA;CHogB9C;AGngBmC;EAAW,iBAAA;CHsgB9C;AGrgBmC;EAAW,iBAAA;CHwgB9C;AGvgBmC;EAAW,iBAAA;CH0gB9C;AGzgBmC;EAAW,iBAAA;CH4gB9C;AG3gBmC;EAAW,iBAAA;CH8gB9C;AG7gBmC;EAAW,iBAAA;CHghB9C;AG/gBmC;EAAW,iBAAA;CHkhB9C;AGjhBmC;EAAW,iBAAA;CHohB9C;AGnhBmC;EAAW,iBAAA;CHshB9C;AGrhBmC;EAAW,iBAAA;CHwhB9C;AGvhBmC;EAAW,iBAAA;CH0hB9C;AGzhBmC;EAAW,iBAAA;CH4hB9C;AG3hBmC;EAAW,iBAAA;CH8hB9C;AG7hBmC;EAAW,iBAAA;CHgiB9C;AG/hBmC;EAAW,iBAAA;CHkiB9C;AGjiBmC;EAAW,iBAAA;CHoiB9C;AGniBmC;EAAW,iBAAA;CHsiB9C;AGriBmC;EAAW,iBAAA;CHwiB9C;AGviBmC;EAAW,iBAAA;CH0iB9C;AGziBmC;EAAW,iBAAA;CH4iB9C;AG3iBmC;EAAW,iBAAA;CH8iB9C;AG7iBmC;EAAW,iBAAA;CHgjB9C;AG/iBmC;EAAW,iBAAA;CHkjB9C;AGjjBmC;EAAW,iBAAA;CHojB9C;AGnjBmC;EAAW,iBAAA;CHsjB9C;AGrjBmC;EAAW,iBAAA;CHwjB9C;AGvjBmC;EAAW,iBAAA;CH0jB9C;AGzjBmC;EAAW,iBAAA;CH4jB9C;AG3jBmC;EAAW,iBAAA;CH8jB9C;AG7jBmC;EAAW,iBAAA;CHgkB9C;AG/jBmC;EAAW,iBAAA;CHkkB9C;AGjkBmC;EAAW,iBAAA;CHokB9C;AGnkBmC;EAAW,iBAAA;CHskB9C;AGrkBmC;EAAW,iBAAA;CHwkB9C;AGvkBmC;EAAW,iBAAA;CH0kB9C;AGzkBmC;EAAW,iBAAA;CH4kB9C;AG3kBmC;EAAW,iBAAA;CH8kB9C;AG7kBmC;EAAW,iBAAA;CHglB9C;AG/kBmC;EAAW,iBAAA;CHklB9C;AGjlBmC;EAAW,iBAAA;CHolB9C;AGnlBmC;EAAW,iBAAA;CHslB9C;AGrlBmC;EAAW,iBAAA;CHwlB9C;AGvlBmC;EAAW,iBAAA;CH0lB9C;AGzlBmC;EAAW,iBAAA;CH4lB9C;AG3lBmC;EAAW,iBAAA;CH8lB9C;AG7lBmC;EAAW,iBAAA;CHgmB9C;AG/lBmC;EAAW,iBAAA;CHkmB9C;AGjmBmC;EAAW,iBAAA;CHomB9C;AGnmBmC;EAAW,iBAAA;CHsmB9C;AGrmBmC;EAAW,iBAAA;CHwmB9C;AGvmBmC;EAAW,iBAAA;CH0mB9C;AGzmBmC;EAAW,iBAAA;CH4mB9C;AG3mBmC;EAAW,iBAAA;CH8mB9C;AG7mBmC;EAAW,iBAAA;CHgnB9C;AG/mBmC;EAAW,iBAAA;CHknB9C;AGjnBmC;EAAW,iBAAA;CHonB9C;AGnnBmC;EAAW,iBAAA;CHsnB9C;AGrnBmC;EAAW,iBAAA;CHwnB9C;AGvnBmC;EAAW,iBAAA;CH0nB9C;AGznBmC;EAAW,iBAAA;CH4nB9C;AG3nBmC;EAAW,iBAAA;CH8nB9C;AG7nBmC;EAAW,iBAAA;CHgoB9C;AG/nBmC;EAAW,iBAAA;CHkoB9C;AGjoBmC;EAAW,iBAAA;CHooB9C;AGnoBmC;EAAW,iBAAA;CHsoB9C;AGroBmC;EAAW,iBAAA;CHwoB9C;AG/nBmC;EAAW,iBAAA;CHkoB9C;AGjoBmC;EAAW,iBAAA;CHooB9C;AGnoBmC;EAAW,iBAAA;CHsoB9C;AGroBmC;EAAW,iBAAA;CHwoB9C;AGvoBmC;EAAW,iBAAA;CH0oB9C;AGzoBmC;EAAW,iBAAA;CH4oB9C;AG3oBmC;EAAW,iBAAA;CH8oB9C;AG7oBmC;EAAW,iBAAA;CHgpB9C;AG/oBmC;EAAW,iBAAA;CHkpB9C;AGjpBmC;EAAW,iBAAA;CHopB9C;AGnpBmC;EAAW,iBAAA;CHspB9C;AGrpBmC;EAAW,iBAAA;CHwpB9C;AGvpBmC;EAAW,iBAAA;CH0pB9C;AGzpBmC;EAAW,iBAAA;CH4pB9C;AG3pBmC;EAAW,iBAAA;CH8pB9C;AG7pBmC;EAAW,iBAAA;CHgqB9C;AG/pBmC;EAAW,iBAAA;CHkqB9C;AGjqBmC;EAAW,iBAAA;CHoqB9C;AGnqBmC;EAAW,iBAAA;CHsqB9C;AGrqBmC;EAAW,iBAAA;CHwqB9C;AGvqBmC;EAAW,iBAAA;CH0qB9C;AGzqBmC;EAAW,iBAAA;CH4qB9C;AG3qBmC;EAAW,iBAAA;CH8qB9C;AG7qBmC;EAAW,iBAAA;CHgrB9C;AG/qBmC;EAAW,iBAAA;CHkrB9C;AGjrBmC;EAAW,iBAAA;CHorB9C;AGnrBmC;EAAW,iBAAA;CHsrB9C;AGrrBmC;EAAW,iBAAA;CHwrB9C;AGvrBmC;EAAW,iBAAA;CH0rB9C;AGzrBmC;EAAW,iBAAA;CH4rB9C;AG3rBmC;EAAW,iBAAA;CH8rB9C;AG7rBmC;EAAW,iBAAA;CHgsB9C;AG/rBmC;EAAW,iBAAA;CHksB9C;AGjsBmC;EAAW,iBAAA;CHosB9C;AGnsBmC;EAAW,iBAAA;CHssB9C;AGrsBmC;EAAW,iBAAA;CHwsB9C;AGvsBmC;EAAW,iBAAA;CH0sB9C;AGzsBmC;EAAW,iBAAA;CH4sB9C;AG3sBmC;EAAW,iBAAA;CH8sB9C;AG7sBmC;EAAW,iBAAA;CHgtB9C;AG/sBmC;EAAW,iBAAA;CHktB9C;AGjtBmC;EAAW,iBAAA;CHotB9C;AGntBmC;EAAW,iBAAA;CHstB9C;AGrtBmC;EAAW,iBAAA;CHwtB9C;AGvtBmC;EAAW,iBAAA;CH0tB9C;AGztBmC;EAAW,iBAAA;CH4tB9C;AG3tBmC;EAAW,iBAAA;CH8tB9C;AG7tBmC;EAAW,iBAAA;CHguB9C;AG/tBmC;EAAW,iBAAA;CHkuB9C;AGjuBmC;EAAW,iBAAA;CHouB9C;AGnuBmC;EAAW,iBAAA;CHsuB9C;AGruBmC;EAAW,iBAAA;CHwuB9C;AGvuBmC;EAAW,iBAAA;CH0uB9C;AGzuBmC;EAAW,iBAAA;CH4uB9C;AG3uBmC;EAAW,iBAAA;CH8uB9C;AG7uBmC;EAAW,iBAAA;CHgvB9C;AIthCD;ECgEE,+BAAA;EACG,4BAAA;EACK,uBAAA;CLy9BT;AIxhCD;;EC6DE,+BAAA;EACG,4BAAA;EACK,uBAAA;CL+9BT;AIthCD;EACE,gBAAA;EACA,8CAAA;CJwhCD;AIrhCD;EACE,4DAAA;EACA,gBAAA;EACA,wBAAA;EACA,eAAA;EACA,0BAAA;CJuhCD;AInhCD;;;;EAIE,qBAAA;EACA,mBAAA;EACA,qBAAA;CJqhCD;AI/gCD;EACE,eAAA;EACA,sBAAA;CJihCD;AI/gCC;;EAEE,eAAA;EACA,2BAAA;CJihCH;AI9gCC;EErDA,qBAAA;EAEA,2CAAA;EACA,qBAAA;CNqkCD;AIxgCD;EACE,UAAA;CJ0gCD;AIpgCD;EACE,uBAAA;CJsgCD;AIlgCD;;;;;EGvEE,eAAA;EACA,gBAAA;EACA,aAAA;CPglCD;AItgCD;EACE,mBAAA;CJwgCD;AIlgCD;EACE,aAAA;EACA,wBAAA;EACA,0BAAA;EACA,0BAAA;EACA,mBAAA;EC6FA,yCAAA;EACK,oCAAA;EACG,iCAAA;EEvLR,sBAAA;EACA,gBAAA;EACA,aAAA;CPgmCD;AIlgCD;EACE,mBAAA;CJogCD;AI9/BD;EACE,iBAAA;EACA,oBAAA;EACA,UAAA;EACA,8BAAA;CJggCD;AIx/BD;EACE,mBAAA;EACA,WAAA;EACA,YAAA;EACA,aAAA;EACA,WAAA;EACA,iBAAA;EACA,uBAAA;EACA,UAAA;CJ0/BD;AIl/BC;;EAEE,iBAAA;EACA,YAAA;EACA,aAAA;EACA,UAAA;EACA,kBAAA;EACA,WAAA;CJo/BH;AIz+BD;EACE,gBAAA;CJ2+BD;AQloCD;;;;;;;;;;;;EAEE,qBAAA;EACA,iBAAA;EACA,iBAAA;EACA,eAAA;CR8oCD;AQnpCD;;;;;;;;;;;;;;;;;;;;;;;;EASI,oBAAA;EACA,eAAA;EACA,eAAA;CRoqCH;AQhqCD;;;;;;EAGE,iBAAA;EACA,oBAAA;CRqqCD;AQzqCD;;;;;;;;;;;;EAQI,eAAA;CR+qCH;AQ5qCD;;;;;;EAGE,iBAAA;EACA,oBAAA;CRirCD;AQrrCD;;;;;;;;;;;;EAQI,eAAA;CR2rCH;AQvrCD;;EAAU,gBAAA;CR2rCT;AQ1rCD;;EAAU,gBAAA;CR8rCT;AQ7rCD;;EAAU,gBAAA;CRisCT;AQhsCD;;EAAU,gBAAA;CRosCT;AQnsCD;;EAAU,gBAAA;CRusCT;AQtsCD;;EAAU,gBAAA;CR0sCT;AQpsCD;EACE,iBAAA;CRssCD;AQnsCD;EACE,oBAAA;EACA,gBAAA;EACA,iBAAA;EACA,iBAAA;CRqsCD;AQhsCD;EAAA;IAFI,gBAAA;GRssCD;CACF;AQ9rCD;;EAEE,eAAA;CRgsCD;AQ7rCD;;EAEE,0BAAA;EACA,cAAA;CR+rCD;AQ3rCD;EAAuB,iBAAA;CR8rCtB;AQ7rCD;EAAuB,kBAAA;CRgsCtB;AQ/rCD;EAAuB,mBAAA;CRksCtB;AQjsCD;EAAuB,oBAAA;CRosCtB;AQnsCD;EAAuB,oBAAA;CRssCtB;AQnsCD;EAAuB,0BAAA;CRssCtB;AQrsCD;EAAuB,0BAAA;CRwsCtB;AQvsCD;EAAuB,2BAAA;CR0sCtB;AQvsCD;EACE,eAAA;CRysCD;AQvsCD;ECrGE,eAAA;CT+yCD;AS9yCC;;EAEE,eAAA;CTgzCH;AQ3sCD;ECxGE,eAAA;CTszCD;ASrzCC;;EAEE,eAAA;CTuzCH;AQ/sCD;EC3GE,eAAA;CT6zCD;AS5zCC;;EAEE,eAAA;CT8zCH;AQntCD;EC9GE,eAAA;CTo0CD;ASn0CC;;EAEE,eAAA;CTq0CH;AQvtCD;ECjHE,eAAA;CT20CD;AS10CC;;EAEE,eAAA;CT40CH;AQvtCD;EAGE,YAAA;EE3HA,0BAAA;CVm1CD;AUl1CC;;EAEE,0BAAA;CVo1CH;AQztCD;EE9HE,0BAAA;CV01CD;AUz1CC;;EAEE,0BAAA;CV21CH;AQ7tCD;EEjIE,0BAAA;CVi2CD;AUh2CC;;EAEE,0BAAA;CVk2CH;AQjuCD;EEpIE,0BAAA;CVw2CD;AUv2CC;;EAEE,0BAAA;CVy2CH;AQruCD;EEvIE,0BAAA;CV+2CD;AU92CC;;EAEE,0BAAA;CVg3CH;AQpuCD;EACE,oBAAA;EACA,oBAAA;EACA,iCAAA;CRsuCD;AQ9tCD;;EAEE,cAAA;EACA,oBAAA;CRguCD;AQnuCD;;;;EAMI,iBAAA;CRmuCH;AQ5tCD;EACE,gBAAA;EACA,iBAAA;CR8tCD;AQ1tCD;EALE,gBAAA;EACA,iBAAA;EAMA,kBAAA;CR6tCD;AQ/tCD;EAKI,sBAAA;EACA,kBAAA;EACA,mBAAA;CR6tCH;AQxtCD;EACE,cAAA;EACA,oBAAA;CR0tCD;AQxtCD;;EAEE,wBAAA;CR0tCD;AQxtCD;EACE,kBAAA;CR0tCD;AQxtCD;EACE,eAAA;CR0tCD;AQjsCD;EAAA;IAVM,YAAA;IACA,aAAA;IACA,YAAA;IACA,kBAAA;IGtNJ,iBAAA;IACA,wBAAA;IACA,oBAAA;GXs6CC;EQ3sCH;IAHM,mBAAA;GRitCH;CACF;AQxsCD;;EAGE,aAAA;EACA,kCAAA;CRysCD;AQvsCD;EACE,eAAA;EA9IqB,0BAAA;CRw1CtB;AQrsCD;EACE,mBAAA;EACA,iBAAA;EACA,kBAAA;EACA,+BAAA;CRusCD;AQlsCG;;;EACE,iBAAA;CRssCL;AQhtCD;;;EAmBI,eAAA;EACA,eAAA;EACA,wBAAA;EACA,eAAA;CRksCH;AQhsCG;;;EACE,uBAAA;CRosCL;AQ5rCD;;EAEE,oBAAA;EACA,gBAAA;EACA,gCAAA;EACA,eAAA;EACA,kBAAA;CR8rCD;AQxrCG;;;;;;EAAW,YAAA;CRgsCd;AQ/rCG;;;;;;EACE,uBAAA;CRssCL;AQhsCD;EACE,oBAAA;EACA,mBAAA;EACA,wBAAA;CRksCD;AYx+CD;;;;EAIE,+DAAA;CZ0+CD;AYt+CD;EACE,iBAAA;EACA,eAAA;EACA,eAAA;EACA,0BAAA;EACA,mBAAA;CZw+CD;AYp+CD;EACE,iBAAA;EACA,eAAA;EACA,eAAA;EACA,0BAAA;EACA,mBAAA;EACA,uDAAA;UAAA,+CAAA;CZs+CD;AY5+CD;EASI,WAAA;EACA,gBAAA;EACA,kBAAA;EACA,yBAAA;UAAA,iBAAA;CZs+CH;AYj+CD;EACE,eAAA;EACA,eAAA;EACA,iBAAA;EACA,gBAAA;EACA,wBAAA;EACA,sBAAA;EACA,sBAAA;EACA,eAAA;EACA,0BAAA;EACA,0BAAA;EACA,mBAAA;CZm+CD;AY9+CD;EAeI,WAAA;EACA,mBAAA;EACA,eAAA;EACA,sBAAA;EACA,8BAAA;EACA,iBAAA;CZk+CH;AY79CD;EACE,kBAAA;EACA,mBAAA;CZ+9CD;AazhDD;ECHE,mBAAA;EACA,kBAAA;EACA,mBAAA;EACA,oBAAA;Cd+hDD;AazhDC;EAAA;IAFE,aAAA;Gb+hDD;CACF;Aa3hDC;EAAA;IAFE,aAAA;GbiiDD;CACF;Aa7hDD;EAAA;IAFI,cAAA;GbmiDD;CACF;Aa1hDD;ECvBE,mBAAA;EACA,kBAAA;EACA,mBAAA;EACA,oBAAA;CdojDD;AavhDD;ECvBE,mBAAA;EACA,oBAAA;CdijDD;AejjDG;EACE,mBAAA;EAEA,gBAAA;EAEA,mBAAA;EACA,oBAAA;CfijDL;AejiDG;EACE,YAAA;CfmiDL;Ae5hDC;EACE,YAAA;Cf8hDH;Ae/hDC;EACE,oBAAA;CfiiDH;AeliDC;EACE,oBAAA;CfoiDH;AeriDC;EACE,WAAA;CfuiDH;AexiDC;EACE,oBAAA;Cf0iDH;Ae3iDC;EACE,oBAAA;Cf6iDH;Ae9iDC;EACE,WAAA;CfgjDH;AejjDC;EACE,oBAAA;CfmjDH;AepjDC;EACE,oBAAA;CfsjDH;AevjDC;EACE,WAAA;CfyjDH;Ae1jDC;EACE,oBAAA;Cf4jDH;Ae7jDC;EACE,mBAAA;Cf+jDH;AejjDC;EACE,YAAA;CfmjDH;AepjDC;EACE,oBAAA;CfsjDH;AevjDC;EACE,oBAAA;CfyjDH;Ae1jDC;EACE,WAAA;Cf4jDH;Ae7jDC;EACE,oBAAA;Cf+jDH;AehkDC;EACE,oBAAA;CfkkDH;AenkDC;EACE,WAAA;CfqkDH;AetkDC;EACE,oBAAA;CfwkDH;AezkDC;EACE,oBAAA;Cf2kDH;Ae5kDC;EACE,WAAA;Cf8kDH;Ae/kDC;EACE,oBAAA;CfilDH;AellDC;EACE,mBAAA;CfolDH;AehlDC;EACE,YAAA;CfklDH;AelmDC;EACE,WAAA;CfomDH;AermDC;EACE,mBAAA;CfumDH;AexmDC;EACE,mBAAA;Cf0mDH;Ae3mDC;EACE,UAAA;Cf6mDH;Ae9mDC;EACE,mBAAA;CfgnDH;AejnDC;EACE,mBAAA;CfmnDH;AepnDC;EACE,UAAA;CfsnDH;AevnDC;EACE,mBAAA;CfynDH;Ae1nDC;EACE,mBAAA;Cf4nDH;Ae7nDC;EACE,UAAA;Cf+nDH;AehoDC;EACE,mBAAA;CfkoDH;AenoDC;EACE,kBAAA;CfqoDH;AejoDC;EACE,WAAA;CfmoDH;AernDC;EACE,kBAAA;CfunDH;AexnDC;EACE,0BAAA;Cf0nDH;Ae3nDC;EACE,0BAAA;Cf6nDH;Ae9nDC;EACE,iBAAA;CfgoDH;AejoDC;EACE,0BAAA;CfmoDH;AepoDC;EACE,0BAAA;CfsoDH;AevoDC;EACE,iBAAA;CfyoDH;Ae1oDC;EACE,0BAAA;Cf4oDH;Ae7oDC;EACE,0BAAA;Cf+oDH;AehpDC;EACE,iBAAA;CfkpDH;AenpDC;EACE,0BAAA;CfqpDH;AetpDC;EACE,yBAAA;CfwpDH;AezpDC;EACE,gBAAA;Cf2pDH;Aa3pDD;EElCI;IACE,YAAA;GfgsDH;EezrDD;IACE,YAAA;Gf2rDD;Ee5rDD;IACE,oBAAA;Gf8rDD;Ee/rDD;IACE,oBAAA;GfisDD;EelsDD;IACE,WAAA;GfosDD;EersDD;IACE,oBAAA;GfusDD;EexsDD;IACE,oBAAA;Gf0sDD;Ee3sDD;IACE,WAAA;Gf6sDD;Ee9sDD;IACE,oBAAA;GfgtDD;EejtDD;IACE,oBAAA;GfmtDD;EeptDD;IACE,WAAA;GfstDD;EevtDD;IACE,oBAAA;GfytDD;Ee1tDD;IACE,mBAAA;Gf4tDD;Ee9sDD;IACE,YAAA;GfgtDD;EejtDD;IACE,oBAAA;GfmtDD;EeptDD;IACE,oBAAA;GfstDD;EevtDD;IACE,WAAA;GfytDD;Ee1tDD;IACE,oBAAA;Gf4tDD;Ee7tDD;IACE,oBAAA;Gf+tDD;EehuDD;IACE,WAAA;GfkuDD;EenuDD;IACE,oBAAA;GfquDD;EetuDD;IACE,oBAAA;GfwuDD;EezuDD;IACE,WAAA;Gf2uDD;Ee5uDD;IACE,oBAAA;Gf8uDD;Ee/uDD;IACE,mBAAA;GfivDD;Ee7uDD;IACE,YAAA;Gf+uDD;Ee/vDD;IACE,WAAA;GfiwDD;EelwDD;IACE,mBAAA;GfowDD;EerwDD;IACE,mBAAA;GfuwDD;EexwDD;IACE,UAAA;Gf0wDD;Ee3wDD;IACE,mBAAA;Gf6wDD;Ee9wDD;IACE,mBAAA;GfgxDD;EejxDD;IACE,UAAA;GfmxDD;EepxDD;IACE,mBAAA;GfsxDD;EevxDD;IACE,mBAAA;GfyxDD;Ee1xDD;IACE,UAAA;Gf4xDD;Ee7xDD;IACE,mBAAA;Gf+xDD;EehyDD;IACE,kBAAA;GfkyDD;Ee9xDD;IACE,WAAA;GfgyDD;EelxDD;IACE,kBAAA;GfoxDD;EerxDD;IACE,0BAAA;GfuxDD;EexxDD;IACE,0BAAA;Gf0xDD;Ee3xDD;IACE,iBAAA;Gf6xDD;Ee9xDD;IACE,0BAAA;GfgyDD;EejyDD;IACE,0BAAA;GfmyDD;EepyDD;IACE,iBAAA;GfsyDD;EevyDD;IACE,0BAAA;GfyyDD;Ee1yDD;IACE,0BAAA;Gf4yDD;Ee7yDD;IACE,iBAAA;Gf+yDD;EehzDD;IACE,0BAAA;GfkzDD;EenzDD;IACE,yBAAA;GfqzDD;EetzDD;IACE,gBAAA;GfwzDD;CACF;AahzDD;EE3CI;IACE,YAAA;Gf81DH;Eev1DD;IACE,YAAA;Gfy1DD;Ee11DD;IACE,oBAAA;Gf41DD;Ee71DD;IACE,oBAAA;Gf+1DD;Eeh2DD;IACE,WAAA;Gfk2DD;Een2DD;IACE,oBAAA;Gfq2DD;Eet2DD;IACE,oBAAA;Gfw2DD;Eez2DD;IACE,WAAA;Gf22DD;Ee52DD;IACE,oBAAA;Gf82DD;Ee/2DD;IACE,oBAAA;Gfi3DD;Eel3DD;IACE,WAAA;Gfo3DD;Eer3DD;IACE,oBAAA;Gfu3DD;Eex3DD;IACE,mBAAA;Gf03DD;Ee52DD;IACE,YAAA;Gf82DD;Ee/2DD;IACE,oBAAA;Gfi3DD;Eel3DD;IACE,oBAAA;Gfo3DD;Eer3DD;IACE,WAAA;Gfu3DD;Eex3DD;IACE,oBAAA;Gf03DD;Ee33DD;IACE,oBAAA;Gf63DD;Ee93DD;IACE,WAAA;Gfg4DD;Eej4DD;IACE,oBAAA;Gfm4DD;Eep4DD;IACE,oBAAA;Gfs4DD;Eev4DD;IACE,WAAA;Gfy4DD;Ee14DD;IACE,oBAAA;Gf44DD;Ee74DD;IACE,mBAAA;Gf+4DD;Ee34DD;IACE,YAAA;Gf64DD;Ee75DD;IACE,WAAA;Gf+5DD;Eeh6DD;IACE,mBAAA;Gfk6DD;Een6DD;IACE,mBAAA;Gfq6DD;Eet6DD;IACE,UAAA;Gfw6DD;Eez6DD;IACE,mBAAA;Gf26DD;Ee56DD;IACE,mBAAA;Gf86DD;Ee/6DD;IACE,UAAA;Gfi7DD;Eel7DD;IACE,mBAAA;Gfo7DD;Eer7DD;IACE,mBAAA;Gfu7DD;Eex7DD;IACE,UAAA;Gf07DD;Ee37DD;IACE,mBAAA;Gf67DD;Ee97DD;IACE,kBAAA;Gfg8DD;Ee57DD;IACE,WAAA;Gf87DD;Eeh7DD;IACE,kBAAA;Gfk7DD;Een7DD;IACE,0BAAA;Gfq7DD;Eet7DD;IACE,0BAAA;Gfw7DD;Eez7DD;IACE,iBAAA;Gf27DD;Ee57DD;IACE,0BAAA;Gf87DD;Ee/7DD;IACE,0BAAA;Gfi8DD;Eel8DD;IACE,iBAAA;Gfo8DD;Eer8DD;IACE,0BAAA;Gfu8DD;Eex8DD;IACE,0BAAA;Gf08DD;Ee38DD;IACE,iBAAA;Gf68DD;Ee98DD;IACE,0BAAA;Gfg9DD;Eej9DD;IACE,yBAAA;Gfm9DD;Eep9DD;IACE,gBAAA;Gfs9DD;CACF;Aa38DD;EE9CI;IACE,YAAA;Gf4/DH;Eer/DD;IACE,YAAA;Gfu/DD;Eex/DD;IACE,oBAAA;Gf0/DD;Ee3/DD;IACE,oBAAA;Gf6/DD;Ee9/DD;IACE,WAAA;GfggED;EejgED;IACE,oBAAA;GfmgED;EepgED;IACE,oBAAA;GfsgED;EevgED;IACE,WAAA;GfygED;Ee1gED;IACE,oBAAA;Gf4gED;Ee7gED;IACE,oBAAA;Gf+gED;EehhED;IACE,WAAA;GfkhED;EenhED;IACE,oBAAA;GfqhED;EethED;IACE,mBAAA;GfwhED;Ee1gED;IACE,YAAA;Gf4gED;Ee7gED;IACE,oBAAA;Gf+gED;EehhED;IACE,oBAAA;GfkhED;EenhED;IACE,WAAA;GfqhED;EethED;IACE,oBAAA;GfwhED;EezhED;IACE,oBAAA;Gf2hED;Ee5hED;IACE,WAAA;Gf8hED;Ee/hED;IACE,oBAAA;GfiiED;EeliED;IACE,oBAAA;GfoiED;EeriED;IACE,WAAA;GfuiED;EexiED;IACE,oBAAA;Gf0iED;Ee3iED;IACE,mBAAA;Gf6iED;EeziED;IACE,YAAA;Gf2iED;Ee3jED;IACE,WAAA;Gf6jED;Ee9jED;IACE,mBAAA;GfgkED;EejkED;IACE,mBAAA;GfmkED;EepkED;IACE,UAAA;GfskED;EevkED;IACE,mBAAA;GfykED;Ee1kED;IACE,mBAAA;Gf4kED;Ee7kED;IACE,UAAA;Gf+kED;EehlED;IACE,mBAAA;GfklED;EenlED;IACE,mBAAA;GfqlED;EetlED;IACE,UAAA;GfwlED;EezlED;IACE,mBAAA;Gf2lED;Ee5lED;IACE,kBAAA;Gf8lED;Ee1lED;IACE,WAAA;Gf4lED;Ee9kED;IACE,kBAAA;GfglED;EejlED;IACE,0BAAA;GfmlED;EeplED;IACE,0BAAA;GfslED;EevlED;IACE,iBAAA;GfylED;Ee1lED;IACE,0BAAA;Gf4lED;Ee7lED;IACE,0BAAA;Gf+lED;EehmED;IACE,iBAAA;GfkmED;EenmED;IACE,0BAAA;GfqmED;EetmED;IACE,0BAAA;GfwmED;EezmED;IACE,iBAAA;Gf2mED;Ee5mED;IACE,0BAAA;Gf8mED;Ee/mED;IACE,yBAAA;GfinED;EelnED;IACE,gBAAA;GfonED;CACF;AgBxrED;EACE,8BAAA;ChB0rED;AgBxrED;EACE,iBAAA;EACA,oBAAA;EACA,eAAA;EACA,iBAAA;ChB0rED;AgBxrED;EACE,iBAAA;ChB0rED;AgBprED;EACE,YAAA;EACA,gBAAA;EACA,oBAAA;ChBsrED;AgBzrED;;;;;;EAWQ,aAAA;EACA,wBAAA;EACA,oBAAA;EACA,8BAAA;ChBsrEP;AgBpsED;EAoBI,uBAAA;EACA,iCAAA;ChBmrEH;AgBxsED;;;;;;EA8BQ,cAAA;ChBkrEP;AgBhtED;EAoCI,8BAAA;ChB+qEH;AgBntED;EAyCI,0BAAA;ChB6qEH;AgBtqED;;;;;;EAOQ,aAAA;ChBuqEP;AgB5pED;EACE,0BAAA;ChB8pED;AgB/pED;;;;;;EAQQ,0BAAA;ChB+pEP;AgBvqED;;EAeM,yBAAA;ChB4pEL;AgBlpED;EAEI,0BAAA;ChBmpEH;AgB1oED;EAEI,0BAAA;ChB2oEH;AgBloED;EACE,iBAAA;EACA,YAAA;EACA,sBAAA;ChBooED;AgB/nEG;;EACE,iBAAA;EACA,YAAA;EACA,oBAAA;ChBkoEL;AiB9wEC;;;;;;;;;;;;EAOI,0BAAA;CjBqxEL;AiB/wEC;;;;;EAMI,0BAAA;CjBgxEL;AiBnyEC;;;;;;;;;;;;EAOI,0BAAA;CjB0yEL;AiBpyEC;;;;;EAMI,0BAAA;CjBqyEL;AiBxzEC;;;;;;;;;;;;EAOI,0BAAA;CjB+zEL;AiBzzEC;;;;;EAMI,0BAAA;CjB0zEL;AiB70EC;;;;;;;;;;;;EAOI,0BAAA;CjBo1EL;AiB90EC;;;;;EAMI,0BAAA;CjB+0EL;AiBl2EC;;;;;;;;;;;;EAOI,0BAAA;CjBy2EL;AiBn2EC;;;;;EAMI,0BAAA;CjBo2EL;AgBltED;EACE,iBAAA;EACA,kBAAA;ChBotED;AgBvpED;EAAA;IA1DI,YAAA;IACA,oBAAA;IACA,mBAAA;IACA,6CAAA;IACA,0BAAA;GhBqtED;EgB/pEH;IAlDM,iBAAA;GhBotEH;EgBlqEH;;;;;;IAzCY,oBAAA;GhBmtET;EgB1qEH;IAjCM,UAAA;GhB8sEH;EgB7qEH;;;;;;IAxBY,eAAA;GhB6sET;EgBrrEH;;;;;;IApBY,gBAAA;GhBitET;EgB7rEH;;;;IAPY,iBAAA;GhB0sET;CACF;AkBp6ED;EACE,WAAA;EACA,UAAA;EACA,UAAA;EAIA,aAAA;ClBm6ED;AkBh6ED;EACE,eAAA;EACA,YAAA;EACA,WAAA;EACA,oBAAA;EACA,gBAAA;EACA,qBAAA;EACA,eAAA;EACA,UAAA;EACA,iCAAA;ClBk6ED;AkB/5ED;EACE,sBAAA;EACA,gBAAA;EACA,mBAAA;EACA,kBAAA;ClBi6ED;AkBt5ED;Eb4BE,+BAAA;EACG,4BAAA;EACK,uBAAA;CL63ET;AkBt5ED;;EAEE,gBAAA;EACA,mBAAA;EACA,oBAAA;ClBw5ED;AkBr5ED;EACE,eAAA;ClBu5ED;AkBn5ED;EACE,eAAA;EACA,YAAA;ClBq5ED;AkBj5ED;;EAEE,aAAA;ClBm5ED;AkB/4ED;;;EZvEE,qBAAA;EAEA,2CAAA;EACA,qBAAA;CN09ED;AkB/4ED;EACE,eAAA;EACA,iBAAA;EACA,gBAAA;EACA,wBAAA;EACA,eAAA;ClBi5ED;AkBv3ED;EACE,eAAA;EACA,YAAA;EACA,aAAA;EACA,kBAAA;EACA,gBAAA;EACA,wBAAA;EACA,eAAA;EACA,0BAAA;EACA,uBAAA;EACA,0BAAA;EACA,mBAAA;EbxDA,yDAAA;EACQ,iDAAA;EAyHR,uFAAA;EACK,0EAAA;EACG,uEAAA;CL0zET;AmBl8EC;EACE,sBAAA;EACA,WAAA;EdUF,uFAAA;EACQ,+EAAA;CL27ET;AK15EC;EACE,eAAA;EACA,WAAA;CL45EH;AK15EC;EAA0B,eAAA;CL65E3B;AK55EC;EAAgC,eAAA;CL+5EjC;AkB/3EC;;;EAGE,0BAAA;EACA,WAAA;ClBi4EH;AkB93EC;;EAEE,oBAAA;ClBg4EH;AkB53EC;EACE,aAAA;ClB83EH;AkBl3ED;EACE,yBAAA;ClBo3ED;AkB50ED;EAtBI;;;;IACE,kBAAA;GlBw2EH;EkBr2EC;;;;;;;;IAEE,kBAAA;GlB62EH;EkB12EC;;;;;;;;IAEE,kBAAA;GlBk3EH;CACF;AkBx2ED;EACE,oBAAA;ClB02ED;AkBl2ED;;EAEE,mBAAA;EACA,eAAA;EACA,iBAAA;EACA,oBAAA;ClBo2ED;AkBz2ED;;EAQI,iBAAA;EACA,mBAAA;EACA,iBAAA;EACA,oBAAA;EACA,gBAAA;ClBq2EH;AkBl2ED;;;;EAIE,mBAAA;EACA,mBAAA;EACA,mBAAA;ClBo2ED;AkBj2ED;;EAEE,iBAAA;ClBm2ED;AkB/1ED;;EAEE,mBAAA;EACA,sBAAA;EACA,mBAAA;EACA,iBAAA;EACA,uBAAA;EACA,oBAAA;EACA,gBAAA;ClBi2ED;AkB/1ED;;EAEE,cAAA;EACA,kBAAA;ClBi2ED;AkBx1EC;;;;;;EAGE,oBAAA;ClB61EH;AkBv1EC;;;;EAEE,oBAAA;ClB21EH;AkBr1EC;;;;EAGI,oBAAA;ClBw1EL;AkB70ED;EAEE,iBAAA;EACA,oBAAA;EAEA,iBAAA;EACA,iBAAA;ClB60ED;AkB30EC;;EAEE,gBAAA;EACA,iBAAA;ClB60EH;AkBh0ED;EC7PE,aAAA;EACA,kBAAA;EACA,gBAAA;EACA,iBAAA;EACA,mBAAA;CnBgkFD;AmB9jFC;EACE,aAAA;EACA,kBAAA;CnBgkFH;AmB7jFC;;EAEE,aAAA;CnB+jFH;AkB50ED;EAEI,aAAA;EACA,kBAAA;EACA,gBAAA;EACA,iBAAA;EACA,mBAAA;ClB60EH;AkBn1ED;EASI,aAAA;EACA,kBAAA;ClB60EH;AkBv1ED;;EAcI,aAAA;ClB60EH;AkB31ED;EAiBI,aAAA;EACA,iBAAA;EACA,kBAAA;EACA,gBAAA;EACA,iBAAA;ClB60EH;AkBz0ED;ECzRE,aAAA;EACA,mBAAA;EACA,gBAAA;EACA,uBAAA;EACA,mBAAA;CnBqmFD;AmBnmFC;EACE,aAAA;EACA,kBAAA;CnBqmFH;AmBlmFC;;EAEE,aAAA;CnBomFH;AkBr1ED;EAEI,aAAA;EACA,mBAAA;EACA,gBAAA;EACA,uBAAA;EACA,mBAAA;ClBs1EH;AkB51ED;EASI,aAAA;EACA,kBAAA;ClBs1EH;AkBh2ED;;EAcI,aAAA;ClBs1EH;AkBp2ED;EAiBI,aAAA;EACA,iBAAA;EACA,mBAAA;EACA,gBAAA;EACA,uBAAA;ClBs1EH;AkB70ED;EAEE,mBAAA;ClB80ED;AkBh1ED;EAMI,sBAAA;ClB60EH;AkBz0ED;EACE,mBAAA;EACA,OAAA;EACA,SAAA;EACA,WAAA;EACA,eAAA;EACA,YAAA;EACA,aAAA;EACA,kBAAA;EACA,mBAAA;EACA,qBAAA;ClB20ED;AkBz0ED;;;EAGE,YAAA;EACA,aAAA;EACA,kBAAA;ClB20ED;AkBz0ED;;;EAGE,YAAA;EACA,aAAA;EACA,kBAAA;ClB20ED;AkBv0ED;;;;;;;;;;ECpZI,eAAA;CnBuuFH;AkBn1ED;EChZI,sBAAA;Ed+CF,yDAAA;EACQ,iDAAA;CLwrFT;AmBtuFG;EACE,sBAAA;Ed4CJ,0EAAA;EACQ,kEAAA;CL6rFT;AkB71ED;ECtYI,eAAA;EACA,sBAAA;EACA,0BAAA;CnBsuFH;AkBl2ED;EChYI,eAAA;CnBquFH;AkBl2ED;;;;;;;;;;ECvZI,eAAA;CnBqwFH;AkB92ED;ECnZI,sBAAA;Ed+CF,yDAAA;EACQ,iDAAA;CLstFT;AmBpwFG;EACE,sBAAA;Ed4CJ,0EAAA;EACQ,kEAAA;CL2tFT;AkBx3ED;ECzYI,eAAA;EACA,sBAAA;EACA,0BAAA;CnBowFH;AkB73ED;ECnYI,eAAA;CnBmwFH;AkB73ED;;;;;;;;;;EC1ZI,eAAA;CnBmyFH;AkBz4ED;ECtZI,sBAAA;Ed+CF,yDAAA;EACQ,iDAAA;CLovFT;AmBlyFG;EACE,sBAAA;Ed4CJ,0EAAA;EACQ,kEAAA;CLyvFT;AkBn5ED;EC5YI,eAAA;EACA,sBAAA;EACA,0BAAA;CnBkyFH;AkBx5ED;ECtYI,eAAA;CnBiyFH;AkBp5EC;EACG,UAAA;ClBs5EJ;AkBp5EC;EACG,OAAA;ClBs5EJ;AkB54ED;EACE,eAAA;EACA,gBAAA;EACA,oBAAA;EACA,eAAA;ClB84ED;AkB3zED;EAAA;IA9DM,sBAAA;IACA,iBAAA;IACA,uBAAA;GlB63EH;EkBj0EH;IAvDM,sBAAA;IACA,YAAA;IACA,uBAAA;GlB23EH;EkBt0EH;IAhDM,sBAAA;GlBy3EH;EkBz0EH;IA5CM,sBAAA;IACA,uBAAA;GlBw3EH;EkB70EH;;;IAtCQ,YAAA;GlBw3EL;EkBl1EH;IAhCM,YAAA;GlBq3EH;EkBr1EH;IA5BM,iBAAA;IACA,uBAAA;GlBo3EH;EkBz1EH;;IApBM,sBAAA;IACA,cAAA;IACA,iBAAA;IACA,uBAAA;GlBi3EH;EkBh2EH;;IAdQ,gBAAA;GlBk3EL;EkBp2EH;;IATM,mBAAA;IACA,eAAA;GlBi3EH;EkBz2EH;IAHM,OAAA;GlB+2EH;CACF;AkBr2ED;;;;EASI,cAAA;EACA,iBAAA;EACA,iBAAA;ClBk2EH;AkB72ED;;EAiBI,iBAAA;ClBg2EH;AkBj3ED;EJhhBE,mBAAA;EACA,oBAAA;Cdo4FD;AkB90EC;EAAA;IAVI,kBAAA;IACA,iBAAA;IACA,iBAAA;GlB41EH;CACF;AkB53ED;EAwCI,YAAA;ClBu1EH;AkBz0EC;EAAA;IAJM,yBAAA;IACA,gBAAA;GlBi1EL;CACF;AkBv0EC;EAAA;IAJM,iBAAA;IACA,gBAAA;GlB+0EL;CACF;AoBl6FD;EACE,sBAAA;EACA,iBAAA;EACA,oBAAA;EACA,mBAAA;EACA,uBAAA;EACA,+BAAA;MAAA,2BAAA;EACA,gBAAA;EACA,uBAAA;EACA,8BAAA;EACA,oBAAA;EC6CA,kBAAA;EACA,gBAAA;EACA,wBAAA;EACA,mBAAA;EhB4JA,0BAAA;EACG,uBAAA;EACC,sBAAA;EACI,kBAAA;CL6tFT;AoBr6FG;;;;;;EdrBF,qBAAA;EAEA,2CAAA;EACA,qBAAA;CNi8FD;AoBz6FC;;;EAGE,eAAA;EACA,sBAAA;CpB26FH;AoBx6FC;;EAEE,WAAA;EACA,uBAAA;Ef2BF,yDAAA;EACQ,iDAAA;CLg5FT;AoBx6FC;;;EAGE,oBAAA;EE7CF,cAAA;EAGA,0BAAA;EjB8DA,yBAAA;EACQ,iBAAA;CLy5FT;AoBx6FG;;EAEE,qBAAA;CpB06FL;AoBj6FD;EC3DE,eAAA;EACA,0BAAA;EACA,sBAAA;CrB+9FD;AqB79FC;;EAEE,eAAA;EACA,0BAAA;EACI,sBAAA;CrB+9FP;AqB79FC;EACE,eAAA;EACA,0BAAA;EACI,sBAAA;CrB+9FP;AqB79FC;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrB+9FP;AqB79FG;;;;;;;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBq+FT;AqBl+FC;;;EAGE,uBAAA;CrBo+FH;AqB/9FG;;;;;;;;;;;;;;;;;;EAME,0BAAA;EACI,sBAAA;CrB6+FT;AoB/9FD;ECTI,eAAA;EACA,0BAAA;CrB2+FH;AoBh+FD;EC9DE,eAAA;EACA,0BAAA;EACA,sBAAA;CrBiiGD;AqB/hGC;;EAEE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBiiGP;AqB/hGC;EACE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBiiGP;AqB/hGC;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBiiGP;AqB/hGG;;;;;;;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBuiGT;AqBpiGC;;;EAGE,uBAAA;CrBsiGH;AqBjiGG;;;;;;;;;;;;;;;;;;EAME,0BAAA;EACI,sBAAA;CrB+iGT;AoB9hGD;ECZI,eAAA;EACA,0BAAA;CrB6iGH;AoB9hGD;EClEE,eAAA;EACA,0BAAA;EACA,sBAAA;CrBmmGD;AqBjmGC;;EAEE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBmmGP;AqBjmGC;EACE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBmmGP;AqBjmGC;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBmmGP;AqBjmGG;;;;;;;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBymGT;AqBtmGC;;;EAGE,uBAAA;CrBwmGH;AqBnmGG;;;;;;;;;;;;;;;;;;EAME,0BAAA;EACI,sBAAA;CrBinGT;AoB5lGD;EChBI,eAAA;EACA,0BAAA;CrB+mGH;AoB5lGD;ECtEE,eAAA;EACA,0BAAA;EACA,sBAAA;CrBqqGD;AqBnqGC;;EAEE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBqqGP;AqBnqGC;EACE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBqqGP;AqBnqGC;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBqqGP;AqBnqGG;;;;;;;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrB2qGT;AqBxqGC;;;EAGE,uBAAA;CrB0qGH;AqBrqGG;;;;;;;;;;;;;;;;;;EAME,0BAAA;EACI,sBAAA;CrBmrGT;AoB1pGD;ECpBI,eAAA;EACA,0BAAA;CrBirGH;AoB1pGD;EC1EE,eAAA;EACA,0BAAA;EACA,sBAAA;CrBuuGD;AqBruGC;;EAEE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBuuGP;AqBruGC;EACE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBuuGP;AqBruGC;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrBuuGP;AqBruGG;;;;;;;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrB6uGT;AqB1uGC;;;EAGE,uBAAA;CrB4uGH;AqBvuGG;;;;;;;;;;;;;;;;;;EAME,0BAAA;EACI,sBAAA;CrBqvGT;AoBxtGD;ECxBI,eAAA;EACA,0BAAA;CrBmvGH;AoBxtGD;EC9EE,eAAA;EACA,0BAAA;EACA,sBAAA;CrByyGD;AqBvyGC;;EAEE,eAAA;EACA,0BAAA;EACI,sBAAA;CrByyGP;AqBvyGC;EACE,eAAA;EACA,0BAAA;EACI,sBAAA;CrByyGP;AqBvyGC;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrByyGP;AqBvyGG;;;;;;;;;EAGE,eAAA;EACA,0BAAA;EACI,sBAAA;CrB+yGT;AqB5yGC;;;EAGE,uBAAA;CrB8yGH;AqBzyGG;;;;;;;;;;;;;;;;;;EAME,0BAAA;EACI,sBAAA;CrBuzGT;AoBtxGD;EC5BI,eAAA;EACA,0BAAA;CrBqzGH;AoBjxGD;EACE,eAAA;EACA,oBAAA;EACA,iBAAA;CpBmxGD;AoBjxGC;;;;;EAKE,8BAAA;EfnCF,yBAAA;EACQ,iBAAA;CLuzGT;AoBlxGC;;;;EAIE,0BAAA;CpBoxGH;AoBlxGC;;EAEE,eAAA;EACA,2BAAA;EACA,8BAAA;CpBoxGH;AoBhxGG;;;;EAEE,eAAA;EACA,sBAAA;CpBoxGL;AoB3wGD;;ECrEE,mBAAA;EACA,gBAAA;EACA,uBAAA;EACA,mBAAA;CrBo1GD;AoB9wGD;;ECzEE,kBAAA;EACA,gBAAA;EACA,iBAAA;EACA,mBAAA;CrB21GD;AoBjxGD;;EC7EE,iBAAA;EACA,gBAAA;EACA,iBAAA;EACA,mBAAA;CrBk2GD;AoBhxGD;EACE,eAAA;EACA,YAAA;CpBkxGD;AoB9wGD;EACE,gBAAA;CpBgxGD;AoBzwGC;;;EACE,YAAA;CpB6wGH;AuBv6GD;EACE,WAAA;ElBoLA,yCAAA;EACK,oCAAA;EACG,iCAAA;CLsvGT;AuB16GC;EACE,WAAA;CvB46GH;AuBx6GD;EACE,cAAA;CvB06GD;AuBx6GC;EAAY,eAAA;CvB26Gb;AuB16GC;EAAY,mBAAA;CvB66Gb;AuB56GC;EAAY,yBAAA;CvB+6Gb;AuB56GD;EACE,mBAAA;EACA,UAAA;EACA,iBAAA;ElBuKA,gDAAA;EACQ,2CAAA;KAAA,wCAAA;EAOR,mCAAA;EACQ,8BAAA;KAAA,2BAAA;EAGR,yCAAA;EACQ,oCAAA;KAAA,iCAAA;CLgwGT;AwB18GD;EACE,sBAAA;EACA,SAAA;EACA,UAAA;EACA,iBAAA;EACA,uBAAA;EACA,uBAAA;EACA,yBAAA;EACA,oCAAA;EACA,mCAAA;CxB48GD;AwBx8GD;;EAEE,mBAAA;CxB08GD;AwBt8GD;EACE,WAAA;CxBw8GD;AwBp8GD;EACE,mBAAA;EACA,UAAA;EACA,QAAA;EACA,cAAA;EACA,cAAA;EACA,YAAA;EACA,iBAAA;EACA,eAAA;EACA,gBAAA;EACA,iBAAA;EACA,gBAAA;EACA,iBAAA;EACA,0BAAA;EACA,0BAAA;EACA,sCAAA;EACA,mBAAA;EnBsBA,oDAAA;EACQ,4CAAA;EmBrBR,qCAAA;UAAA,6BAAA;CxBu8GD;AwBl8GC;EACE,SAAA;EACA,WAAA;CxBo8GH;AwB79GD;ECzBE,YAAA;EACA,cAAA;EACA,iBAAA;EACA,0BAAA;CzBy/GD;AwBn+GD;EAmCI,eAAA;EACA,kBAAA;EACA,YAAA;EACA,oBAAA;EACA,wBAAA;EACA,eAAA;EACA,oBAAA;CxBm8GH;AwB77GC;;EAEE,sBAAA;EACA,eAAA;EACA,0BAAA;CxB+7GH;AwBz7GC;;;EAGE,eAAA;EACA,sBAAA;EACA,WAAA;EACA,0BAAA;CxB27GH;AwBl7GC;;;EAGE,eAAA;CxBo7GH;AwBh7GC;;EAEE,sBAAA;EACA,8BAAA;EACA,uBAAA;EE3GF,oEAAA;EF6GE,oBAAA;CxBk7GH;AwB76GD;EAGI,eAAA;CxB66GH;AwBh7GD;EAQI,WAAA;CxB26GH;AwBn6GD;EACE,WAAA;EACA,SAAA;CxBq6GD;AwB75GD;EACE,QAAA;EACA,YAAA;CxB+5GD;AwB35GD;EACE,eAAA;EACA,kBAAA;EACA,gBAAA;EACA,wBAAA;EACA,eAAA;EACA,oBAAA;CxB65GD;AwBz5GD;EACE,gBAAA;EACA,QAAA;EACA,SAAA;EACA,UAAA;EACA,OAAA;EACA,aAAA;CxB25GD;AwBv5GD;EACE,SAAA;EACA,WAAA;CxBy5GD;AwBj5GD;;EAII,cAAA;EACA,0BAAA;EACA,4BAAA;EACA,YAAA;CxBi5GH;AwBx5GD;;EAWI,UAAA;EACA,aAAA;EACA,mBAAA;CxBi5GH;AwB53GD;EAXE;IApEA,WAAA;IACA,SAAA;GxB+8GC;EwB54GD;IA1DA,QAAA;IACA,YAAA;GxBy8GC;CACF;A2BzlHD;;EAEE,mBAAA;EACA,sBAAA;EACA,uBAAA;C3B2lHD;A2B/lHD;;EAMI,mBAAA;EACA,YAAA;C3B6lHH;A2B3lHG;;;;;;;;EAIE,WAAA;C3BimHL;A2B3lHD;;;;EAKI,kBAAA;C3B4lHH;A2BvlHD;EACE,kBAAA;C3BylHD;A2B1lHD;;;EAOI,YAAA;C3BwlHH;A2B/lHD;;;EAYI,iBAAA;C3BwlHH;A2BplHD;EACE,iBAAA;C3BslHD;A2BllHD;EACE,eAAA;C3BolHD;A2BnlHC;EClDA,8BAAA;EACG,2BAAA;C5BwoHJ;A2BllHD;;EC/CE,6BAAA;EACG,0BAAA;C5BqoHJ;A2BjlHD;EACE,YAAA;C3BmlHD;A2BjlHD;EACE,iBAAA;C3BmlHD;A2BjlHD;;ECnEE,8BAAA;EACG,2BAAA;C5BwpHJ;A2BhlHD;ECjEE,6BAAA;EACG,0BAAA;C5BopHJ;A2B/kHD;;EAEE,WAAA;C3BilHD;A2BhkHD;EACE,kBAAA;EACA,mBAAA;C3BkkHD;A2BhkHD;EACE,mBAAA;EACA,oBAAA;C3BkkHD;A2B7jHD;EtB/CE,yDAAA;EACQ,iDAAA;CL+mHT;A2B7jHC;EtBnDA,yBAAA;EACQ,iBAAA;CLmnHT;A2B1jHD;EACE,eAAA;C3B4jHD;A2BzjHD;EACE,wBAAA;EACA,uBAAA;C3B2jHD;A2BxjHD;EACE,wBAAA;C3B0jHD;A2BnjHD;;;EAII,eAAA;EACA,YAAA;EACA,YAAA;EACA,gBAAA;C3BojHH;A2B3jHD;EAcM,YAAA;C3BgjHL;A2B9jHD;;;;EAsBI,iBAAA;EACA,eAAA;C3B8iHH;A2BziHC;EACE,iBAAA;C3B2iHH;A2BziHC;EACE,6BAAA;ECpKF,8BAAA;EACC,6BAAA;C5BgtHF;A2B1iHC;EACE,+BAAA;EChLF,2BAAA;EACC,0BAAA;C5B6tHF;A2B1iHD;EACE,iBAAA;C3B4iHD;A2B1iHD;;EC/KE,8BAAA;EACC,6BAAA;C5B6tHF;A2BziHD;EC7LE,2BAAA;EACC,0BAAA;C5ByuHF;A2BriHD;EACE,eAAA;EACA,YAAA;EACA,oBAAA;EACA,0BAAA;C3BuiHD;A2B3iHD;;EAOI,YAAA;EACA,oBAAA;EACA,UAAA;C3BwiHH;A2BjjHD;EAYI,YAAA;C3BwiHH;A2BpjHD;EAgBI,WAAA;C3BuiHH;A2BthHD;;;;EAKM,mBAAA;EACA,uBAAA;EACA,qBAAA;C3BuhHL;A6BjwHD;EACE,mBAAA;EACA,eAAA;EACA,0BAAA;C7BmwHD;A6BhwHC;EACE,YAAA;EACA,gBAAA;EACA,iBAAA;C7BkwHH;A6B3wHD;EAeI,mBAAA;EACA,WAAA;EAKA,YAAA;EAEA,YAAA;EACA,iBAAA;C7B0vHH;A6BjvHD;;;EV8BE,aAAA;EACA,mBAAA;EACA,gBAAA;EACA,uBAAA;EACA,mBAAA;CnBwtHD;AmBttHC;;;EACE,aAAA;EACA,kBAAA;CnB0tHH;AmBvtHC;;;;;;EAEE,aAAA;CnB6tHH;A6BnwHD;;;EVyBE,aAAA;EACA,kBAAA;EACA,gBAAA;EACA,iBAAA;EACA,mBAAA;CnB+uHD;AmB7uHC;;;EACE,aAAA;EACA,kBAAA;CnBivHH;AmB9uHC;;;;;;EAEE,aAAA;CnBovHH;A6BjxHD;;;EAGE,oBAAA;C7BmxHD;A6BjxHC;;;EACE,iBAAA;C7BqxHH;A6BjxHD;;EAEE,UAAA;EACA,oBAAA;EACA,uBAAA;C7BmxHD;A6B9wHD;EACE,kBAAA;EACA,gBAAA;EACA,oBAAA;EACA,eAAA;EACA,eAAA;EACA,mBAAA;EACA,0BAAA;EACA,0BAAA;EACA,mBAAA;C7BgxHD;A6B7wHC;EACE,kBAAA;EACA,gBAAA;EACA,mBAAA;C7B+wHH;A6B7wHC;EACE,mBAAA;EACA,gBAAA;EACA,mBAAA;C7B+wHH;A6BnyHD;;EA0BI,cAAA;C7B6wHH;A6BxwHD;;;;;;;EDhGE,8BAAA;EACG,2BAAA;C5Bi3HJ;A6BzwHD;EACE,gBAAA;C7B2wHD;A6BzwHD;;;;;;;EDpGE,6BAAA;EACG,0BAAA;C5Bs3HJ;A6B1wHD;EACE,eAAA;C7B4wHD;A6BvwHD;EACE,mBAAA;EAGA,aAAA;EACA,oBAAA;C7BuwHD;A6B5wHD;EAUI,mBAAA;C7BqwHH;A6B/wHD;EAYM,kBAAA;C7BswHL;A6BnwHG;;;EAGE,WAAA;C7BqwHL;A6BhwHC;;EAGI,mBAAA;C7BiwHL;A6B9vHC;;EAGI,WAAA;EACA,kBAAA;C7B+vHL;A8B15HD;EACE,iBAAA;EACA,gBAAA;EACA,iBAAA;C9B45HD;A8B/5HD;EAOI,mBAAA;EACA,eAAA;C9B25HH;A8Bn6HD;EAWM,mBAAA;EACA,eAAA;EACA,mBAAA;C9B25HL;A8B15HK;;EAEE,sBAAA;EACA,0BAAA;C9B45HP;A8Bv5HG;EACE,eAAA;C9By5HL;A8Bv5HK;;EAEE,eAAA;EACA,sBAAA;EACA,8BAAA;EACA,oBAAA;C9By5HP;A8Bl5HG;;;EAGE,0BAAA;EACA,sBAAA;C9Bo5HL;A8B77HD;ELHE,YAAA;EACA,cAAA;EACA,iBAAA;EACA,0BAAA;CzBm8HD;A8Bn8HD;EA0DI,gBAAA;C9B44HH;A8Bn4HD;EACE,iCAAA;C9Bq4HD;A8Bt4HD;EAGI,YAAA;EAEA,oBAAA;C9Bq4HH;A8B14HD;EASM,kBAAA;EACA,wBAAA;EACA,8BAAA;EACA,2BAAA;C9Bo4HL;A8Bn4HK;EACE,sCAAA;C9Bq4HP;A8B/3HK;;;EAGE,eAAA;EACA,0BAAA;EACA,0BAAA;EACA,iCAAA;EACA,gBAAA;C9Bi4HP;A8B53HC;EAqDA,YAAA;EA8BA,iBAAA;C9B6yHD;A8Bh4HC;EAwDE,YAAA;C9B20HH;A8Bn4HC;EA0DI,mBAAA;EACA,mBAAA;C9B40HL;A8Bv4HC;EAgEE,UAAA;EACA,WAAA;C9B00HH;A8B9zHD;EAAA;IAPM,oBAAA;IACA,UAAA;G9By0HH;E8Bn0HH;IAJQ,iBAAA;G9B00HL;CACF;A8Bp5HC;EAuFE,gBAAA;EACA,mBAAA;C9Bg0HH;A8Bx5HC;;;EA8FE,0BAAA;C9B+zHH;A8BjzHD;EAAA;IATM,iCAAA;IACA,2BAAA;G9B8zHH;E8BtzHH;;;IAHM,6BAAA;G9B8zHH;CACF;A8B/5HD;EAEI,YAAA;C9Bg6HH;A8Bl6HD;EAMM,mBAAA;C9B+5HL;A8Br6HD;EASM,iBAAA;C9B+5HL;A8B15HK;;;EAGE,eAAA;EACA,0BAAA;C9B45HP;A8Bp5HD;EAEI,YAAA;C9Bq5HH;A8Bv5HD;EAIM,gBAAA;EACA,eAAA;C9Bs5HL;A8B14HD;EACE,YAAA;C9B44HD;A8B74HD;EAII,YAAA;C9B44HH;A8Bh5HD;EAMM,mBAAA;EACA,mBAAA;C9B64HL;A8Bp5HD;EAYI,UAAA;EACA,WAAA;C9B24HH;A8B/3HD;EAAA;IAPM,oBAAA;IACA,UAAA;G9B04HH;E8Bp4HH;IAJQ,iBAAA;G9B24HL;CACF;A8Bn4HD;EACE,iBAAA;C9Bq4HD;A8Bt4HD;EAKI,gBAAA;EACA,mBAAA;C9Bo4HH;A8B14HD;;;EAYI,0BAAA;C9Bm4HH;A8Br3HD;EAAA;IATM,iCAAA;IACA,2BAAA;G9Bk4HH;E8B13HH;;;IAHM,6BAAA;G9Bk4HH;CACF;A8Bz3HD;EAEI,cAAA;C9B03HH;A8B53HD;EAKI,eAAA;C9B03HH;A8Bj3HD;EAEE,iBAAA;EF3OA,2BAAA;EACC,0BAAA;C5B8lIF;A+BxlID;EACE,mBAAA;EACA,iBAAA;EACA,oBAAA;EACA,8BAAA;C/B0lID;A+BllID;EAAA;IAFI,mBAAA;G/BwlID;CACF;A+BzkID;EAAA;IAFI,YAAA;G/B+kID;CACF;A+BjkID;EACE,oBAAA;EACA,oBAAA;EACA,mBAAA;EACA,kCAAA;EACA,2DAAA;UAAA,mDAAA;EAEA,kCAAA;C/BkkID;A+BhkIC;EACE,iBAAA;C/BkkIH;A+BtiID;EAAA;IAxBI,YAAA;IACA,cAAA;IACA,yBAAA;YAAA,iBAAA;G/BkkID;E+BhkIC;IACE,0BAAA;IACA,wBAAA;IACA,kBAAA;IACA,6BAAA;G/BkkIH;E+B/jIC;IACE,oBAAA;G/BikIH;E+B5jIC;;;IAGE,gBAAA;IACA,iBAAA;G/B8jIH;CACF;A+B1jID;;EAGI,kBAAA;C/B2jIH;A+BtjIC;EAAA;;IAFI,kBAAA;G/B6jIH;CACF;A+BpjID;;;;EAII,oBAAA;EACA,mBAAA;C/BsjIH;A+BhjIC;EAAA;;;;IAHI,gBAAA;IACA,eAAA;G/B0jIH;CACF;A+B9iID;EACE,cAAA;EACA,sBAAA;C/BgjID;A+B3iID;EAAA;IAFI,iBAAA;G/BijID;CACF;A+B7iID;;EAEE,gBAAA;EACA,SAAA;EACA,QAAA;EACA,cAAA;C/B+iID;A+BziID;EAAA;;IAFI,iBAAA;G/BgjID;CACF;A+B9iID;EACE,OAAA;EACA,sBAAA;C/BgjID;A+B9iID;EACE,UAAA;EACA,iBAAA;EACA,sBAAA;C/BgjID;A+B1iID;EACE,YAAA;EACA,mBAAA;EACA,gBAAA;EACA,kBAAA;EACA,aAAA;C/B4iID;A+B1iIC;;EAEE,sBAAA;C/B4iIH;A+BrjID;EAaI,eAAA;C/B2iIH;A+BliID;EALI;;IAEE,mBAAA;G/B0iIH;CACF;A+BhiID;EACE,mBAAA;EACA,aAAA;EACA,mBAAA;EACA,kBAAA;EC9LA,gBAAA;EACA,mBAAA;ED+LA,8BAAA;EACA,uBAAA;EACA,8BAAA;EACA,mBAAA;C/BmiID;A+B/hIC;EACE,WAAA;C/BiiIH;A+B/iID;EAmBI,eAAA;EACA,YAAA;EACA,YAAA;EACA,mBAAA;C/B+hIH;A+BrjID;EAyBI,gBAAA;C/B+hIH;A+BzhID;EAAA;IAFI,cAAA;G/B+hID;CACF;A+BthID;EACE,oBAAA;C/BwhID;A+BzhID;EAII,kBAAA;EACA,qBAAA;EACA,kBAAA;C/BwhIH;A+B5/HC;EAAA;IAtBI,iBAAA;IACA,YAAA;IACA,YAAA;IACA,cAAA;IACA,8BAAA;IACA,UAAA;IACA,yBAAA;YAAA,iBAAA;G/BshIH;E+BtgID;;IAbM,2BAAA;G/BuhIL;E+B1gID;IAVM,kBAAA;G/BuhIL;E+BthIK;;IAEE,uBAAA;G/BwhIP;CACF;A+BtgID;EAAA;IAXI,YAAA;IACA,UAAA;G/BqhID;E+B3gIH;IAPM,YAAA;G/BqhIH;E+B9gIH;IALQ,kBAAA;IACA,qBAAA;G/BshIL;CACF;A+B3gID;EACE,mBAAA;EACA,oBAAA;EACA,mBAAA;EACA,kCAAA;EACA,qCAAA;E1B9NA,6FAAA;EACQ,qFAAA;E2B/DR,gBAAA;EACA,mBAAA;ChC4yID;AkB5xHD;EAAA;IA9DM,sBAAA;IACA,iBAAA;IACA,uBAAA;GlB81HH;EkBlyHH;IAvDM,sBAAA;IACA,YAAA;IACA,uBAAA;GlB41HH;EkBvyHH;IAhDM,sBAAA;GlB01HH;EkB1yHH;IA5CM,sBAAA;IACA,uBAAA;GlBy1HH;EkB9yHH;;;IAtCQ,YAAA;GlBy1HL;EkBnzHH;IAhCM,YAAA;GlBs1HH;EkBtzHH;IA5BM,iBAAA;IACA,uBAAA;GlBq1HH;EkB1zHH;;IApBM,sBAAA;IACA,cAAA;IACA,iBAAA;IACA,uBAAA;GlBk1HH;EkBj0HH;;IAdQ,gBAAA;GlBm1HL;EkBr0HH;;IATM,mBAAA;IACA,eAAA;GlBk1HH;EkB10HH;IAHM,OAAA;GlBg1HH;CACF;A+BpjIC;EAAA;IANI,mBAAA;G/B8jIH;E+B5jIG;IACE,iBAAA;G/B8jIL;CACF;A+B7iID;EAAA;IARI,YAAA;IACA,UAAA;IACA,eAAA;IACA,gBAAA;IACA,eAAA;IACA,kBAAA;I1BzPF,yBAAA;IACQ,iBAAA;GLmzIP;CACF;A+BnjID;EACE,cAAA;EHpUA,2BAAA;EACC,0BAAA;C5B03IF;A+BnjID;EACE,iBAAA;EHzUA,6BAAA;EACC,4BAAA;EAOD,8BAAA;EACC,6BAAA;C5By3IF;A+B/iID;EChVE,gBAAA;EACA,mBAAA;ChCk4ID;A+BhjIC;ECnVA,iBAAA;EACA,oBAAA;ChCs4ID;A+BjjIC;ECtVA,iBAAA;EACA,oBAAA;ChC04ID;A+B3iID;EChWE,iBAAA;EACA,oBAAA;ChC84ID;A+BviID;EAAA;IAJI,YAAA;IACA,kBAAA;IACA,mBAAA;G/B+iID;CACF;A+BlhID;EAhBE;IExWA,uBAAA;GjC84IC;E+BriID;IE5WA,wBAAA;IF8WE,oBAAA;G/BuiID;E+BziID;IAKI,gBAAA;G/BuiIH;CACF;A+B9hID;EACE,0BAAA;EACA,sBAAA;C/BgiID;A+BliID;EAKI,eAAA;C/BgiIH;A+B/hIG;;EAEE,eAAA;EACA,8BAAA;C/BiiIL;A+B1iID;EAcI,eAAA;C/B+hIH;A+B7iID;EAmBM,eAAA;C/B6hIL;A+B3hIK;;EAEE,eAAA;EACA,8BAAA;C/B6hIP;A+BzhIK;;;EAGE,eAAA;EACA,0BAAA;C/B2hIP;A+BvhIK;;;EAGE,eAAA;EACA,8BAAA;C/ByhIP;A+BjkID;EA8CI,sBAAA;C/BshIH;A+BrhIG;;EAEE,0BAAA;C/BuhIL;A+BxkID;EAoDM,0BAAA;C/BuhIL;A+B3kID;;EA0DI,sBAAA;C/BqhIH;A+B9gIK;;;EAGE,0BAAA;EACA,eAAA;C/BghIP;A+B/+HC;EAAA;IAzBQ,eAAA;G/B4gIP;E+B3gIO;;IAEE,eAAA;IACA,8BAAA;G/B6gIT;E+BzgIO;;;IAGE,eAAA;IACA,0BAAA;G/B2gIT;E+BvgIO;;;IAGE,eAAA;IACA,8BAAA;G/BygIT;CACF;A+B3mID;EA8GI,eAAA;C/BggIH;A+B//HG;EACE,eAAA;C/BigIL;A+BjnID;EAqHI,eAAA;C/B+/HH;A+B9/HG;;EAEE,eAAA;C/BggIL;A+B5/HK;;;;EAEE,eAAA;C/BggIP;A+Bx/HD;EACE,0BAAA;EACA,sBAAA;C/B0/HD;A+B5/HD;EAKI,eAAA;C/B0/HH;A+Bz/HG;;EAEE,eAAA;EACA,8BAAA;C/B2/HL;A+BpgID;EAcI,eAAA;C/By/HH;A+BvgID;EAmBM,eAAA;C/Bu/HL;A+Br/HK;;EAEE,eAAA;EACA,8BAAA;C/Bu/HP;A+Bn/HK;;;EAGE,eAAA;EACA,0BAAA;C/Bq/HP;A+Bj/HK;;;EAGE,eAAA;EACA,8BAAA;C/Bm/HP;A+B3hID;EA+CI,sBAAA;C/B++HH;A+B9+HG;;EAEE,0BAAA;C/Bg/HL;A+BliID;EAqDM,0BAAA;C/Bg/HL;A+BriID;;EA2DI,sBAAA;C/B8+HH;A+Bx+HK;;;EAGE,0BAAA;EACA,eAAA;C/B0+HP;A+Bn8HC;EAAA;IA/BQ,sBAAA;G/Bs+HP;E+Bv8HD;IA5BQ,0BAAA;G/Bs+HP;E+B18HD;IAzBQ,eAAA;G/Bs+HP;E+Br+HO;;IAEE,eAAA;IACA,8BAAA;G/Bu+HT;E+Bn+HO;;;IAGE,eAAA;IACA,0BAAA;G/Bq+HT;E+Bj+HO;;;IAGE,eAAA;IACA,8BAAA;G/Bm+HT;CACF;A+B3kID;EA+GI,eAAA;C/B+9HH;A+B99HG;EACE,eAAA;C/Bg+HL;A+BjlID;EAsHI,eAAA;C/B89HH;A+B79HG;;EAEE,eAAA;C/B+9HL;A+B39HK;;;;EAEE,eAAA;C/B+9HP;AkCzmJD;EACE,kBAAA;EACA,oBAAA;EACA,iBAAA;EACA,0BAAA;EACA,mBAAA;ClC2mJD;AkChnJD;EAQI,sBAAA;ClC2mJH;AkCnnJD;EAWM,kBAAA;EACA,eAAA;EACA,eAAA;ClC2mJL;AkCxnJD;EAkBI,eAAA;ClCymJH;AmC7nJD;EACE,sBAAA;EACA,gBAAA;EACA,eAAA;EACA,mBAAA;CnC+nJD;AmCnoJD;EAOI,gBAAA;CnC+nJH;AmCtoJD;;EAUM,mBAAA;EACA,YAAA;EACA,kBAAA;EACA,wBAAA;EACA,sBAAA;EACA,eAAA;EACA,0BAAA;EACA,0BAAA;EACA,kBAAA;CnCgoJL;AmC9nJG;;EAGI,eAAA;EPXN,+BAAA;EACG,4BAAA;C5B2oJJ;AmC7nJG;;EPvBF,gCAAA;EACG,6BAAA;C5BwpJJ;AmCxnJG;;;;EAEE,WAAA;EACA,eAAA;EACA,0BAAA;EACA,sBAAA;CnC4nJL;AmCtnJG;;;;;;EAGE,WAAA;EACA,eAAA;EACA,0BAAA;EACA,sBAAA;EACA,gBAAA;CnC2nJL;AmClrJD;;;;;;EAkEM,eAAA;EACA,0BAAA;EACA,sBAAA;EACA,oBAAA;CnCwnJL;AmC/mJD;;EC3EM,mBAAA;EACA,gBAAA;EACA,uBAAA;CpC8rJL;AoC5rJG;;ERKF,+BAAA;EACG,4BAAA;C5B2rJJ;AoC3rJG;;ERTF,gCAAA;EACG,6BAAA;C5BwsJJ;AmC1nJD;;EChFM,kBAAA;EACA,gBAAA;EACA,iBAAA;CpC8sJL;AoC5sJG;;ERKF,+BAAA;EACG,4BAAA;C5B2sJJ;AoC3sJG;;ERTF,gCAAA;EACG,6BAAA;C5BwtJJ;AqC3tJD;EACE,gBAAA;EACA,eAAA;EACA,iBAAA;EACA,mBAAA;CrC6tJD;AqCjuJD;EAOI,gBAAA;CrC6tJH;AqCpuJD;;EAUM,sBAAA;EACA,kBAAA;EACA,0BAAA;EACA,0BAAA;EACA,oBAAA;CrC8tJL;AqC5uJD;;EAmBM,sBAAA;EACA,0BAAA;CrC6tJL;AqCjvJD;;EA2BM,aAAA;CrC0tJL;AqCrvJD;;EAkCM,YAAA;CrCutJL;AqCzvJD;;;;EA2CM,eAAA;EACA,0BAAA;EACA,oBAAA;CrCotJL;AsClwJD;EACE,gBAAA;EACA,wBAAA;EACA,eAAA;EACA,kBAAA;EACA,eAAA;EACA,eAAA;EACA,mBAAA;EACA,oBAAA;EACA,yBAAA;EACA,qBAAA;CtCowJD;AsChwJG;;EAEE,eAAA;EACA,sBAAA;EACA,gBAAA;CtCkwJL;AsC7vJC;EACE,cAAA;CtC+vJH;AsC3vJC;EACE,mBAAA;EACA,UAAA;CtC6vJH;AsCtvJD;ECtCE,0BAAA;CvC+xJD;AuC5xJG;;EAEE,0BAAA;CvC8xJL;AsCzvJD;EC1CE,0BAAA;CvCsyJD;AuCnyJG;;EAEE,0BAAA;CvCqyJL;AsC5vJD;EC9CE,0BAAA;CvC6yJD;AuC1yJG;;EAEE,0BAAA;CvC4yJL;AsC/vJD;EClDE,0BAAA;CvCozJD;AuCjzJG;;EAEE,0BAAA;CvCmzJL;AsClwJD;ECtDE,0BAAA;CvC2zJD;AuCxzJG;;EAEE,0BAAA;CvC0zJL;AsCrwJD;EC1DE,0BAAA;CvCk0JD;AuC/zJG;;EAEE,0BAAA;CvCi0JL;AwCn0JD;EACE,sBAAA;EACA,gBAAA;EACA,iBAAA;EACA,gBAAA;EACA,kBAAA;EACA,eAAA;EACA,eAAA;EACA,uBAAA;EACA,oBAAA;EACA,mBAAA;EACA,0BAAA;EACA,oBAAA;CxCq0JD;AwCl0JC;EACE,cAAA;CxCo0JH;AwCh0JC;EACE,mBAAA;EACA,UAAA;CxCk0JH;AwC/zJC;;EAEE,OAAA;EACA,iBAAA;CxCi0JH;AwC5zJG;;EAEE,eAAA;EACA,sBAAA;EACA,gBAAA;CxC8zJL;AwCzzJC;;EAEE,eAAA;EACA,0BAAA;CxC2zJH;AwCxzJC;EACE,aAAA;CxC0zJH;AwCvzJC;EACE,kBAAA;CxCyzJH;AwCtzJC;EACE,iBAAA;CxCwzJH;AyCl3JD;EACE,kBAAA;EACA,qBAAA;EACA,oBAAA;EACA,eAAA;EACA,0BAAA;CzCo3JD;AyCz3JD;;EASI,eAAA;CzCo3JH;AyC73JD;EAaI,oBAAA;EACA,gBAAA;EACA,iBAAA;CzCm3JH;AyCl4JD;EAmBI,0BAAA;CzCk3JH;AyC/2JC;;EAEE,mBAAA;CzCi3JH;AyCz4JD;EA4BI,gBAAA;CzCg3JH;AyC91JD;EAAA;IAdI,kBAAA;IACA,qBAAA;GzCg3JD;EyC92JC;;IAEE,mBAAA;IACA,oBAAA;GzCg3JH;EyCx2JH;;IAHM,gBAAA;GzC+2JH;CACF;A0C15JD;EACE,eAAA;EACA,aAAA;EACA,oBAAA;EACA,wBAAA;EACA,0BAAA;EACA,0BAAA;EACA,mBAAA;ErCiLA,4CAAA;EACK,uCAAA;EACG,oCAAA;CL4uJT;A0Ct6JD;;EAaI,kBAAA;EACA,mBAAA;C1C65JH;A0Cz5JC;;;EAGE,sBAAA;C1C25JH;A0Ch7JD;EA0BI,aAAA;EACA,eAAA;C1Cy5JH;A2Cl7JD;EACE,cAAA;EACA,oBAAA;EACA,8BAAA;EACA,mBAAA;C3Co7JD;A2Cx7JD;EAQI,cAAA;EAEA,eAAA;C3Ck7JH;A2C57JD;EAeI,kBAAA;C3Cg7JH;A2C/7JD;;EAqBI,iBAAA;C3C86JH;A2Cn8JD;EAyBI,gBAAA;C3C66JH;A2Cr6JD;;EAEE,oBAAA;C3Cu6JD;A2Cz6JD;;EAMI,mBAAA;EACA,UAAA;EACA,aAAA;EACA,eAAA;C3Cu6JH;A2C/5JD;ECvDE,0BAAA;EACA,sBAAA;EACA,eAAA;C5Cy9JD;A2Cp6JD;EClDI,0BAAA;C5Cy9JH;A2Cv6JD;EC/CI,eAAA;C5Cy9JH;A2Ct6JD;EC3DE,0BAAA;EACA,sBAAA;EACA,eAAA;C5Co+JD;A2C36JD;ECtDI,0BAAA;C5Co+JH;A2C96JD;ECnDI,eAAA;C5Co+JH;A2C76JD;EC/DE,0BAAA;EACA,sBAAA;EACA,eAAA;C5C++JD;A2Cl7JD;EC1DI,0BAAA;C5C++JH;A2Cr7JD;ECvDI,eAAA;C5C++JH;A2Cp7JD;ECnEE,0BAAA;EACA,sBAAA;EACA,eAAA;C5C0/JD;A2Cz7JD;EC9DI,0BAAA;C5C0/JH;A2C57JD;EC3DI,eAAA;C5C0/JH;A6C5/JD;EACE;IAAQ,4BAAA;G7C+/JP;E6C9/JD;IAAQ,yBAAA;G7CigKP;CACF;A6C9/JD;EACE;IAAQ,4BAAA;G7CigKP;E6ChgKD;IAAQ,yBAAA;G7CmgKP;CACF;A6CtgKD;EACE;IAAQ,4BAAA;G7CigKP;E6ChgKD;IAAQ,yBAAA;G7CmgKP;CACF;A6C5/JD;EACE,iBAAA;EACA,aAAA;EACA,oBAAA;EACA,0BAAA;EACA,mBAAA;ExCsCA,uDAAA;EACQ,+CAAA;CLy9JT;A6C3/JD;EACE,YAAA;EACA,UAAA;EACA,aAAA;EACA,gBAAA;EACA,kBAAA;EACA,eAAA;EACA,mBAAA;EACA,0BAAA;ExCyBA,uDAAA;EACQ,+CAAA;EAyHR,oCAAA;EACK,+BAAA;EACG,4BAAA;CL62JT;A6Cx/JD;;ECCI,8MAAA;EACA,yMAAA;EACA,sMAAA;EDAF,mCAAA;UAAA,2BAAA;C7C4/JD;A6Cr/JD;;ExC5CE,2DAAA;EACK,sDAAA;EACG,mDAAA;CLqiKT;A6Cl/JD;EErEE,0BAAA;C/C0jKD;A+CvjKC;EDgDE,8MAAA;EACA,yMAAA;EACA,sMAAA;C9C0gKH;A6Ct/JD;EEzEE,0BAAA;C/CkkKD;A+C/jKC;EDgDE,8MAAA;EACA,yMAAA;EACA,sMAAA;C9CkhKH;A6C1/JD;EE7EE,0BAAA;C/C0kKD;A+CvkKC;EDgDE,8MAAA;EACA,yMAAA;EACA,sMAAA;C9C0hKH;A6C9/JD;EEjFE,0BAAA;C/CklKD;A+C/kKC;EDgDE,8MAAA;EACA,yMAAA;EACA,sMAAA;C9CkiKH;AgD1lKD;EAEE,iBAAA;ChD2lKD;AgDzlKC;EACE,cAAA;ChD2lKH;AgDvlKD;;EAEE,QAAA;EACA,iBAAA;ChDylKD;AgDtlKD;EACE,eAAA;ChDwlKD;AgDrlKD;EACE,eAAA;ChDulKD;AgDplKC;EACE,gBAAA;ChDslKH;AgDllKD;;EAEE,mBAAA;ChDolKD;AgDjlKD;;EAEE,oBAAA;ChDmlKD;AgDhlKD;;;EAGE,oBAAA;EACA,oBAAA;ChDklKD;AgD/kKD;EACE,uBAAA;ChDilKD;AgD9kKD;EACE,uBAAA;ChDglKD;AgD5kKD;EACE,cAAA;EACA,mBAAA;ChD8kKD;AgDxkKD;EACE,gBAAA;EACA,iBAAA;ChD0kKD;AiDjoKD;EAEE,oBAAA;EACA,gBAAA;CjDkoKD;AiD1nKD;EACE,mBAAA;EACA,eAAA;EACA,mBAAA;EAEA,oBAAA;EACA,0BAAA;EACA,0BAAA;CjD2nKD;AiDxnKC;ErB3BA,6BAAA;EACC,4BAAA;C5BspKF;AiDznKC;EACE,iBAAA;ErBvBF,gCAAA;EACC,+BAAA;C5BmpKF;AiDlnKD;;EAEE,eAAA;CjDonKD;AiDtnKD;;EAKI,eAAA;CjDqnKH;AiDjnKC;;;;EAEE,sBAAA;EACA,eAAA;EACA,0BAAA;CjDqnKH;AiDjnKD;EACE,YAAA;EACA,iBAAA;CjDmnKD;AiD9mKC;;;EAGE,0BAAA;EACA,eAAA;EACA,oBAAA;CjDgnKH;AiDrnKC;;;EASI,eAAA;CjDinKL;AiD1nKC;;;EAYI,eAAA;CjDmnKL;AiD9mKC;;;EAGE,WAAA;EACA,eAAA;EACA,0BAAA;EACA,sBAAA;CjDgnKH;AiDtnKC;;;;;;;;;EAYI,eAAA;CjDqnKL;AiDjoKC;;;EAeI,eAAA;CjDunKL;AkDztKC;EACE,eAAA;EACA,0BAAA;ClD2tKH;AkDztKG;;EAEE,eAAA;ClD2tKL;AkD7tKG;;EAKI,eAAA;ClD4tKP;AkDztKK;;;;EAEE,eAAA;EACA,0BAAA;ClD6tKP;AkD3tKK;;;;;;EAGE,YAAA;EACA,0BAAA;EACA,sBAAA;ClDguKP;AkDtvKC;EACE,eAAA;EACA,0BAAA;ClDwvKH;AkDtvKG;;EAEE,eAAA;ClDwvKL;AkD1vKG;;EAKI,eAAA;ClDyvKP;AkDtvKK;;;;EAEE,eAAA;EACA,0BAAA;ClD0vKP;AkDxvKK;;;;;;EAGE,YAAA;EACA,0BAAA;EACA,sBAAA;ClD6vKP;AkDnxKC;EACE,eAAA;EACA,0BAAA;ClDqxKH;AkDnxKG;;EAEE,eAAA;ClDqxKL;AkDvxKG;;EAKI,eAAA;ClDsxKP;AkDnxKK;;;;EAEE,eAAA;EACA,0BAAA;ClDuxKP;AkDrxKK;;;;;;EAGE,YAAA;EACA,0BAAA;EACA,sBAAA;ClD0xKP;AkDhzKC;EACE,eAAA;EACA,0BAAA;ClDkzKH;AkDhzKG;;EAEE,eAAA;ClDkzKL;AkDpzKG;;EAKI,eAAA;ClDmzKP;AkDhzKK;;;;EAEE,eAAA;EACA,0BAAA;ClDozKP;AkDlzKK;;;;;;EAGE,YAAA;EACA,0BAAA;EACA,sBAAA;ClDuzKP;AiDttKD;EACE,cAAA;EACA,mBAAA;CjDwtKD;AiDttKD;EACE,iBAAA;EACA,iBAAA;CjDwtKD;AmDl1KD;EACE,oBAAA;EACA,0BAAA;EACA,8BAAA;EACA,mBAAA;E9C0DA,kDAAA;EACQ,0CAAA;CL2xKT;AmDj1KD;EACE,cAAA;CnDm1KD;AmD90KD;EACE,mBAAA;EACA,qCAAA;EvBpBA,6BAAA;EACC,4BAAA;C5Bq2KF;AmDp1KD;EAMI,eAAA;CnDi1KH;AmD50KD;EACE,cAAA;EACA,iBAAA;EACA,gBAAA;EACA,eAAA;CnD80KD;AmDl1KD;;;;;EAWI,eAAA;CnD80KH;AmDz0KD;EACE,mBAAA;EACA,0BAAA;EACA,8BAAA;EvBxCA,gCAAA;EACC,+BAAA;C5Bo3KF;AmDn0KD;;EAGI,iBAAA;CnDo0KH;AmDv0KD;;EAMM,oBAAA;EACA,iBAAA;CnDq0KL;AmDj0KG;;EAEI,cAAA;EvBvEN,6BAAA;EACC,4BAAA;C5B24KF;AmD/zKG;;EAEI,iBAAA;EvBvEN,gCAAA;EACC,+BAAA;C5By4KF;AmDx1KD;EvB1DE,2BAAA;EACC,0BAAA;C5Bq5KF;AmD3zKD;EAEI,oBAAA;CnD4zKH;AmDzzKD;EACE,oBAAA;CnD2zKD;AmDnzKD;;;EAII,iBAAA;CnDozKH;AmDxzKD;;;EAOM,mBAAA;EACA,oBAAA;CnDszKL;AmD9zKD;;EvBzGE,6BAAA;EACC,4BAAA;C5B26KF;AmDn0KD;;;;EAmBQ,4BAAA;EACA,6BAAA;CnDszKP;AmD10KD;;;;;;;;EAwBU,4BAAA;CnD4zKT;AmDp1KD;;;;;;;;EA4BU,6BAAA;CnDk0KT;AmD91KD;;EvBjGE,gCAAA;EACC,+BAAA;C5Bm8KF;AmDn2KD;;;;EAyCQ,+BAAA;EACA,gCAAA;CnDg0KP;AmD12KD;;;;;;;;EA8CU,+BAAA;CnDs0KT;AmDp3KD;;;;;;;;EAkDU,gCAAA;CnD40KT;AmD93KD;;;;EA2DI,8BAAA;CnDy0KH;AmDp4KD;;EA+DI,cAAA;CnDy0KH;AmDx4KD;;EAmEI,UAAA;CnDy0KH;AmD54KD;;;;;;;;;;;;EA0EU,eAAA;CnDg1KT;AmD15KD;;;;;;;;;;;;EA8EU,gBAAA;CnD01KT;AmDx6KD;;;;;;;;EAuFU,iBAAA;CnD21KT;AmDl7KD;;;;;;;;EAgGU,iBAAA;CnD41KT;AmD57KD;EAsGI,UAAA;EACA,iBAAA;CnDy1KH;AmD/0KD;EACE,oBAAA;CnDi1KD;AmDl1KD;EAKI,iBAAA;EACA,mBAAA;CnDg1KH;AmDt1KD;EASM,gBAAA;CnDg1KL;AmDz1KD;EAcI,iBAAA;CnD80KH;AmD51KD;;EAkBM,8BAAA;CnD80KL;AmDh2KD;EAuBI,cAAA;CnD40KH;AmDn2KD;EAyBM,iCAAA;CnD60KL;AmDt0KD;EC1PE,sBAAA;CpDmkLD;AoDjkLC;EACE,eAAA;EACA,0BAAA;EACA,sBAAA;CpDmkLH;AoDtkLC;EAMI,0BAAA;CpDmkLL;AoDzkLC;EASI,eAAA;EACA,0BAAA;CpDmkLL;AoDhkLC;EAEI,6BAAA;CpDikLL;AmDr1KD;EC7PE,sBAAA;CpDqlLD;AoDnlLC;EACE,eAAA;EACA,0BAAA;EACA,sBAAA;CpDqlLH;AoDxlLC;EAMI,0BAAA;CpDqlLL;AoD3lLC;EASI,eAAA;EACA,0BAAA;CpDqlLL;AoDllLC;EAEI,6BAAA;CpDmlLL;AmDp2KD;EChQE,sBAAA;CpDumLD;AoDrmLC;EACE,eAAA;EACA,0BAAA;EACA,sBAAA;CpDumLH;AoD1mLC;EAMI,0BAAA;CpDumLL;AoD7mLC;EASI,eAAA;EACA,0BAAA;CpDumLL;AoDpmLC;EAEI,6BAAA;CpDqmLL;AmDn3KD;ECnQE,sBAAA;CpDynLD;AoDvnLC;EACE,eAAA;EACA,0BAAA;EACA,sBAAA;CpDynLH;AoD5nLC;EAMI,0BAAA;CpDynLL;AoD/nLC;EASI,eAAA;EACA,0BAAA;CpDynLL;AoDtnLC;EAEI,6BAAA;CpDunLL;AmDl4KD;ECtQE,sBAAA;CpD2oLD;AoDzoLC;EACE,eAAA;EACA,0BAAA;EACA,sBAAA;CpD2oLH;AoD9oLC;EAMI,0BAAA;CpD2oLL;AoDjpLC;EASI,eAAA;EACA,0BAAA;CpD2oLL;AoDxoLC;EAEI,6BAAA;CpDyoLL;AmDj5KD;ECzQE,sBAAA;CpD6pLD;AoD3pLC;EACE,eAAA;EACA,0BAAA;EACA,sBAAA;CpD6pLH;AoDhqLC;EAMI,0BAAA;CpD6pLL;AoDnqLC;EASI,eAAA;EACA,0BAAA;CpD6pLL;AoD1pLC;EAEI,6BAAA;CpD2pLL;AqD3qLD;EACE,mBAAA;EACA,eAAA;EACA,UAAA;EACA,WAAA;EACA,iBAAA;CrD6qLD;AqDlrLD;;;;;EAYI,mBAAA;EACA,OAAA;EACA,QAAA;EACA,UAAA;EACA,aAAA;EACA,YAAA;EACA,UAAA;CrD6qLH;AqDxqLD;EACE,uBAAA;CrD0qLD;AqDtqLD;EACE,oBAAA;CrDwqLD;AsDnsLD;EACE,iBAAA;EACA,cAAA;EACA,oBAAA;EACA,0BAAA;EACA,0BAAA;EACA,mBAAA;EjDwDA,wDAAA;EACQ,gDAAA;CL8oLT;AsD7sLD;EASI,mBAAA;EACA,kCAAA;CtDusLH;AsDlsLD;EACE,cAAA;EACA,mBAAA;CtDosLD;AsDlsLD;EACE,aAAA;EACA,mBAAA;CtDosLD;AuD1tLD;EACE,aAAA;EACA,gBAAA;EACA,kBAAA;EACA,eAAA;EACA,eAAA;EACA,6BAAA;EjCRA,aAAA;EAGA,0BAAA;CtBmuLD;AuD3tLC;;EAEE,eAAA;EACA,sBAAA;EACA,gBAAA;EjCfF,aAAA;EAGA,0BAAA;CtB2uLD;AuDvtLC;EACE,WAAA;EACA,gBAAA;EACA,wBAAA;EACA,UAAA;EACA,yBAAA;CvDytLH;AwD9uLD;EACE,iBAAA;CxDgvLD;AwD5uLD;EACE,cAAA;EACA,iBAAA;EACA,gBAAA;EACA,OAAA;EACA,SAAA;EACA,UAAA;EACA,QAAA;EACA,cAAA;EACA,kCAAA;EAIA,WAAA;CxD2uLD;AwDxuLC;EnD+GA,sCAAA;EACI,kCAAA;EACC,iCAAA;EACG,8BAAA;EAkER,oDAAA;EAEK,0CAAA;EACG,oCAAA;CL2jLT;AwD9uLC;EnD2GA,mCAAA;EACI,+BAAA;EACC,8BAAA;EACG,2BAAA;CLsoLT;AwDlvLD;EACE,mBAAA;EACA,iBAAA;CxDovLD;AwDhvLD;EACE,mBAAA;EACA,YAAA;EACA,aAAA;CxDkvLD;AwD9uLD;EACE,mBAAA;EACA,0BAAA;EACA,0BAAA;EACA,qCAAA;EACA,mBAAA;EnDaA,iDAAA;EACQ,yCAAA;EmDZR,qCAAA;UAAA,6BAAA;EAEA,WAAA;CxDgvLD;AwD5uLD;EACE,gBAAA;EACA,OAAA;EACA,SAAA;EACA,UAAA;EACA,QAAA;EACA,cAAA;EACA,0BAAA;CxD8uLD;AwD5uLC;ElCrEA,WAAA;EAGA,yBAAA;CtBkzLD;AwD/uLC;ElCtEA,aAAA;EAGA,0BAAA;CtBszLD;AwD9uLD;EACE,cAAA;EACA,iCAAA;EACA,0BAAA;CxDgvLD;AwD7uLD;EACE,iBAAA;CxD+uLD;AwD3uLD;EACE,UAAA;EACA,wBAAA;CxD6uLD;AwDxuLD;EACE,mBAAA;EACA,cAAA;CxD0uLD;AwDtuLD;EACE,cAAA;EACA,kBAAA;EACA,8BAAA;CxDwuLD;AwD3uLD;EAQI,iBAAA;EACA,iBAAA;CxDsuLH;AwD/uLD;EAaI,kBAAA;CxDquLH;AwDlvLD;EAiBI,eAAA;CxDouLH;AwD/tLD;EACE,mBAAA;EACA,aAAA;EACA,YAAA;EACA,aAAA;EACA,iBAAA;CxDiuLD;AwD/sLD;EAZE;IACE,aAAA;IACA,kBAAA;GxD8tLD;EwD5tLD;InDvEA,kDAAA;IACQ,0CAAA;GLsyLP;EwD3tLD;IAAY,aAAA;GxD8tLX;CACF;AwDztLD;EAFE;IAAY,aAAA;GxD+tLX;CACF;AyD92LD;EACE,mBAAA;EACA,cAAA;EACA,eAAA;ECRA,4DAAA;EAEA,mBAAA;EACA,oBAAA;EACA,uBAAA;EACA,iBAAA;EACA,wBAAA;EACA,iBAAA;EACA,kBAAA;EACA,sBAAA;EACA,kBAAA;EACA,qBAAA;EACA,oBAAA;EACA,mBAAA;EACA,qBAAA;EACA,kBAAA;EDHA,gBAAA;EnCVA,WAAA;EAGA,yBAAA;CtBq4LD;AyD13LC;EnCdA,aAAA;EAGA,0BAAA;CtBy4LD;AyD73LC;EAAW,iBAAA;EAAmB,eAAA;CzDi4L/B;AyDh4LC;EAAW,iBAAA;EAAmB,eAAA;CzDo4L/B;AyDn4LC;EAAW,gBAAA;EAAmB,eAAA;CzDu4L/B;AyDt4LC;EAAW,kBAAA;EAAmB,eAAA;CzD04L/B;AyDt4LD;EACE,iBAAA;EACA,iBAAA;EACA,eAAA;EACA,mBAAA;EACA,0BAAA;EACA,mBAAA;CzDw4LD;AyDp4LD;EACE,mBAAA;EACA,SAAA;EACA,UAAA;EACA,0BAAA;EACA,oBAAA;CzDs4LD;AyDl4LC;EACE,UAAA;EACA,UAAA;EACA,kBAAA;EACA,wBAAA;EACA,0BAAA;CzDo4LH;AyDl4LC;EACE,UAAA;EACA,WAAA;EACA,oBAAA;EACA,wBAAA;EACA,0BAAA;CzDo4LH;AyDl4LC;EACE,UAAA;EACA,UAAA;EACA,oBAAA;EACA,wBAAA;EACA,0BAAA;CzDo4LH;AyDl4LC;EACE,SAAA;EACA,QAAA;EACA,iBAAA;EACA,4BAAA;EACA,4BAAA;CzDo4LH;AyDl4LC;EACE,SAAA;EACA,SAAA;EACA,iBAAA;EACA,4BAAA;EACA,2BAAA;CzDo4LH;AyDl4LC;EACE,OAAA;EACA,UAAA;EACA,kBAAA;EACA,wBAAA;EACA,6BAAA;CzDo4LH;AyDl4LC;EACE,OAAA;EACA,WAAA;EACA,iBAAA;EACA,wBAAA;EACA,6BAAA;CzDo4LH;AyDl4LC;EACE,OAAA;EACA,UAAA;EACA,iBAAA;EACA,wBAAA;EACA,6BAAA;CzDo4LH;A2Dj+LD;EACE,mBAAA;EACA,OAAA;EACA,QAAA;EACA,cAAA;EACA,cAAA;EACA,iBAAA;EACA,aAAA;EDXA,4DAAA;EAEA,mBAAA;EACA,oBAAA;EACA,uBAAA;EACA,iBAAA;EACA,wBAAA;EACA,iBAAA;EACA,kBAAA;EACA,sBAAA;EACA,kBAAA;EACA,qBAAA;EACA,oBAAA;EACA,mBAAA;EACA,qBAAA;EACA,kBAAA;ECAA,gBAAA;EAEA,0BAAA;EACA,qCAAA;UAAA,6BAAA;EACA,0BAAA;EACA,qCAAA;EACA,mBAAA;EtD8CA,kDAAA;EACQ,0CAAA;CLi8LT;A2D5+LC;EAAY,kBAAA;C3D++Lb;A2D9+LC;EAAY,kBAAA;C3Di/Lb;A2Dh/LC;EAAY,iBAAA;C3Dm/Lb;A2Dl/LC;EAAY,mBAAA;C3Dq/Lb;A2Dl/LD;EACE,UAAA;EACA,kBAAA;EACA,gBAAA;EACA,0BAAA;EACA,iCAAA;EACA,2BAAA;C3Do/LD;A2Dj/LD;EACE,kBAAA;C3Dm/LD;A2D3+LC;;EAEE,mBAAA;EACA,eAAA;EACA,SAAA;EACA,UAAA;EACA,0BAAA;EACA,oBAAA;C3D6+LH;A2D1+LD;EACE,mBAAA;C3D4+LD;A2D1+LD;EACE,mBAAA;EACA,YAAA;C3D4+LD;A2Dx+LC;EACE,UAAA;EACA,mBAAA;EACA,uBAAA;EACA,0BAAA;EACA,sCAAA;EACA,cAAA;C3D0+LH;A2Dz+LG;EACE,aAAA;EACA,YAAA;EACA,mBAAA;EACA,uBAAA;EACA,0BAAA;C3D2+LL;A2Dx+LC;EACE,SAAA;EACA,YAAA;EACA,kBAAA;EACA,qBAAA;EACA,4BAAA;EACA,wCAAA;C3D0+LH;A2Dz+LG;EACE,aAAA;EACA,UAAA;EACA,cAAA;EACA,qBAAA;EACA,4BAAA;C3D2+LL;A2Dx+LC;EACE,UAAA;EACA,mBAAA;EACA,oBAAA;EACA,6BAAA;EACA,yCAAA;EACA,WAAA;C3D0+LH;A2Dz+LG;EACE,aAAA;EACA,SAAA;EACA,mBAAA;EACA,oBAAA;EACA,6BAAA;C3D2+LL;A2Dv+LC;EACE,SAAA;EACA,aAAA;EACA,kBAAA;EACA,sBAAA;EACA,2BAAA;EACA,uCAAA;C3Dy+LH;A2Dx+LG;EACE,aAAA;EACA,WAAA;EACA,sBAAA;EACA,2BAAA;EACA,cAAA;C3D0+LL;A4DnmMD;EACE,mBAAA;C5DqmMD;A4DlmMD;EACE,mBAAA;EACA,iBAAA;EACA,YAAA;C5DomMD;A4DvmMD;EAMI,cAAA;EACA,mBAAA;EvD6KF,0CAAA;EACK,qCAAA;EACG,kCAAA;CLw7LT;A4D9mMD;;EAcM,eAAA;C5DomML;A4D1kMC;EAAA;IvDiKA,uDAAA;IAEK,6CAAA;IACG,uCAAA;IA7JR,oCAAA;IAEQ,4BAAA;IA+GR,4BAAA;IAEQ,oBAAA;GL69LP;E4DxmMG;;IvDmHJ,2CAAA;IACQ,mCAAA;IuDjHF,QAAA;G5D2mML;E4DzmMG;;IvD8GJ,4CAAA;IACQ,oCAAA;IuD5GF,QAAA;G5D4mML;E4D1mMG;;;IvDyGJ,wCAAA;IACQ,gCAAA;IuDtGF,QAAA;G5D6mML;CACF;A4DnpMD;;;EA6CI,eAAA;C5D2mMH;A4DxpMD;EAiDI,QAAA;C5D0mMH;A4D3pMD;;EAsDI,mBAAA;EACA,OAAA;EACA,YAAA;C5DymMH;A4DjqMD;EA4DI,WAAA;C5DwmMH;A4DpqMD;EA+DI,YAAA;C5DwmMH;A4DvqMD;;EAmEI,QAAA;C5DwmMH;A4D3qMD;EAuEI,YAAA;C5DumMH;A4D9qMD;EA0EI,WAAA;C5DumMH;A4D/lMD;EACE,mBAAA;EACA,OAAA;EACA,QAAA;EACA,UAAA;EACA,WAAA;EtC9FA,aAAA;EAGA,0BAAA;EsC6FA,gBAAA;EACA,eAAA;EACA,mBAAA;EACA,0CAAA;C5DkmMD;A4D7lMC;EdlGE,mGAAA;EACA,8FAAA;EACA,qHAAA;EAAA,+FAAA;EACA,4BAAA;EACA,uHAAA;C9CksMH;A4DjmMC;EACE,WAAA;EACA,SAAA;EdvGA,mGAAA;EACA,8FAAA;EACA,qHAAA;EAAA,+FAAA;EACA,4BAAA;EACA,uHAAA;C9C2sMH;A4DnmMC;;EAEE,WAAA;EACA,eAAA;EACA,sBAAA;EtCtHF,aAAA;EAGA,0BAAA;CtB0tMD;A4DpoMD;;;;EAsCI,mBAAA;EACA,SAAA;EACA,kBAAA;EACA,WAAA;EACA,sBAAA;C5DomMH;A4D9oMD;;EA8CI,UAAA;EACA,mBAAA;C5DomMH;A4DnpMD;;EAmDI,WAAA;EACA,oBAAA;C5DomMH;A4DxpMD;;EAwDI,YAAA;EACA,aAAA;EACA,eAAA;EACA,mBAAA;C5DomMH;A4D/lMG;EACE,iBAAA;C5DimML;A4D7lMG;EACE,iBAAA;C5D+lML;A4DrlMD;EACE,mBAAA;EACA,aAAA;EACA,UAAA;EACA,YAAA;EACA,WAAA;EACA,kBAAA;EACA,gBAAA;EACA,iBAAA;EACA,mBAAA;C5DulMD;A4DhmMD;EAYI,sBAAA;EACA,YAAA;EACA,aAAA;EACA,YAAA;EACA,oBAAA;EACA,0BAAA;EACA,oBAAA;EACA,gBAAA;EAWA,0BAAA;EACA,mCAAA;C5D6kMH;A4D5mMD;EAkCI,UAAA;EACA,YAAA;EACA,aAAA;EACA,0BAAA;C5D6kMH;A4DtkMD;EACE,mBAAA;EACA,UAAA;EACA,WAAA;EACA,aAAA;EACA,YAAA;EACA,kBAAA;EACA,qBAAA;EACA,eAAA;EACA,mBAAA;EACA,0CAAA;C5DwkMD;A4DvkMC;EACE,kBAAA;C5DykMH;A4DhiMD;EAhCE;;;;IAKI,YAAA;IACA,aAAA;IACA,kBAAA;IACA,gBAAA;G5DkkMH;E4D1kMD;;IAYI,mBAAA;G5DkkMH;E4D9kMD;;IAgBI,oBAAA;G5DkkMH;E4D7jMD;IACE,UAAA;IACA,WAAA;IACA,qBAAA;G5D+jMD;E4D3jMD;IACE,aAAA;G5D6jMD;CACF;A6D3zMC;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;EAEE,aAAA;EACA,eAAA;C7Dy1MH;A6Dv1MC;;;;;;;;;;;;;;;EACE,YAAA;C7Du2MH;AiC/2MD;E6BRE,eAAA;EACA,kBAAA;EACA,mBAAA;C9D03MD;AiCj3MD;EACE,wBAAA;CjCm3MD;AiCj3MD;EACE,uBAAA;CjCm3MD;AiC32MD;EACE,yBAAA;CjC62MD;AiC32MD;EACE,0BAAA;CjC62MD;AiC32MD;EACE,mBAAA;CjC62MD;AiC32MD;E8BzBE,YAAA;EACA,mBAAA;EACA,kBAAA;EACA,8BAAA;EACA,UAAA;C/Du4MD;AiCz2MD;EACE,yBAAA;CjC22MD;AiCp2MD;EACE,gBAAA;CjCs2MD;AgEv4MD;EACE,oBAAA;ChEy4MD;AgEn4MD;;;;ECdE,yBAAA;CjEu5MD;AgEl4MD;;;;;;;;;;;;EAYE,yBAAA;ChEo4MD;AgE73MD;EAAA;IChDE,0BAAA;GjEi7MC;EiEh7MD;IAAU,0BAAA;GjEm7MT;EiEl7MD;IAAU,8BAAA;GjEq7MT;EiEp7MD;;IACU,+BAAA;GjEu7MT;CACF;AgEv4MD;EAAA;IAFI,0BAAA;GhE64MD;CACF;AgEv4MD;EAAA;IAFI,2BAAA;GhE64MD;CACF;AgEv4MD;EAAA;IAFI,iCAAA;GhE64MD;CACF;AgEt4MD;EAAA;ICrEE,0BAAA;GjE+8MC;EiE98MD;IAAU,0BAAA;GjEi9MT;EiEh9MD;IAAU,8BAAA;GjEm9MT;EiEl9MD;;IACU,+BAAA;GjEq9MT;CACF;AgEh5MD;EAAA;IAFI,0BAAA;GhEs5MD;CACF;AgEh5MD;EAAA;IAFI,2BAAA;GhEs5MD;CACF;AgEh5MD;EAAA;IAFI,iCAAA;GhEs5MD;CACF;AgE/4MD;EAAA;IC1FE,0BAAA;GjE6+MC;EiE5+MD;IAAU,0BAAA;GjE++MT;EiE9+MD;IAAU,8BAAA;GjEi/MT;EiEh/MD;;IACU,+BAAA;GjEm/MT;CACF;AgEz5MD;EAAA;IAFI,0BAAA;GhE+5MD;CACF;AgEz5MD;EAAA;IAFI,2BAAA;GhE+5MD;CACF;AgEz5MD;EAAA;IAFI,iCAAA;GhE+5MD;CACF;AgEx5MD;EAAA;IC/GE,0BAAA;GjE2gNC;EiE1gND;IAAU,0BAAA;GjE6gNT;EiE5gND;IAAU,8BAAA;GjE+gNT;EiE9gND;;IACU,+BAAA;GjEihNT;CACF;AgEl6MD;EAAA;IAFI,0BAAA;GhEw6MD;CACF;AgEl6MD;EAAA;IAFI,2BAAA;GhEw6MD;CACF;AgEl6MD;EAAA;IAFI,iCAAA;GhEw6MD;CACF;AgEj6MD;EAAA;IC5HE,yBAAA;GjEiiNC;CACF;AgEj6MD;EAAA;ICjIE,yBAAA;GjEsiNC;CACF;AgEj6MD;EAAA;ICtIE,yBAAA;GjE2iNC;CACF;AgEj6MD;EAAA;IC3IE,yBAAA;GjEgjNC;CACF;AgE95MD;ECnJE,yBAAA;CjEojND;AgE35MD;EAAA;ICjKE,0BAAA;GjEgkNC;EiE/jND;IAAU,0BAAA;GjEkkNT;EiEjkND;IAAU,8BAAA;GjEokNT;EiEnkND;;IACU,+BAAA;GjEskNT;CACF;AgEz6MD;EACE,yBAAA;ChE26MD;AgEt6MD;EAAA;IAFI,0BAAA;GhE46MD;CACF;AgE16MD;EACE,yBAAA;ChE46MD;AgEv6MD;EAAA;IAFI,2BAAA;GhE66MD;CACF;AgE36MD;EACE,yBAAA;ChE66MD;AgEx6MD;EAAA;IAFI,iCAAA;GhE86MD;CACF;AgEv6MD;EAAA;ICpLE,yBAAA;GjE+lNC;CACF","file":"bootstrap.css","sourcesContent":["/*!\n * Bootstrap v3.3.5 (http://getbootstrap.com)\n * Copyright 2011-2015 Twitter, Inc.\n * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)\n */\n/*! normalize.css v3.0.3 | MIT License | github.com/necolas/normalize.css */\nhtml {\n font-family: sans-serif;\n -ms-text-size-adjust: 100%;\n -webkit-text-size-adjust: 100%;\n}\nbody {\n margin: 0;\n}\narticle,\naside,\ndetails,\nfigcaption,\nfigure,\nfooter,\nheader,\nhgroup,\nmain,\nmenu,\nnav,\nsection,\nsummary {\n display: block;\n}\naudio,\ncanvas,\nprogress,\nvideo {\n display: inline-block;\n vertical-align: baseline;\n}\naudio:not([controls]) {\n display: none;\n height: 0;\n}\n[hidden],\ntemplate {\n display: none;\n}\na {\n background-color: transparent;\n}\na:active,\na:hover {\n outline: 0;\n}\nabbr[title] {\n border-bottom: 1px dotted;\n}\nb,\nstrong {\n font-weight: bold;\n}\ndfn {\n font-style: italic;\n}\nh1 {\n font-size: 2em;\n margin: 0.67em 0;\n}\nmark {\n background: #ff0;\n color: #000;\n}\nsmall {\n font-size: 80%;\n}\nsub,\nsup {\n font-size: 75%;\n line-height: 0;\n position: relative;\n vertical-align: baseline;\n}\nsup {\n top: -0.5em;\n}\nsub {\n bottom: -0.25em;\n}\nimg {\n border: 0;\n}\nsvg:not(:root) {\n overflow: hidden;\n}\nfigure {\n margin: 1em 40px;\n}\nhr {\n box-sizing: content-box;\n height: 0;\n}\npre {\n overflow: auto;\n}\ncode,\nkbd,\npre,\nsamp {\n font-family: monospace, monospace;\n font-size: 1em;\n}\nbutton,\ninput,\noptgroup,\nselect,\ntextarea {\n color: inherit;\n font: inherit;\n margin: 0;\n}\nbutton {\n overflow: visible;\n}\nbutton,\nselect {\n text-transform: none;\n}\nbutton,\nhtml input[type=\"button\"],\ninput[type=\"reset\"],\ninput[type=\"submit\"] {\n -webkit-appearance: button;\n cursor: pointer;\n}\nbutton[disabled],\nhtml input[disabled] {\n cursor: default;\n}\nbutton::-moz-focus-inner,\ninput::-moz-focus-inner {\n border: 0;\n padding: 0;\n}\ninput {\n line-height: normal;\n}\ninput[type=\"checkbox\"],\ninput[type=\"radio\"] {\n box-sizing: border-box;\n padding: 0;\n}\ninput[type=\"number\"]::-webkit-inner-spin-button,\ninput[type=\"number\"]::-webkit-outer-spin-button {\n height: auto;\n}\ninput[type=\"search\"] {\n -webkit-appearance: textfield;\n box-sizing: content-box;\n}\ninput[type=\"search\"]::-webkit-search-cancel-button,\ninput[type=\"search\"]::-webkit-search-decoration {\n -webkit-appearance: none;\n}\nfieldset {\n border: 1px solid #c0c0c0;\n margin: 0 2px;\n padding: 0.35em 0.625em 0.75em;\n}\nlegend {\n border: 0;\n padding: 0;\n}\ntextarea {\n overflow: auto;\n}\noptgroup {\n font-weight: bold;\n}\ntable {\n border-collapse: collapse;\n border-spacing: 0;\n}\ntd,\nth {\n padding: 0;\n}\n/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */\n@media print {\n *,\n *:before,\n *:after {\n background: transparent !important;\n color: #000 !important;\n box-shadow: none !important;\n text-shadow: none !important;\n }\n a,\n a:visited {\n text-decoration: underline;\n }\n a[href]:after {\n content: \" (\" attr(href) \")\";\n }\n abbr[title]:after {\n content: \" (\" attr(title) \")\";\n }\n a[href^=\"#\"]:after,\n a[href^=\"javascript:\"]:after {\n content: \"\";\n }\n pre,\n blockquote {\n border: 1px solid #999;\n page-break-inside: avoid;\n }\n thead {\n display: table-header-group;\n }\n tr,\n img {\n page-break-inside: avoid;\n }\n img {\n max-width: 100% !important;\n }\n p,\n h2,\n h3 {\n orphans: 3;\n widows: 3;\n }\n h2,\n h3 {\n page-break-after: avoid;\n }\n .navbar {\n display: none;\n }\n .btn > .caret,\n .dropup > .btn > .caret {\n border-top-color: #000 !important;\n }\n .label {\n border: 1px solid #000;\n }\n .table {\n border-collapse: collapse !important;\n }\n .table td,\n .table th {\n background-color: #fff !important;\n }\n .table-bordered th,\n .table-bordered td {\n border: 1px solid #ddd !important;\n }\n}\n@font-face {\n font-family: 'Glyphicons Halflings';\n src: url('../fonts/glyphicons-halflings-regular.eot');\n src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff2') format('woff2'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg');\n}\n.glyphicon {\n position: relative;\n top: 1px;\n display: inline-block;\n font-family: 'Glyphicons Halflings';\n font-style: normal;\n font-weight: normal;\n line-height: 1;\n -webkit-font-smoothing: antialiased;\n -moz-osx-font-smoothing: grayscale;\n}\n.glyphicon-asterisk:before {\n content: \"\\2a\";\n}\n.glyphicon-plus:before {\n content: \"\\2b\";\n}\n.glyphicon-euro:before,\n.glyphicon-eur:before {\n content: \"\\20ac\";\n}\n.glyphicon-minus:before {\n content: \"\\2212\";\n}\n.glyphicon-cloud:before {\n content: \"\\2601\";\n}\n.glyphicon-envelope:before {\n content: \"\\2709\";\n}\n.glyphicon-pencil:before {\n content: \"\\270f\";\n}\n.glyphicon-glass:before {\n content: \"\\e001\";\n}\n.glyphicon-music:before {\n content: \"\\e002\";\n}\n.glyphicon-search:before {\n content: \"\\e003\";\n}\n.glyphicon-heart:before {\n content: \"\\e005\";\n}\n.glyphicon-star:before {\n content: \"\\e006\";\n}\n.glyphicon-star-empty:before {\n content: \"\\e007\";\n}\n.glyphicon-user:before {\n content: \"\\e008\";\n}\n.glyphicon-film:before {\n content: \"\\e009\";\n}\n.glyphicon-th-large:before {\n content: \"\\e010\";\n}\n.glyphicon-th:before {\n content: \"\\e011\";\n}\n.glyphicon-th-list:before {\n content: \"\\e012\";\n}\n.glyphicon-ok:before {\n content: \"\\e013\";\n}\n.glyphicon-remove:before {\n content: \"\\e014\";\n}\n.glyphicon-zoom-in:before {\n content: \"\\e015\";\n}\n.glyphicon-zoom-out:before {\n content: \"\\e016\";\n}\n.glyphicon-off:before {\n content: \"\\e017\";\n}\n.glyphicon-signal:before {\n content: \"\\e018\";\n}\n.glyphicon-cog:before {\n content: \"\\e019\";\n}\n.glyphicon-trash:before {\n content: \"\\e020\";\n}\n.glyphicon-home:before {\n content: \"\\e021\";\n}\n.glyphicon-file:before {\n content: \"\\e022\";\n}\n.glyphicon-time:before {\n content: \"\\e023\";\n}\n.glyphicon-road:before {\n content: \"\\e024\";\n}\n.glyphicon-download-alt:before {\n content: \"\\e025\";\n}\n.glyphicon-download:before {\n content: \"\\e026\";\n}\n.glyphicon-upload:before {\n content: \"\\e027\";\n}\n.glyphicon-inbox:before {\n content: \"\\e028\";\n}\n.glyphicon-play-circle:before {\n content: \"\\e029\";\n}\n.glyphicon-repeat:before {\n content: \"\\e030\";\n}\n.glyphicon-refresh:before {\n content: \"\\e031\";\n}\n.glyphicon-list-alt:before {\n content: \"\\e032\";\n}\n.glyphicon-lock:before {\n content: \"\\e033\";\n}\n.glyphicon-flag:before {\n content: \"\\e034\";\n}\n.glyphicon-headphones:before {\n content: \"\\e035\";\n}\n.glyphicon-volume-off:before {\n content: \"\\e036\";\n}\n.glyphicon-volume-down:before {\n content: \"\\e037\";\n}\n.glyphicon-volume-up:before {\n content: \"\\e038\";\n}\n.glyphicon-qrcode:before {\n content: \"\\e039\";\n}\n.glyphicon-barcode:before {\n content: \"\\e040\";\n}\n.glyphicon-tag:before {\n content: \"\\e041\";\n}\n.glyphicon-tags:before {\n content: \"\\e042\";\n}\n.glyphicon-book:before {\n content: \"\\e043\";\n}\n.glyphicon-bookmark:before {\n content: \"\\e044\";\n}\n.glyphicon-print:before {\n content: \"\\e045\";\n}\n.glyphicon-camera:before {\n content: \"\\e046\";\n}\n.glyphicon-font:before {\n content: \"\\e047\";\n}\n.glyphicon-bold:before {\n content: \"\\e048\";\n}\n.glyphicon-italic:before {\n content: \"\\e049\";\n}\n.glyphicon-text-height:before {\n content: \"\\e050\";\n}\n.glyphicon-text-width:before {\n content: \"\\e051\";\n}\n.glyphicon-align-left:before {\n content: \"\\e052\";\n}\n.glyphicon-align-center:before {\n content: \"\\e053\";\n}\n.glyphicon-align-right:before {\n content: \"\\e054\";\n}\n.glyphicon-align-justify:before {\n content: \"\\e055\";\n}\n.glyphicon-list:before {\n content: \"\\e056\";\n}\n.glyphicon-indent-left:before {\n content: \"\\e057\";\n}\n.glyphicon-indent-right:before {\n content: \"\\e058\";\n}\n.glyphicon-facetime-video:before {\n content: \"\\e059\";\n}\n.glyphicon-picture:before {\n content: \"\\e060\";\n}\n.glyphicon-map-marker:before {\n content: \"\\e062\";\n}\n.glyphicon-adjust:before {\n content: \"\\e063\";\n}\n.glyphicon-tint:before {\n content: \"\\e064\";\n}\n.glyphicon-edit:before {\n content: \"\\e065\";\n}\n.glyphicon-share:before {\n content: \"\\e066\";\n}\n.glyphicon-check:before {\n content: \"\\e067\";\n}\n.glyphicon-move:before {\n content: \"\\e068\";\n}\n.glyphicon-step-backward:before {\n content: \"\\e069\";\n}\n.glyphicon-fast-backward:before {\n content: \"\\e070\";\n}\n.glyphicon-backward:before {\n content: \"\\e071\";\n}\n.glyphicon-play:before {\n content: \"\\e072\";\n}\n.glyphicon-pause:before {\n content: \"\\e073\";\n}\n.glyphicon-stop:before {\n content: \"\\e074\";\n}\n.glyphicon-forward:before {\n content: \"\\e075\";\n}\n.glyphicon-fast-forward:before {\n content: \"\\e076\";\n}\n.glyphicon-step-forward:before {\n content: \"\\e077\";\n}\n.glyphicon-eject:before {\n content: \"\\e078\";\n}\n.glyphicon-chevron-left:before {\n content: \"\\e079\";\n}\n.glyphicon-chevron-right:before {\n content: \"\\e080\";\n}\n.glyphicon-plus-sign:before {\n content: \"\\e081\";\n}\n.glyphicon-minus-sign:before {\n content: \"\\e082\";\n}\n.glyphicon-remove-sign:before {\n content: \"\\e083\";\n}\n.glyphicon-ok-sign:before {\n content: \"\\e084\";\n}\n.glyphicon-question-sign:before {\n content: \"\\e085\";\n}\n.glyphicon-info-sign:before {\n content: \"\\e086\";\n}\n.glyphicon-screenshot:before {\n content: \"\\e087\";\n}\n.glyphicon-remove-circle:before {\n content: \"\\e088\";\n}\n.glyphicon-ok-circle:before {\n content: \"\\e089\";\n}\n.glyphicon-ban-circle:before {\n content: \"\\e090\";\n}\n.glyphicon-arrow-left:before {\n content: \"\\e091\";\n}\n.glyphicon-arrow-right:before {\n content: \"\\e092\";\n}\n.glyphicon-arrow-up:before {\n content: \"\\e093\";\n}\n.glyphicon-arrow-down:before {\n content: \"\\e094\";\n}\n.glyphicon-share-alt:before {\n content: \"\\e095\";\n}\n.glyphicon-resize-full:before {\n content: \"\\e096\";\n}\n.glyphicon-resize-small:before {\n content: \"\\e097\";\n}\n.glyphicon-exclamation-sign:before {\n content: \"\\e101\";\n}\n.glyphicon-gift:before {\n content: \"\\e102\";\n}\n.glyphicon-leaf:before {\n content: \"\\e103\";\n}\n.glyphicon-fire:before {\n content: \"\\e104\";\n}\n.glyphicon-eye-open:before {\n content: \"\\e105\";\n}\n.glyphicon-eye-close:before {\n content: \"\\e106\";\n}\n.glyphicon-warning-sign:before {\n content: \"\\e107\";\n}\n.glyphicon-plane:before {\n content: \"\\e108\";\n}\n.glyphicon-calendar:before {\n content: \"\\e109\";\n}\n.glyphicon-random:before {\n content: \"\\e110\";\n}\n.glyphicon-comment:before {\n content: \"\\e111\";\n}\n.glyphicon-magnet:before {\n content: \"\\e112\";\n}\n.glyphicon-chevron-up:before {\n content: \"\\e113\";\n}\n.glyphicon-chevron-down:before {\n content: \"\\e114\";\n}\n.glyphicon-retweet:before {\n content: \"\\e115\";\n}\n.glyphicon-shopping-cart:before {\n content: \"\\e116\";\n}\n.glyphicon-folder-close:before {\n content: \"\\e117\";\n}\n.glyphicon-folder-open:before {\n content: \"\\e118\";\n}\n.glyphicon-resize-vertical:before {\n content: \"\\e119\";\n}\n.glyphicon-resize-horizontal:before {\n content: \"\\e120\";\n}\n.glyphicon-hdd:before {\n content: \"\\e121\";\n}\n.glyphicon-bullhorn:before {\n content: \"\\e122\";\n}\n.glyphicon-bell:before {\n content: \"\\e123\";\n}\n.glyphicon-certificate:before {\n content: \"\\e124\";\n}\n.glyphicon-thumbs-up:before {\n content: \"\\e125\";\n}\n.glyphicon-thumbs-down:before {\n content: \"\\e126\";\n}\n.glyphicon-hand-right:before {\n content: \"\\e127\";\n}\n.glyphicon-hand-left:before {\n content: \"\\e128\";\n}\n.glyphicon-hand-up:before {\n content: \"\\e129\";\n}\n.glyphicon-hand-down:before {\n content: \"\\e130\";\n}\n.glyphicon-circle-arrow-right:before {\n content: \"\\e131\";\n}\n.glyphicon-circle-arrow-left:before {\n content: \"\\e132\";\n}\n.glyphicon-circle-arrow-up:before {\n content: \"\\e133\";\n}\n.glyphicon-circle-arrow-down:before {\n content: \"\\e134\";\n}\n.glyphicon-globe:before {\n content: \"\\e135\";\n}\n.glyphicon-wrench:before {\n content: \"\\e136\";\n}\n.glyphicon-tasks:before {\n content: \"\\e137\";\n}\n.glyphicon-filter:before {\n content: \"\\e138\";\n}\n.glyphicon-briefcase:before {\n content: \"\\e139\";\n}\n.glyphicon-fullscreen:before {\n content: \"\\e140\";\n}\n.glyphicon-dashboard:before {\n content: \"\\e141\";\n}\n.glyphicon-paperclip:before {\n content: \"\\e142\";\n}\n.glyphicon-heart-empty:before {\n content: \"\\e143\";\n}\n.glyphicon-link:before {\n content: \"\\e144\";\n}\n.glyphicon-phone:before {\n content: \"\\e145\";\n}\n.glyphicon-pushpin:before {\n content: \"\\e146\";\n}\n.glyphicon-usd:before {\n content: \"\\e148\";\n}\n.glyphicon-gbp:before {\n content: \"\\e149\";\n}\n.glyphicon-sort:before {\n content: \"\\e150\";\n}\n.glyphicon-sort-by-alphabet:before {\n content: \"\\e151\";\n}\n.glyphicon-sort-by-alphabet-alt:before {\n content: \"\\e152\";\n}\n.glyphicon-sort-by-order:before {\n content: \"\\e153\";\n}\n.glyphicon-sort-by-order-alt:before {\n content: \"\\e154\";\n}\n.glyphicon-sort-by-attributes:before {\n content: \"\\e155\";\n}\n.glyphicon-sort-by-attributes-alt:before {\n content: \"\\e156\";\n}\n.glyphicon-unchecked:before {\n content: \"\\e157\";\n}\n.glyphicon-expand:before {\n content: \"\\e158\";\n}\n.glyphicon-collapse-down:before {\n content: \"\\e159\";\n}\n.glyphicon-collapse-up:before {\n content: \"\\e160\";\n}\n.glyphicon-log-in:before {\n content: \"\\e161\";\n}\n.glyphicon-flash:before {\n content: \"\\e162\";\n}\n.glyphicon-log-out:before {\n content: \"\\e163\";\n}\n.glyphicon-new-window:before {\n content: \"\\e164\";\n}\n.glyphicon-record:before {\n content: \"\\e165\";\n}\n.glyphicon-save:before {\n content: \"\\e166\";\n}\n.glyphicon-open:before {\n content: \"\\e167\";\n}\n.glyphicon-saved:before {\n content: \"\\e168\";\n}\n.glyphicon-import:before {\n content: \"\\e169\";\n}\n.glyphicon-export:before {\n content: \"\\e170\";\n}\n.glyphicon-send:before {\n content: \"\\e171\";\n}\n.glyphicon-floppy-disk:before {\n content: \"\\e172\";\n}\n.glyphicon-floppy-saved:before {\n content: \"\\e173\";\n}\n.glyphicon-floppy-remove:before {\n content: \"\\e174\";\n}\n.glyphicon-floppy-save:before {\n content: \"\\e175\";\n}\n.glyphicon-floppy-open:before {\n content: \"\\e176\";\n}\n.glyphicon-credit-card:before {\n content: \"\\e177\";\n}\n.glyphicon-transfer:before {\n content: \"\\e178\";\n}\n.glyphicon-cutlery:before {\n content: \"\\e179\";\n}\n.glyphicon-header:before {\n content: \"\\e180\";\n}\n.glyphicon-compressed:before {\n content: \"\\e181\";\n}\n.glyphicon-earphone:before {\n content: \"\\e182\";\n}\n.glyphicon-phone-alt:before {\n content: \"\\e183\";\n}\n.glyphicon-tower:before {\n content: \"\\e184\";\n}\n.glyphicon-stats:before {\n content: \"\\e185\";\n}\n.glyphicon-sd-video:before {\n content: \"\\e186\";\n}\n.glyphicon-hd-video:before {\n content: \"\\e187\";\n}\n.glyphicon-subtitles:before {\n content: \"\\e188\";\n}\n.glyphicon-sound-stereo:before {\n content: \"\\e189\";\n}\n.glyphicon-sound-dolby:before {\n content: \"\\e190\";\n}\n.glyphicon-sound-5-1:before {\n content: \"\\e191\";\n}\n.glyphicon-sound-6-1:before {\n content: \"\\e192\";\n}\n.glyphicon-sound-7-1:before {\n content: \"\\e193\";\n}\n.glyphicon-copyright-mark:before {\n content: \"\\e194\";\n}\n.glyphicon-registration-mark:before {\n content: \"\\e195\";\n}\n.glyphicon-cloud-download:before {\n content: \"\\e197\";\n}\n.glyphicon-cloud-upload:before {\n content: \"\\e198\";\n}\n.glyphicon-tree-conifer:before {\n content: \"\\e199\";\n}\n.glyphicon-tree-deciduous:before {\n content: \"\\e200\";\n}\n.glyphicon-cd:before {\n content: \"\\e201\";\n}\n.glyphicon-save-file:before {\n content: \"\\e202\";\n}\n.glyphicon-open-file:before {\n content: \"\\e203\";\n}\n.glyphicon-level-up:before {\n content: \"\\e204\";\n}\n.glyphicon-copy:before {\n content: \"\\e205\";\n}\n.glyphicon-paste:before {\n content: \"\\e206\";\n}\n.glyphicon-alert:before {\n content: \"\\e209\";\n}\n.glyphicon-equalizer:before {\n content: \"\\e210\";\n}\n.glyphicon-king:before {\n content: \"\\e211\";\n}\n.glyphicon-queen:before {\n content: \"\\e212\";\n}\n.glyphicon-pawn:before {\n content: \"\\e213\";\n}\n.glyphicon-bishop:before {\n content: \"\\e214\";\n}\n.glyphicon-knight:before {\n content: \"\\e215\";\n}\n.glyphicon-baby-formula:before {\n content: \"\\e216\";\n}\n.glyphicon-tent:before {\n content: \"\\26fa\";\n}\n.glyphicon-blackboard:before {\n content: \"\\e218\";\n}\n.glyphicon-bed:before {\n content: \"\\e219\";\n}\n.glyphicon-apple:before {\n content: \"\\f8ff\";\n}\n.glyphicon-erase:before {\n content: \"\\e221\";\n}\n.glyphicon-hourglass:before {\n content: \"\\231b\";\n}\n.glyphicon-lamp:before {\n content: \"\\e223\";\n}\n.glyphicon-duplicate:before {\n content: \"\\e224\";\n}\n.glyphicon-piggy-bank:before {\n content: \"\\e225\";\n}\n.glyphicon-scissors:before {\n content: \"\\e226\";\n}\n.glyphicon-bitcoin:before {\n content: \"\\e227\";\n}\n.glyphicon-btc:before {\n content: \"\\e227\";\n}\n.glyphicon-xbt:before {\n content: \"\\e227\";\n}\n.glyphicon-yen:before {\n content: \"\\00a5\";\n}\n.glyphicon-jpy:before {\n content: \"\\00a5\";\n}\n.glyphicon-ruble:before {\n content: \"\\20bd\";\n}\n.glyphicon-rub:before {\n content: \"\\20bd\";\n}\n.glyphicon-scale:before {\n content: \"\\e230\";\n}\n.glyphicon-ice-lolly:before {\n content: \"\\e231\";\n}\n.glyphicon-ice-lolly-tasted:before {\n content: \"\\e232\";\n}\n.glyphicon-education:before {\n content: \"\\e233\";\n}\n.glyphicon-option-horizontal:before {\n content: \"\\e234\";\n}\n.glyphicon-option-vertical:before {\n content: \"\\e235\";\n}\n.glyphicon-menu-hamburger:before {\n content: \"\\e236\";\n}\n.glyphicon-modal-window:before {\n content: \"\\e237\";\n}\n.glyphicon-oil:before {\n content: \"\\e238\";\n}\n.glyphicon-grain:before {\n content: \"\\e239\";\n}\n.glyphicon-sunglasses:before {\n content: \"\\e240\";\n}\n.glyphicon-text-size:before {\n content: \"\\e241\";\n}\n.glyphicon-text-color:before {\n content: \"\\e242\";\n}\n.glyphicon-text-background:before {\n content: \"\\e243\";\n}\n.glyphicon-object-align-top:before {\n content: \"\\e244\";\n}\n.glyphicon-object-align-bottom:before {\n content: \"\\e245\";\n}\n.glyphicon-object-align-horizontal:before {\n content: \"\\e246\";\n}\n.glyphicon-object-align-left:before {\n content: \"\\e247\";\n}\n.glyphicon-object-align-vertical:before {\n content: \"\\e248\";\n}\n.glyphicon-object-align-right:before {\n content: \"\\e249\";\n}\n.glyphicon-triangle-right:before {\n content: \"\\e250\";\n}\n.glyphicon-triangle-left:before {\n content: \"\\e251\";\n}\n.glyphicon-triangle-bottom:before {\n content: \"\\e252\";\n}\n.glyphicon-triangle-top:before {\n content: \"\\e253\";\n}\n.glyphicon-console:before {\n content: \"\\e254\";\n}\n.glyphicon-superscript:before {\n content: \"\\e255\";\n}\n.glyphicon-subscript:before {\n content: \"\\e256\";\n}\n.glyphicon-menu-left:before {\n content: \"\\e257\";\n}\n.glyphicon-menu-right:before {\n content: \"\\e258\";\n}\n.glyphicon-menu-down:before {\n content: \"\\e259\";\n}\n.glyphicon-menu-up:before {\n content: \"\\e260\";\n}\n* {\n -webkit-box-sizing: border-box;\n -moz-box-sizing: border-box;\n box-sizing: border-box;\n}\n*:before,\n*:after {\n -webkit-box-sizing: border-box;\n -moz-box-sizing: border-box;\n box-sizing: border-box;\n}\nhtml {\n font-size: 10px;\n -webkit-tap-highlight-color: rgba(0, 0, 0, 0);\n}\nbody {\n font-family: \"Helvetica Neue\", Helvetica, Arial, sans-serif;\n font-size: 14px;\n line-height: 1.42857143;\n color: #333333;\n background-color: #ffffff;\n}\ninput,\nbutton,\nselect,\ntextarea {\n font-family: inherit;\n font-size: inherit;\n line-height: inherit;\n}\na {\n color: #337ab7;\n text-decoration: none;\n}\na:hover,\na:focus {\n color: #23527c;\n text-decoration: underline;\n}\na:focus {\n outline: thin dotted;\n outline: 5px auto -webkit-focus-ring-color;\n outline-offset: -2px;\n}\nfigure {\n margin: 0;\n}\nimg {\n vertical-align: middle;\n}\n.img-responsive,\n.thumbnail > img,\n.thumbnail a > img,\n.carousel-inner > .item > img,\n.carousel-inner > .item > a > img {\n display: block;\n max-width: 100%;\n height: auto;\n}\n.img-rounded {\n border-radius: 6px;\n}\n.img-thumbnail {\n padding: 4px;\n line-height: 1.42857143;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n border-radius: 4px;\n -webkit-transition: all 0.2s ease-in-out;\n -o-transition: all 0.2s ease-in-out;\n transition: all 0.2s ease-in-out;\n display: inline-block;\n max-width: 100%;\n height: auto;\n}\n.img-circle {\n border-radius: 50%;\n}\nhr {\n margin-top: 20px;\n margin-bottom: 20px;\n border: 0;\n border-top: 1px solid #eeeeee;\n}\n.sr-only {\n position: absolute;\n width: 1px;\n height: 1px;\n margin: -1px;\n padding: 0;\n overflow: hidden;\n clip: rect(0, 0, 0, 0);\n border: 0;\n}\n.sr-only-focusable:active,\n.sr-only-focusable:focus {\n position: static;\n width: auto;\n height: auto;\n margin: 0;\n overflow: visible;\n clip: auto;\n}\n[role=\"button\"] {\n cursor: pointer;\n}\nh1,\nh2,\nh3,\nh4,\nh5,\nh6,\n.h1,\n.h2,\n.h3,\n.h4,\n.h5,\n.h6 {\n font-family: inherit;\n font-weight: 500;\n line-height: 1.1;\n color: inherit;\n}\nh1 small,\nh2 small,\nh3 small,\nh4 small,\nh5 small,\nh6 small,\n.h1 small,\n.h2 small,\n.h3 small,\n.h4 small,\n.h5 small,\n.h6 small,\nh1 .small,\nh2 .small,\nh3 .small,\nh4 .small,\nh5 .small,\nh6 .small,\n.h1 .small,\n.h2 .small,\n.h3 .small,\n.h4 .small,\n.h5 .small,\n.h6 .small {\n font-weight: normal;\n line-height: 1;\n color: #777777;\n}\nh1,\n.h1,\nh2,\n.h2,\nh3,\n.h3 {\n margin-top: 20px;\n margin-bottom: 10px;\n}\nh1 small,\n.h1 small,\nh2 small,\n.h2 small,\nh3 small,\n.h3 small,\nh1 .small,\n.h1 .small,\nh2 .small,\n.h2 .small,\nh3 .small,\n.h3 .small {\n font-size: 65%;\n}\nh4,\n.h4,\nh5,\n.h5,\nh6,\n.h6 {\n margin-top: 10px;\n margin-bottom: 10px;\n}\nh4 small,\n.h4 small,\nh5 small,\n.h5 small,\nh6 small,\n.h6 small,\nh4 .small,\n.h4 .small,\nh5 .small,\n.h5 .small,\nh6 .small,\n.h6 .small {\n font-size: 75%;\n}\nh1,\n.h1 {\n font-size: 36px;\n}\nh2,\n.h2 {\n font-size: 30px;\n}\nh3,\n.h3 {\n font-size: 24px;\n}\nh4,\n.h4 {\n font-size: 18px;\n}\nh5,\n.h5 {\n font-size: 14px;\n}\nh6,\n.h6 {\n font-size: 12px;\n}\np {\n margin: 0 0 10px;\n}\n.lead {\n margin-bottom: 20px;\n font-size: 16px;\n font-weight: 300;\n line-height: 1.4;\n}\n@media (min-width: 768px) {\n .lead {\n font-size: 21px;\n }\n}\nsmall,\n.small {\n font-size: 85%;\n}\nmark,\n.mark {\n background-color: #fcf8e3;\n padding: .2em;\n}\n.text-left {\n text-align: left;\n}\n.text-right {\n text-align: right;\n}\n.text-center {\n text-align: center;\n}\n.text-justify {\n text-align: justify;\n}\n.text-nowrap {\n white-space: nowrap;\n}\n.text-lowercase {\n text-transform: lowercase;\n}\n.text-uppercase {\n text-transform: uppercase;\n}\n.text-capitalize {\n text-transform: capitalize;\n}\n.text-muted {\n color: #777777;\n}\n.text-primary {\n color: #337ab7;\n}\na.text-primary:hover,\na.text-primary:focus {\n color: #286090;\n}\n.text-success {\n color: #3c763d;\n}\na.text-success:hover,\na.text-success:focus {\n color: #2b542c;\n}\n.text-info {\n color: #31708f;\n}\na.text-info:hover,\na.text-info:focus {\n color: #245269;\n}\n.text-warning {\n color: #8a6d3b;\n}\na.text-warning:hover,\na.text-warning:focus {\n color: #66512c;\n}\n.text-danger {\n color: #a94442;\n}\na.text-danger:hover,\na.text-danger:focus {\n color: #843534;\n}\n.bg-primary {\n color: #fff;\n background-color: #337ab7;\n}\na.bg-primary:hover,\na.bg-primary:focus {\n background-color: #286090;\n}\n.bg-success {\n background-color: #dff0d8;\n}\na.bg-success:hover,\na.bg-success:focus {\n background-color: #c1e2b3;\n}\n.bg-info {\n background-color: #d9edf7;\n}\na.bg-info:hover,\na.bg-info:focus {\n background-color: #afd9ee;\n}\n.bg-warning {\n background-color: #fcf8e3;\n}\na.bg-warning:hover,\na.bg-warning:focus {\n background-color: #f7ecb5;\n}\n.bg-danger {\n background-color: #f2dede;\n}\na.bg-danger:hover,\na.bg-danger:focus {\n background-color: #e4b9b9;\n}\n.page-header {\n padding-bottom: 9px;\n margin: 40px 0 20px;\n border-bottom: 1px solid #eeeeee;\n}\nul,\nol {\n margin-top: 0;\n margin-bottom: 10px;\n}\nul ul,\nol ul,\nul ol,\nol ol {\n margin-bottom: 0;\n}\n.list-unstyled {\n padding-left: 0;\n list-style: none;\n}\n.list-inline {\n padding-left: 0;\n list-style: none;\n margin-left: -5px;\n}\n.list-inline > li {\n display: inline-block;\n padding-left: 5px;\n padding-right: 5px;\n}\ndl {\n margin-top: 0;\n margin-bottom: 20px;\n}\ndt,\ndd {\n line-height: 1.42857143;\n}\ndt {\n font-weight: bold;\n}\ndd {\n margin-left: 0;\n}\n@media (min-width: 768px) {\n .dl-horizontal dt {\n float: left;\n width: 160px;\n clear: left;\n text-align: right;\n overflow: hidden;\n text-overflow: ellipsis;\n white-space: nowrap;\n }\n .dl-horizontal dd {\n margin-left: 180px;\n }\n}\nabbr[title],\nabbr[data-original-title] {\n cursor: help;\n border-bottom: 1px dotted #777777;\n}\n.initialism {\n font-size: 90%;\n text-transform: uppercase;\n}\nblockquote {\n padding: 10px 20px;\n margin: 0 0 20px;\n font-size: 17.5px;\n border-left: 5px solid #eeeeee;\n}\nblockquote p:last-child,\nblockquote ul:last-child,\nblockquote ol:last-child {\n margin-bottom: 0;\n}\nblockquote footer,\nblockquote small,\nblockquote .small {\n display: block;\n font-size: 80%;\n line-height: 1.42857143;\n color: #777777;\n}\nblockquote footer:before,\nblockquote small:before,\nblockquote .small:before {\n content: '\\2014 \\00A0';\n}\n.blockquote-reverse,\nblockquote.pull-right {\n padding-right: 15px;\n padding-left: 0;\n border-right: 5px solid #eeeeee;\n border-left: 0;\n text-align: right;\n}\n.blockquote-reverse footer:before,\nblockquote.pull-right footer:before,\n.blockquote-reverse small:before,\nblockquote.pull-right small:before,\n.blockquote-reverse .small:before,\nblockquote.pull-right .small:before {\n content: '';\n}\n.blockquote-reverse footer:after,\nblockquote.pull-right footer:after,\n.blockquote-reverse small:after,\nblockquote.pull-right small:after,\n.blockquote-reverse .small:after,\nblockquote.pull-right .small:after {\n content: '\\00A0 \\2014';\n}\naddress {\n margin-bottom: 20px;\n font-style: normal;\n line-height: 1.42857143;\n}\ncode,\nkbd,\npre,\nsamp {\n font-family: Menlo, Monaco, Consolas, \"Courier New\", monospace;\n}\ncode {\n padding: 2px 4px;\n font-size: 90%;\n color: #c7254e;\n background-color: #f9f2f4;\n border-radius: 4px;\n}\nkbd {\n padding: 2px 4px;\n font-size: 90%;\n color: #ffffff;\n background-color: #333333;\n border-radius: 3px;\n box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.25);\n}\nkbd kbd {\n padding: 0;\n font-size: 100%;\n font-weight: bold;\n box-shadow: none;\n}\npre {\n display: block;\n padding: 9.5px;\n margin: 0 0 10px;\n font-size: 13px;\n line-height: 1.42857143;\n word-break: break-all;\n word-wrap: break-word;\n color: #333333;\n background-color: #f5f5f5;\n border: 1px solid #cccccc;\n border-radius: 4px;\n}\npre code {\n padding: 0;\n font-size: inherit;\n color: inherit;\n white-space: pre-wrap;\n background-color: transparent;\n border-radius: 0;\n}\n.pre-scrollable {\n max-height: 340px;\n overflow-y: scroll;\n}\n.container {\n margin-right: auto;\n margin-left: auto;\n padding-left: 15px;\n padding-right: 15px;\n}\n@media (min-width: 768px) {\n .container {\n width: 750px;\n }\n}\n@media (min-width: 992px) {\n .container {\n width: 970px;\n }\n}\n@media (min-width: 1200px) {\n .container {\n width: 1170px;\n }\n}\n.container-fluid {\n margin-right: auto;\n margin-left: auto;\n padding-left: 15px;\n padding-right: 15px;\n}\n.row {\n margin-left: -15px;\n margin-right: -15px;\n}\n.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 {\n position: relative;\n min-height: 1px;\n padding-left: 15px;\n padding-right: 15px;\n}\n.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 {\n float: left;\n}\n.col-xs-12 {\n width: 100%;\n}\n.col-xs-11 {\n width: 91.66666667%;\n}\n.col-xs-10 {\n width: 83.33333333%;\n}\n.col-xs-9 {\n width: 75%;\n}\n.col-xs-8 {\n width: 66.66666667%;\n}\n.col-xs-7 {\n width: 58.33333333%;\n}\n.col-xs-6 {\n width: 50%;\n}\n.col-xs-5 {\n width: 41.66666667%;\n}\n.col-xs-4 {\n width: 33.33333333%;\n}\n.col-xs-3 {\n width: 25%;\n}\n.col-xs-2 {\n width: 16.66666667%;\n}\n.col-xs-1 {\n width: 8.33333333%;\n}\n.col-xs-pull-12 {\n right: 100%;\n}\n.col-xs-pull-11 {\n right: 91.66666667%;\n}\n.col-xs-pull-10 {\n right: 83.33333333%;\n}\n.col-xs-pull-9 {\n right: 75%;\n}\n.col-xs-pull-8 {\n right: 66.66666667%;\n}\n.col-xs-pull-7 {\n right: 58.33333333%;\n}\n.col-xs-pull-6 {\n right: 50%;\n}\n.col-xs-pull-5 {\n right: 41.66666667%;\n}\n.col-xs-pull-4 {\n right: 33.33333333%;\n}\n.col-xs-pull-3 {\n right: 25%;\n}\n.col-xs-pull-2 {\n right: 16.66666667%;\n}\n.col-xs-pull-1 {\n right: 8.33333333%;\n}\n.col-xs-pull-0 {\n right: auto;\n}\n.col-xs-push-12 {\n left: 100%;\n}\n.col-xs-push-11 {\n left: 91.66666667%;\n}\n.col-xs-push-10 {\n left: 83.33333333%;\n}\n.col-xs-push-9 {\n left: 75%;\n}\n.col-xs-push-8 {\n left: 66.66666667%;\n}\n.col-xs-push-7 {\n left: 58.33333333%;\n}\n.col-xs-push-6 {\n left: 50%;\n}\n.col-xs-push-5 {\n left: 41.66666667%;\n}\n.col-xs-push-4 {\n left: 33.33333333%;\n}\n.col-xs-push-3 {\n left: 25%;\n}\n.col-xs-push-2 {\n left: 16.66666667%;\n}\n.col-xs-push-1 {\n left: 8.33333333%;\n}\n.col-xs-push-0 {\n left: auto;\n}\n.col-xs-offset-12 {\n margin-left: 100%;\n}\n.col-xs-offset-11 {\n margin-left: 91.66666667%;\n}\n.col-xs-offset-10 {\n margin-left: 83.33333333%;\n}\n.col-xs-offset-9 {\n margin-left: 75%;\n}\n.col-xs-offset-8 {\n margin-left: 66.66666667%;\n}\n.col-xs-offset-7 {\n margin-left: 58.33333333%;\n}\n.col-xs-offset-6 {\n margin-left: 50%;\n}\n.col-xs-offset-5 {\n margin-left: 41.66666667%;\n}\n.col-xs-offset-4 {\n margin-left: 33.33333333%;\n}\n.col-xs-offset-3 {\n margin-left: 25%;\n}\n.col-xs-offset-2 {\n margin-left: 16.66666667%;\n}\n.col-xs-offset-1 {\n margin-left: 8.33333333%;\n}\n.col-xs-offset-0 {\n margin-left: 0%;\n}\n@media (min-width: 768px) {\n .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 {\n float: left;\n }\n .col-sm-12 {\n width: 100%;\n }\n .col-sm-11 {\n width: 91.66666667%;\n }\n .col-sm-10 {\n width: 83.33333333%;\n }\n .col-sm-9 {\n width: 75%;\n }\n .col-sm-8 {\n width: 66.66666667%;\n }\n .col-sm-7 {\n width: 58.33333333%;\n }\n .col-sm-6 {\n width: 50%;\n }\n .col-sm-5 {\n width: 41.66666667%;\n }\n .col-sm-4 {\n width: 33.33333333%;\n }\n .col-sm-3 {\n width: 25%;\n }\n .col-sm-2 {\n width: 16.66666667%;\n }\n .col-sm-1 {\n width: 8.33333333%;\n }\n .col-sm-pull-12 {\n right: 100%;\n }\n .col-sm-pull-11 {\n right: 91.66666667%;\n }\n .col-sm-pull-10 {\n right: 83.33333333%;\n }\n .col-sm-pull-9 {\n right: 75%;\n }\n .col-sm-pull-8 {\n right: 66.66666667%;\n }\n .col-sm-pull-7 {\n right: 58.33333333%;\n }\n .col-sm-pull-6 {\n right: 50%;\n }\n .col-sm-pull-5 {\n right: 41.66666667%;\n }\n .col-sm-pull-4 {\n right: 33.33333333%;\n }\n .col-sm-pull-3 {\n right: 25%;\n }\n .col-sm-pull-2 {\n right: 16.66666667%;\n }\n .col-sm-pull-1 {\n right: 8.33333333%;\n }\n .col-sm-pull-0 {\n right: auto;\n }\n .col-sm-push-12 {\n left: 100%;\n }\n .col-sm-push-11 {\n left: 91.66666667%;\n }\n .col-sm-push-10 {\n left: 83.33333333%;\n }\n .col-sm-push-9 {\n left: 75%;\n }\n .col-sm-push-8 {\n left: 66.66666667%;\n }\n .col-sm-push-7 {\n left: 58.33333333%;\n }\n .col-sm-push-6 {\n left: 50%;\n }\n .col-sm-push-5 {\n left: 41.66666667%;\n }\n .col-sm-push-4 {\n left: 33.33333333%;\n }\n .col-sm-push-3 {\n left: 25%;\n }\n .col-sm-push-2 {\n left: 16.66666667%;\n }\n .col-sm-push-1 {\n left: 8.33333333%;\n }\n .col-sm-push-0 {\n left: auto;\n }\n .col-sm-offset-12 {\n margin-left: 100%;\n }\n .col-sm-offset-11 {\n margin-left: 91.66666667%;\n }\n .col-sm-offset-10 {\n margin-left: 83.33333333%;\n }\n .col-sm-offset-9 {\n margin-left: 75%;\n }\n .col-sm-offset-8 {\n margin-left: 66.66666667%;\n }\n .col-sm-offset-7 {\n margin-left: 58.33333333%;\n }\n .col-sm-offset-6 {\n margin-left: 50%;\n }\n .col-sm-offset-5 {\n margin-left: 41.66666667%;\n }\n .col-sm-offset-4 {\n margin-left: 33.33333333%;\n }\n .col-sm-offset-3 {\n margin-left: 25%;\n }\n .col-sm-offset-2 {\n margin-left: 16.66666667%;\n }\n .col-sm-offset-1 {\n margin-left: 8.33333333%;\n }\n .col-sm-offset-0 {\n margin-left: 0%;\n }\n}\n@media (min-width: 992px) {\n .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 {\n float: left;\n }\n .col-md-12 {\n width: 100%;\n }\n .col-md-11 {\n width: 91.66666667%;\n }\n .col-md-10 {\n width: 83.33333333%;\n }\n .col-md-9 {\n width: 75%;\n }\n .col-md-8 {\n width: 66.66666667%;\n }\n .col-md-7 {\n width: 58.33333333%;\n }\n .col-md-6 {\n width: 50%;\n }\n .col-md-5 {\n width: 41.66666667%;\n }\n .col-md-4 {\n width: 33.33333333%;\n }\n .col-md-3 {\n width: 25%;\n }\n .col-md-2 {\n width: 16.66666667%;\n }\n .col-md-1 {\n width: 8.33333333%;\n }\n .col-md-pull-12 {\n right: 100%;\n }\n .col-md-pull-11 {\n right: 91.66666667%;\n }\n .col-md-pull-10 {\n right: 83.33333333%;\n }\n .col-md-pull-9 {\n right: 75%;\n }\n .col-md-pull-8 {\n right: 66.66666667%;\n }\n .col-md-pull-7 {\n right: 58.33333333%;\n }\n .col-md-pull-6 {\n right: 50%;\n }\n .col-md-pull-5 {\n right: 41.66666667%;\n }\n .col-md-pull-4 {\n right: 33.33333333%;\n }\n .col-md-pull-3 {\n right: 25%;\n }\n .col-md-pull-2 {\n right: 16.66666667%;\n }\n .col-md-pull-1 {\n right: 8.33333333%;\n }\n .col-md-pull-0 {\n right: auto;\n }\n .col-md-push-12 {\n left: 100%;\n }\n .col-md-push-11 {\n left: 91.66666667%;\n }\n .col-md-push-10 {\n left: 83.33333333%;\n }\n .col-md-push-9 {\n left: 75%;\n }\n .col-md-push-8 {\n left: 66.66666667%;\n }\n .col-md-push-7 {\n left: 58.33333333%;\n }\n .col-md-push-6 {\n left: 50%;\n }\n .col-md-push-5 {\n left: 41.66666667%;\n }\n .col-md-push-4 {\n left: 33.33333333%;\n }\n .col-md-push-3 {\n left: 25%;\n }\n .col-md-push-2 {\n left: 16.66666667%;\n }\n .col-md-push-1 {\n left: 8.33333333%;\n }\n .col-md-push-0 {\n left: auto;\n }\n .col-md-offset-12 {\n margin-left: 100%;\n }\n .col-md-offset-11 {\n margin-left: 91.66666667%;\n }\n .col-md-offset-10 {\n margin-left: 83.33333333%;\n }\n .col-md-offset-9 {\n margin-left: 75%;\n }\n .col-md-offset-8 {\n margin-left: 66.66666667%;\n }\n .col-md-offset-7 {\n margin-left: 58.33333333%;\n }\n .col-md-offset-6 {\n margin-left: 50%;\n }\n .col-md-offset-5 {\n margin-left: 41.66666667%;\n }\n .col-md-offset-4 {\n margin-left: 33.33333333%;\n }\n .col-md-offset-3 {\n margin-left: 25%;\n }\n .col-md-offset-2 {\n margin-left: 16.66666667%;\n }\n .col-md-offset-1 {\n margin-left: 8.33333333%;\n }\n .col-md-offset-0 {\n margin-left: 0%;\n }\n}\n@media (min-width: 1200px) {\n .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {\n float: left;\n }\n .col-lg-12 {\n width: 100%;\n }\n .col-lg-11 {\n width: 91.66666667%;\n }\n .col-lg-10 {\n width: 83.33333333%;\n }\n .col-lg-9 {\n width: 75%;\n }\n .col-lg-8 {\n width: 66.66666667%;\n }\n .col-lg-7 {\n width: 58.33333333%;\n }\n .col-lg-6 {\n width: 50%;\n }\n .col-lg-5 {\n width: 41.66666667%;\n }\n .col-lg-4 {\n width: 33.33333333%;\n }\n .col-lg-3 {\n width: 25%;\n }\n .col-lg-2 {\n width: 16.66666667%;\n }\n .col-lg-1 {\n width: 8.33333333%;\n }\n .col-lg-pull-12 {\n right: 100%;\n }\n .col-lg-pull-11 {\n right: 91.66666667%;\n }\n .col-lg-pull-10 {\n right: 83.33333333%;\n }\n .col-lg-pull-9 {\n right: 75%;\n }\n .col-lg-pull-8 {\n right: 66.66666667%;\n }\n .col-lg-pull-7 {\n right: 58.33333333%;\n }\n .col-lg-pull-6 {\n right: 50%;\n }\n .col-lg-pull-5 {\n right: 41.66666667%;\n }\n .col-lg-pull-4 {\n right: 33.33333333%;\n }\n .col-lg-pull-3 {\n right: 25%;\n }\n .col-lg-pull-2 {\n right: 16.66666667%;\n }\n .col-lg-pull-1 {\n right: 8.33333333%;\n }\n .col-lg-pull-0 {\n right: auto;\n }\n .col-lg-push-12 {\n left: 100%;\n }\n .col-lg-push-11 {\n left: 91.66666667%;\n }\n .col-lg-push-10 {\n left: 83.33333333%;\n }\n .col-lg-push-9 {\n left: 75%;\n }\n .col-lg-push-8 {\n left: 66.66666667%;\n }\n .col-lg-push-7 {\n left: 58.33333333%;\n }\n .col-lg-push-6 {\n left: 50%;\n }\n .col-lg-push-5 {\n left: 41.66666667%;\n }\n .col-lg-push-4 {\n left: 33.33333333%;\n }\n .col-lg-push-3 {\n left: 25%;\n }\n .col-lg-push-2 {\n left: 16.66666667%;\n }\n .col-lg-push-1 {\n left: 8.33333333%;\n }\n .col-lg-push-0 {\n left: auto;\n }\n .col-lg-offset-12 {\n margin-left: 100%;\n }\n .col-lg-offset-11 {\n margin-left: 91.66666667%;\n }\n .col-lg-offset-10 {\n margin-left: 83.33333333%;\n }\n .col-lg-offset-9 {\n margin-left: 75%;\n }\n .col-lg-offset-8 {\n margin-left: 66.66666667%;\n }\n .col-lg-offset-7 {\n margin-left: 58.33333333%;\n }\n .col-lg-offset-6 {\n margin-left: 50%;\n }\n .col-lg-offset-5 {\n margin-left: 41.66666667%;\n }\n .col-lg-offset-4 {\n margin-left: 33.33333333%;\n }\n .col-lg-offset-3 {\n margin-left: 25%;\n }\n .col-lg-offset-2 {\n margin-left: 16.66666667%;\n }\n .col-lg-offset-1 {\n margin-left: 8.33333333%;\n }\n .col-lg-offset-0 {\n margin-left: 0%;\n }\n}\ntable {\n background-color: transparent;\n}\ncaption {\n padding-top: 8px;\n padding-bottom: 8px;\n color: #777777;\n text-align: left;\n}\nth {\n text-align: left;\n}\n.table {\n width: 100%;\n max-width: 100%;\n margin-bottom: 20px;\n}\n.table > thead > tr > th,\n.table > tbody > tr > th,\n.table > tfoot > tr > th,\n.table > thead > tr > td,\n.table > tbody > tr > td,\n.table > tfoot > tr > td {\n padding: 8px;\n line-height: 1.42857143;\n vertical-align: top;\n border-top: 1px solid #dddddd;\n}\n.table > thead > tr > th {\n vertical-align: bottom;\n border-bottom: 2px solid #dddddd;\n}\n.table > caption + thead > tr:first-child > th,\n.table > colgroup + thead > tr:first-child > th,\n.table > thead:first-child > tr:first-child > th,\n.table > caption + thead > tr:first-child > td,\n.table > colgroup + thead > tr:first-child > td,\n.table > thead:first-child > tr:first-child > td {\n border-top: 0;\n}\n.table > tbody + tbody {\n border-top: 2px solid #dddddd;\n}\n.table .table {\n background-color: #ffffff;\n}\n.table-condensed > thead > tr > th,\n.table-condensed > tbody > tr > th,\n.table-condensed > tfoot > tr > th,\n.table-condensed > thead > tr > td,\n.table-condensed > tbody > tr > td,\n.table-condensed > tfoot > tr > td {\n padding: 5px;\n}\n.table-bordered {\n border: 1px solid #dddddd;\n}\n.table-bordered > thead > tr > th,\n.table-bordered > tbody > tr > th,\n.table-bordered > tfoot > tr > th,\n.table-bordered > thead > tr > td,\n.table-bordered > tbody > tr > td,\n.table-bordered > tfoot > tr > td {\n border: 1px solid #dddddd;\n}\n.table-bordered > thead > tr > th,\n.table-bordered > thead > tr > td {\n border-bottom-width: 2px;\n}\n.table-striped > tbody > tr:nth-of-type(odd) {\n background-color: #f9f9f9;\n}\n.table-hover > tbody > tr:hover {\n background-color: #f5f5f5;\n}\ntable col[class*=\"col-\"] {\n position: static;\n float: none;\n display: table-column;\n}\ntable td[class*=\"col-\"],\ntable th[class*=\"col-\"] {\n position: static;\n float: none;\n display: table-cell;\n}\n.table > thead > tr > td.active,\n.table > tbody > tr > td.active,\n.table > tfoot > tr > td.active,\n.table > thead > tr > th.active,\n.table > tbody > tr > th.active,\n.table > tfoot > tr > th.active,\n.table > thead > tr.active > td,\n.table > tbody > tr.active > td,\n.table > tfoot > tr.active > td,\n.table > thead > tr.active > th,\n.table > tbody > tr.active > th,\n.table > tfoot > tr.active > th {\n background-color: #f5f5f5;\n}\n.table-hover > tbody > tr > td.active:hover,\n.table-hover > tbody > tr > th.active:hover,\n.table-hover > tbody > tr.active:hover > td,\n.table-hover > tbody > tr:hover > .active,\n.table-hover > tbody > tr.active:hover > th {\n background-color: #e8e8e8;\n}\n.table > thead > tr > td.success,\n.table > tbody > tr > td.success,\n.table > tfoot > tr > td.success,\n.table > thead > tr > th.success,\n.table > tbody > tr > th.success,\n.table > tfoot > tr > th.success,\n.table > thead > tr.success > td,\n.table > tbody > tr.success > td,\n.table > tfoot > tr.success > td,\n.table > thead > tr.success > th,\n.table > tbody > tr.success > th,\n.table > tfoot > tr.success > th {\n background-color: #dff0d8;\n}\n.table-hover > tbody > tr > td.success:hover,\n.table-hover > tbody > tr > th.success:hover,\n.table-hover > tbody > tr.success:hover > td,\n.table-hover > tbody > tr:hover > .success,\n.table-hover > tbody > tr.success:hover > th {\n background-color: #d0e9c6;\n}\n.table > thead > tr > td.info,\n.table > tbody > tr > td.info,\n.table > tfoot > tr > td.info,\n.table > thead > tr > th.info,\n.table > tbody > tr > th.info,\n.table > tfoot > tr > th.info,\n.table > thead > tr.info > td,\n.table > tbody > tr.info > td,\n.table > tfoot > tr.info > td,\n.table > thead > tr.info > th,\n.table > tbody > tr.info > th,\n.table > tfoot > tr.info > th {\n background-color: #d9edf7;\n}\n.table-hover > tbody > tr > td.info:hover,\n.table-hover > tbody > tr > th.info:hover,\n.table-hover > tbody > tr.info:hover > td,\n.table-hover > tbody > tr:hover > .info,\n.table-hover > tbody > tr.info:hover > th {\n background-color: #c4e3f3;\n}\n.table > thead > tr > td.warning,\n.table > tbody > tr > td.warning,\n.table > tfoot > tr > td.warning,\n.table > thead > tr > th.warning,\n.table > tbody > tr > th.warning,\n.table > tfoot > tr > th.warning,\n.table > thead > tr.warning > td,\n.table > tbody > tr.warning > td,\n.table > tfoot > tr.warning > td,\n.table > thead > tr.warning > th,\n.table > tbody > tr.warning > th,\n.table > tfoot > tr.warning > th {\n background-color: #fcf8e3;\n}\n.table-hover > tbody > tr > td.warning:hover,\n.table-hover > tbody > tr > th.warning:hover,\n.table-hover > tbody > tr.warning:hover > td,\n.table-hover > tbody > tr:hover > .warning,\n.table-hover > tbody > tr.warning:hover > th {\n background-color: #faf2cc;\n}\n.table > thead > tr > td.danger,\n.table > tbody > tr > td.danger,\n.table > tfoot > tr > td.danger,\n.table > thead > tr > th.danger,\n.table > tbody > tr > th.danger,\n.table > tfoot > tr > th.danger,\n.table > thead > tr.danger > td,\n.table > tbody > tr.danger > td,\n.table > tfoot > tr.danger > td,\n.table > thead > tr.danger > th,\n.table > tbody > tr.danger > th,\n.table > tfoot > tr.danger > th {\n background-color: #f2dede;\n}\n.table-hover > tbody > tr > td.danger:hover,\n.table-hover > tbody > tr > th.danger:hover,\n.table-hover > tbody > tr.danger:hover > td,\n.table-hover > tbody > tr:hover > .danger,\n.table-hover > tbody > tr.danger:hover > th {\n background-color: #ebcccc;\n}\n.table-responsive {\n overflow-x: auto;\n min-height: 0.01%;\n}\n@media screen and (max-width: 767px) {\n .table-responsive {\n width: 100%;\n margin-bottom: 15px;\n overflow-y: hidden;\n -ms-overflow-style: -ms-autohiding-scrollbar;\n border: 1px solid #dddddd;\n }\n .table-responsive > .table {\n margin-bottom: 0;\n }\n .table-responsive > .table > thead > tr > th,\n .table-responsive > .table > tbody > tr > th,\n .table-responsive > .table > tfoot > tr > th,\n .table-responsive > .table > thead > tr > td,\n .table-responsive > .table > tbody > tr > td,\n .table-responsive > .table > tfoot > tr > td {\n white-space: nowrap;\n }\n .table-responsive > .table-bordered {\n border: 0;\n }\n .table-responsive > .table-bordered > thead > tr > th:first-child,\n .table-responsive > .table-bordered > tbody > tr > th:first-child,\n .table-responsive > .table-bordered > tfoot > tr > th:first-child,\n .table-responsive > .table-bordered > thead > tr > td:first-child,\n .table-responsive > .table-bordered > tbody > tr > td:first-child,\n .table-responsive > .table-bordered > tfoot > tr > td:first-child {\n border-left: 0;\n }\n .table-responsive > .table-bordered > thead > tr > th:last-child,\n .table-responsive > .table-bordered > tbody > tr > th:last-child,\n .table-responsive > .table-bordered > tfoot > tr > th:last-child,\n .table-responsive > .table-bordered > thead > tr > td:last-child,\n .table-responsive > .table-bordered > tbody > tr > td:last-child,\n .table-responsive > .table-bordered > tfoot > tr > td:last-child {\n border-right: 0;\n }\n .table-responsive > .table-bordered > tbody > tr:last-child > th,\n .table-responsive > .table-bordered > tfoot > tr:last-child > th,\n .table-responsive > .table-bordered > tbody > tr:last-child > td,\n .table-responsive > .table-bordered > tfoot > tr:last-child > td {\n border-bottom: 0;\n }\n}\nfieldset {\n padding: 0;\n margin: 0;\n border: 0;\n min-width: 0;\n}\nlegend {\n display: block;\n width: 100%;\n padding: 0;\n margin-bottom: 20px;\n font-size: 21px;\n line-height: inherit;\n color: #333333;\n border: 0;\n border-bottom: 1px solid #e5e5e5;\n}\nlabel {\n display: inline-block;\n max-width: 100%;\n margin-bottom: 5px;\n font-weight: bold;\n}\ninput[type=\"search\"] {\n -webkit-box-sizing: border-box;\n -moz-box-sizing: border-box;\n box-sizing: border-box;\n}\ninput[type=\"radio\"],\ninput[type=\"checkbox\"] {\n margin: 4px 0 0;\n margin-top: 1px \\9;\n line-height: normal;\n}\ninput[type=\"file\"] {\n display: block;\n}\ninput[type=\"range\"] {\n display: block;\n width: 100%;\n}\nselect[multiple],\nselect[size] {\n height: auto;\n}\ninput[type=\"file\"]:focus,\ninput[type=\"radio\"]:focus,\ninput[type=\"checkbox\"]:focus {\n outline: thin dotted;\n outline: 5px auto -webkit-focus-ring-color;\n outline-offset: -2px;\n}\noutput {\n display: block;\n padding-top: 7px;\n font-size: 14px;\n line-height: 1.42857143;\n color: #555555;\n}\n.form-control {\n display: block;\n width: 100%;\n height: 34px;\n padding: 6px 12px;\n font-size: 14px;\n line-height: 1.42857143;\n color: #555555;\n background-color: #ffffff;\n background-image: none;\n border: 1px solid #cccccc;\n border-radius: 4px;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;\n -o-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;\n transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;\n}\n.form-control:focus {\n border-color: #66afe9;\n outline: 0;\n -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);\n box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);\n}\n.form-control::-moz-placeholder {\n color: #999999;\n opacity: 1;\n}\n.form-control:-ms-input-placeholder {\n color: #999999;\n}\n.form-control::-webkit-input-placeholder {\n color: #999999;\n}\n.form-control[disabled],\n.form-control[readonly],\nfieldset[disabled] .form-control {\n background-color: #eeeeee;\n opacity: 1;\n}\n.form-control[disabled],\nfieldset[disabled] .form-control {\n cursor: not-allowed;\n}\ntextarea.form-control {\n height: auto;\n}\ninput[type=\"search\"] {\n -webkit-appearance: none;\n}\n@media screen and (-webkit-min-device-pixel-ratio: 0) {\n input[type=\"date\"].form-control,\n input[type=\"time\"].form-control,\n input[type=\"datetime-local\"].form-control,\n input[type=\"month\"].form-control {\n line-height: 34px;\n }\n input[type=\"date\"].input-sm,\n input[type=\"time\"].input-sm,\n input[type=\"datetime-local\"].input-sm,\n input[type=\"month\"].input-sm,\n .input-group-sm input[type=\"date\"],\n .input-group-sm input[type=\"time\"],\n .input-group-sm input[type=\"datetime-local\"],\n .input-group-sm input[type=\"month\"] {\n line-height: 30px;\n }\n input[type=\"date\"].input-lg,\n input[type=\"time\"].input-lg,\n input[type=\"datetime-local\"].input-lg,\n input[type=\"month\"].input-lg,\n .input-group-lg input[type=\"date\"],\n .input-group-lg input[type=\"time\"],\n .input-group-lg input[type=\"datetime-local\"],\n .input-group-lg input[type=\"month\"] {\n line-height: 46px;\n }\n}\n.form-group {\n margin-bottom: 15px;\n}\n.radio,\n.checkbox {\n position: relative;\n display: block;\n margin-top: 10px;\n margin-bottom: 10px;\n}\n.radio label,\n.checkbox label {\n min-height: 20px;\n padding-left: 20px;\n margin-bottom: 0;\n font-weight: normal;\n cursor: pointer;\n}\n.radio input[type=\"radio\"],\n.radio-inline input[type=\"radio\"],\n.checkbox input[type=\"checkbox\"],\n.checkbox-inline input[type=\"checkbox\"] {\n position: absolute;\n margin-left: -20px;\n margin-top: 4px \\9;\n}\n.radio + .radio,\n.checkbox + .checkbox {\n margin-top: -5px;\n}\n.radio-inline,\n.checkbox-inline {\n position: relative;\n display: inline-block;\n padding-left: 20px;\n margin-bottom: 0;\n vertical-align: middle;\n font-weight: normal;\n cursor: pointer;\n}\n.radio-inline + .radio-inline,\n.checkbox-inline + .checkbox-inline {\n margin-top: 0;\n margin-left: 10px;\n}\ninput[type=\"radio\"][disabled],\ninput[type=\"checkbox\"][disabled],\ninput[type=\"radio\"].disabled,\ninput[type=\"checkbox\"].disabled,\nfieldset[disabled] input[type=\"radio\"],\nfieldset[disabled] input[type=\"checkbox\"] {\n cursor: not-allowed;\n}\n.radio-inline.disabled,\n.checkbox-inline.disabled,\nfieldset[disabled] .radio-inline,\nfieldset[disabled] .checkbox-inline {\n cursor: not-allowed;\n}\n.radio.disabled label,\n.checkbox.disabled label,\nfieldset[disabled] .radio label,\nfieldset[disabled] .checkbox label {\n cursor: not-allowed;\n}\n.form-control-static {\n padding-top: 7px;\n padding-bottom: 7px;\n margin-bottom: 0;\n min-height: 34px;\n}\n.form-control-static.input-lg,\n.form-control-static.input-sm {\n padding-left: 0;\n padding-right: 0;\n}\n.input-sm {\n height: 30px;\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\nselect.input-sm {\n height: 30px;\n line-height: 30px;\n}\ntextarea.input-sm,\nselect[multiple].input-sm {\n height: auto;\n}\n.form-group-sm .form-control {\n height: 30px;\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\n.form-group-sm select.form-control {\n height: 30px;\n line-height: 30px;\n}\n.form-group-sm textarea.form-control,\n.form-group-sm select[multiple].form-control {\n height: auto;\n}\n.form-group-sm .form-control-static {\n height: 30px;\n min-height: 32px;\n padding: 6px 10px;\n font-size: 12px;\n line-height: 1.5;\n}\n.input-lg {\n height: 46px;\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n border-radius: 6px;\n}\nselect.input-lg {\n height: 46px;\n line-height: 46px;\n}\ntextarea.input-lg,\nselect[multiple].input-lg {\n height: auto;\n}\n.form-group-lg .form-control {\n height: 46px;\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n border-radius: 6px;\n}\n.form-group-lg select.form-control {\n height: 46px;\n line-height: 46px;\n}\n.form-group-lg textarea.form-control,\n.form-group-lg select[multiple].form-control {\n height: auto;\n}\n.form-group-lg .form-control-static {\n height: 46px;\n min-height: 38px;\n padding: 11px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n}\n.has-feedback {\n position: relative;\n}\n.has-feedback .form-control {\n padding-right: 42.5px;\n}\n.form-control-feedback {\n position: absolute;\n top: 0;\n right: 0;\n z-index: 2;\n display: block;\n width: 34px;\n height: 34px;\n line-height: 34px;\n text-align: center;\n pointer-events: none;\n}\n.input-lg + .form-control-feedback,\n.input-group-lg + .form-control-feedback,\n.form-group-lg .form-control + .form-control-feedback {\n width: 46px;\n height: 46px;\n line-height: 46px;\n}\n.input-sm + .form-control-feedback,\n.input-group-sm + .form-control-feedback,\n.form-group-sm .form-control + .form-control-feedback {\n width: 30px;\n height: 30px;\n line-height: 30px;\n}\n.has-success .help-block,\n.has-success .control-label,\n.has-success .radio,\n.has-success .checkbox,\n.has-success .radio-inline,\n.has-success .checkbox-inline,\n.has-success.radio label,\n.has-success.checkbox label,\n.has-success.radio-inline label,\n.has-success.checkbox-inline label {\n color: #3c763d;\n}\n.has-success .form-control {\n border-color: #3c763d;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n}\n.has-success .form-control:focus {\n border-color: #2b542c;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;\n}\n.has-success .input-group-addon {\n color: #3c763d;\n border-color: #3c763d;\n background-color: #dff0d8;\n}\n.has-success .form-control-feedback {\n color: #3c763d;\n}\n.has-warning .help-block,\n.has-warning .control-label,\n.has-warning .radio,\n.has-warning .checkbox,\n.has-warning .radio-inline,\n.has-warning .checkbox-inline,\n.has-warning.radio label,\n.has-warning.checkbox label,\n.has-warning.radio-inline label,\n.has-warning.checkbox-inline label {\n color: #8a6d3b;\n}\n.has-warning .form-control {\n border-color: #8a6d3b;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n}\n.has-warning .form-control:focus {\n border-color: #66512c;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;\n}\n.has-warning .input-group-addon {\n color: #8a6d3b;\n border-color: #8a6d3b;\n background-color: #fcf8e3;\n}\n.has-warning .form-control-feedback {\n color: #8a6d3b;\n}\n.has-error .help-block,\n.has-error .control-label,\n.has-error .radio,\n.has-error .checkbox,\n.has-error .radio-inline,\n.has-error .checkbox-inline,\n.has-error.radio label,\n.has-error.checkbox label,\n.has-error.radio-inline label,\n.has-error.checkbox-inline label {\n color: #a94442;\n}\n.has-error .form-control {\n border-color: #a94442;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n}\n.has-error .form-control:focus {\n border-color: #843534;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;\n}\n.has-error .input-group-addon {\n color: #a94442;\n border-color: #a94442;\n background-color: #f2dede;\n}\n.has-error .form-control-feedback {\n color: #a94442;\n}\n.has-feedback label ~ .form-control-feedback {\n top: 25px;\n}\n.has-feedback label.sr-only ~ .form-control-feedback {\n top: 0;\n}\n.help-block {\n display: block;\n margin-top: 5px;\n margin-bottom: 10px;\n color: #737373;\n}\n@media (min-width: 768px) {\n .form-inline .form-group {\n display: inline-block;\n margin-bottom: 0;\n vertical-align: middle;\n }\n .form-inline .form-control {\n display: inline-block;\n width: auto;\n vertical-align: middle;\n }\n .form-inline .form-control-static {\n display: inline-block;\n }\n .form-inline .input-group {\n display: inline-table;\n vertical-align: middle;\n }\n .form-inline .input-group .input-group-addon,\n .form-inline .input-group .input-group-btn,\n .form-inline .input-group .form-control {\n width: auto;\n }\n .form-inline .input-group > .form-control {\n width: 100%;\n }\n .form-inline .control-label {\n margin-bottom: 0;\n vertical-align: middle;\n }\n .form-inline .radio,\n .form-inline .checkbox {\n display: inline-block;\n margin-top: 0;\n margin-bottom: 0;\n vertical-align: middle;\n }\n .form-inline .radio label,\n .form-inline .checkbox label {\n padding-left: 0;\n }\n .form-inline .radio input[type=\"radio\"],\n .form-inline .checkbox input[type=\"checkbox\"] {\n position: relative;\n margin-left: 0;\n }\n .form-inline .has-feedback .form-control-feedback {\n top: 0;\n }\n}\n.form-horizontal .radio,\n.form-horizontal .checkbox,\n.form-horizontal .radio-inline,\n.form-horizontal .checkbox-inline {\n margin-top: 0;\n margin-bottom: 0;\n padding-top: 7px;\n}\n.form-horizontal .radio,\n.form-horizontal .checkbox {\n min-height: 27px;\n}\n.form-horizontal .form-group {\n margin-left: -15px;\n margin-right: -15px;\n}\n@media (min-width: 768px) {\n .form-horizontal .control-label {\n text-align: right;\n margin-bottom: 0;\n padding-top: 7px;\n }\n}\n.form-horizontal .has-feedback .form-control-feedback {\n right: 15px;\n}\n@media (min-width: 768px) {\n .form-horizontal .form-group-lg .control-label {\n padding-top: 14.333333px;\n font-size: 18px;\n }\n}\n@media (min-width: 768px) {\n .form-horizontal .form-group-sm .control-label {\n padding-top: 6px;\n font-size: 12px;\n }\n}\n.btn {\n display: inline-block;\n margin-bottom: 0;\n font-weight: normal;\n text-align: center;\n vertical-align: middle;\n touch-action: manipulation;\n cursor: pointer;\n background-image: none;\n border: 1px solid transparent;\n white-space: nowrap;\n padding: 6px 12px;\n font-size: 14px;\n line-height: 1.42857143;\n border-radius: 4px;\n -webkit-user-select: none;\n -moz-user-select: none;\n -ms-user-select: none;\n user-select: none;\n}\n.btn:focus,\n.btn:active:focus,\n.btn.active:focus,\n.btn.focus,\n.btn:active.focus,\n.btn.active.focus {\n outline: thin dotted;\n outline: 5px auto -webkit-focus-ring-color;\n outline-offset: -2px;\n}\n.btn:hover,\n.btn:focus,\n.btn.focus {\n color: #333333;\n text-decoration: none;\n}\n.btn:active,\n.btn.active {\n outline: 0;\n background-image: none;\n -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n}\n.btn.disabled,\n.btn[disabled],\nfieldset[disabled] .btn {\n cursor: not-allowed;\n opacity: 0.65;\n filter: alpha(opacity=65);\n -webkit-box-shadow: none;\n box-shadow: none;\n}\na.btn.disabled,\nfieldset[disabled] a.btn {\n pointer-events: none;\n}\n.btn-default {\n color: #333333;\n background-color: #ffffff;\n border-color: #cccccc;\n}\n.btn-default:focus,\n.btn-default.focus {\n color: #333333;\n background-color: #e6e6e6;\n border-color: #8c8c8c;\n}\n.btn-default:hover {\n color: #333333;\n background-color: #e6e6e6;\n border-color: #adadad;\n}\n.btn-default:active,\n.btn-default.active,\n.open > .dropdown-toggle.btn-default {\n color: #333333;\n background-color: #e6e6e6;\n border-color: #adadad;\n}\n.btn-default:active:hover,\n.btn-default.active:hover,\n.open > .dropdown-toggle.btn-default:hover,\n.btn-default:active:focus,\n.btn-default.active:focus,\n.open > .dropdown-toggle.btn-default:focus,\n.btn-default:active.focus,\n.btn-default.active.focus,\n.open > .dropdown-toggle.btn-default.focus {\n color: #333333;\n background-color: #d4d4d4;\n border-color: #8c8c8c;\n}\n.btn-default:active,\n.btn-default.active,\n.open > .dropdown-toggle.btn-default {\n background-image: none;\n}\n.btn-default.disabled,\n.btn-default[disabled],\nfieldset[disabled] .btn-default,\n.btn-default.disabled:hover,\n.btn-default[disabled]:hover,\nfieldset[disabled] .btn-default:hover,\n.btn-default.disabled:focus,\n.btn-default[disabled]:focus,\nfieldset[disabled] .btn-default:focus,\n.btn-default.disabled.focus,\n.btn-default[disabled].focus,\nfieldset[disabled] .btn-default.focus,\n.btn-default.disabled:active,\n.btn-default[disabled]:active,\nfieldset[disabled] .btn-default:active,\n.btn-default.disabled.active,\n.btn-default[disabled].active,\nfieldset[disabled] .btn-default.active {\n background-color: #ffffff;\n border-color: #cccccc;\n}\n.btn-default .badge {\n color: #ffffff;\n background-color: #333333;\n}\n.btn-primary {\n color: #ffffff;\n background-color: #337ab7;\n border-color: #2e6da4;\n}\n.btn-primary:focus,\n.btn-primary.focus {\n color: #ffffff;\n background-color: #286090;\n border-color: #122b40;\n}\n.btn-primary:hover {\n color: #ffffff;\n background-color: #286090;\n border-color: #204d74;\n}\n.btn-primary:active,\n.btn-primary.active,\n.open > .dropdown-toggle.btn-primary {\n color: #ffffff;\n background-color: #286090;\n border-color: #204d74;\n}\n.btn-primary:active:hover,\n.btn-primary.active:hover,\n.open > .dropdown-toggle.btn-primary:hover,\n.btn-primary:active:focus,\n.btn-primary.active:focus,\n.open > .dropdown-toggle.btn-primary:focus,\n.btn-primary:active.focus,\n.btn-primary.active.focus,\n.open > .dropdown-toggle.btn-primary.focus {\n color: #ffffff;\n background-color: #204d74;\n border-color: #122b40;\n}\n.btn-primary:active,\n.btn-primary.active,\n.open > .dropdown-toggle.btn-primary {\n background-image: none;\n}\n.btn-primary.disabled,\n.btn-primary[disabled],\nfieldset[disabled] .btn-primary,\n.btn-primary.disabled:hover,\n.btn-primary[disabled]:hover,\nfieldset[disabled] .btn-primary:hover,\n.btn-primary.disabled:focus,\n.btn-primary[disabled]:focus,\nfieldset[disabled] .btn-primary:focus,\n.btn-primary.disabled.focus,\n.btn-primary[disabled].focus,\nfieldset[disabled] .btn-primary.focus,\n.btn-primary.disabled:active,\n.btn-primary[disabled]:active,\nfieldset[disabled] .btn-primary:active,\n.btn-primary.disabled.active,\n.btn-primary[disabled].active,\nfieldset[disabled] .btn-primary.active {\n background-color: #337ab7;\n border-color: #2e6da4;\n}\n.btn-primary .badge {\n color: #337ab7;\n background-color: #ffffff;\n}\n.btn-success {\n color: #ffffff;\n background-color: #5cb85c;\n border-color: #4cae4c;\n}\n.btn-success:focus,\n.btn-success.focus {\n color: #ffffff;\n background-color: #449d44;\n border-color: #255625;\n}\n.btn-success:hover {\n color: #ffffff;\n background-color: #449d44;\n border-color: #398439;\n}\n.btn-success:active,\n.btn-success.active,\n.open > .dropdown-toggle.btn-success {\n color: #ffffff;\n background-color: #449d44;\n border-color: #398439;\n}\n.btn-success:active:hover,\n.btn-success.active:hover,\n.open > .dropdown-toggle.btn-success:hover,\n.btn-success:active:focus,\n.btn-success.active:focus,\n.open > .dropdown-toggle.btn-success:focus,\n.btn-success:active.focus,\n.btn-success.active.focus,\n.open > .dropdown-toggle.btn-success.focus {\n color: #ffffff;\n background-color: #398439;\n border-color: #255625;\n}\n.btn-success:active,\n.btn-success.active,\n.open > .dropdown-toggle.btn-success {\n background-image: none;\n}\n.btn-success.disabled,\n.btn-success[disabled],\nfieldset[disabled] .btn-success,\n.btn-success.disabled:hover,\n.btn-success[disabled]:hover,\nfieldset[disabled] .btn-success:hover,\n.btn-success.disabled:focus,\n.btn-success[disabled]:focus,\nfieldset[disabled] .btn-success:focus,\n.btn-success.disabled.focus,\n.btn-success[disabled].focus,\nfieldset[disabled] .btn-success.focus,\n.btn-success.disabled:active,\n.btn-success[disabled]:active,\nfieldset[disabled] .btn-success:active,\n.btn-success.disabled.active,\n.btn-success[disabled].active,\nfieldset[disabled] .btn-success.active {\n background-color: #5cb85c;\n border-color: #4cae4c;\n}\n.btn-success .badge {\n color: #5cb85c;\n background-color: #ffffff;\n}\n.btn-info {\n color: #ffffff;\n background-color: #5bc0de;\n border-color: #46b8da;\n}\n.btn-info:focus,\n.btn-info.focus {\n color: #ffffff;\n background-color: #31b0d5;\n border-color: #1b6d85;\n}\n.btn-info:hover {\n color: #ffffff;\n background-color: #31b0d5;\n border-color: #269abc;\n}\n.btn-info:active,\n.btn-info.active,\n.open > .dropdown-toggle.btn-info {\n color: #ffffff;\n background-color: #31b0d5;\n border-color: #269abc;\n}\n.btn-info:active:hover,\n.btn-info.active:hover,\n.open > .dropdown-toggle.btn-info:hover,\n.btn-info:active:focus,\n.btn-info.active:focus,\n.open > .dropdown-toggle.btn-info:focus,\n.btn-info:active.focus,\n.btn-info.active.focus,\n.open > .dropdown-toggle.btn-info.focus {\n color: #ffffff;\n background-color: #269abc;\n border-color: #1b6d85;\n}\n.btn-info:active,\n.btn-info.active,\n.open > .dropdown-toggle.btn-info {\n background-image: none;\n}\n.btn-info.disabled,\n.btn-info[disabled],\nfieldset[disabled] .btn-info,\n.btn-info.disabled:hover,\n.btn-info[disabled]:hover,\nfieldset[disabled] .btn-info:hover,\n.btn-info.disabled:focus,\n.btn-info[disabled]:focus,\nfieldset[disabled] .btn-info:focus,\n.btn-info.disabled.focus,\n.btn-info[disabled].focus,\nfieldset[disabled] .btn-info.focus,\n.btn-info.disabled:active,\n.btn-info[disabled]:active,\nfieldset[disabled] .btn-info:active,\n.btn-info.disabled.active,\n.btn-info[disabled].active,\nfieldset[disabled] .btn-info.active {\n background-color: #5bc0de;\n border-color: #46b8da;\n}\n.btn-info .badge {\n color: #5bc0de;\n background-color: #ffffff;\n}\n.btn-warning {\n color: #ffffff;\n background-color: #f0ad4e;\n border-color: #eea236;\n}\n.btn-warning:focus,\n.btn-warning.focus {\n color: #ffffff;\n background-color: #ec971f;\n border-color: #985f0d;\n}\n.btn-warning:hover {\n color: #ffffff;\n background-color: #ec971f;\n border-color: #d58512;\n}\n.btn-warning:active,\n.btn-warning.active,\n.open > .dropdown-toggle.btn-warning {\n color: #ffffff;\n background-color: #ec971f;\n border-color: #d58512;\n}\n.btn-warning:active:hover,\n.btn-warning.active:hover,\n.open > .dropdown-toggle.btn-warning:hover,\n.btn-warning:active:focus,\n.btn-warning.active:focus,\n.open > .dropdown-toggle.btn-warning:focus,\n.btn-warning:active.focus,\n.btn-warning.active.focus,\n.open > .dropdown-toggle.btn-warning.focus {\n color: #ffffff;\n background-color: #d58512;\n border-color: #985f0d;\n}\n.btn-warning:active,\n.btn-warning.active,\n.open > .dropdown-toggle.btn-warning {\n background-image: none;\n}\n.btn-warning.disabled,\n.btn-warning[disabled],\nfieldset[disabled] .btn-warning,\n.btn-warning.disabled:hover,\n.btn-warning[disabled]:hover,\nfieldset[disabled] .btn-warning:hover,\n.btn-warning.disabled:focus,\n.btn-warning[disabled]:focus,\nfieldset[disabled] .btn-warning:focus,\n.btn-warning.disabled.focus,\n.btn-warning[disabled].focus,\nfieldset[disabled] .btn-warning.focus,\n.btn-warning.disabled:active,\n.btn-warning[disabled]:active,\nfieldset[disabled] .btn-warning:active,\n.btn-warning.disabled.active,\n.btn-warning[disabled].active,\nfieldset[disabled] .btn-warning.active {\n background-color: #f0ad4e;\n border-color: #eea236;\n}\n.btn-warning .badge {\n color: #f0ad4e;\n background-color: #ffffff;\n}\n.btn-danger {\n color: #ffffff;\n background-color: #d9534f;\n border-color: #d43f3a;\n}\n.btn-danger:focus,\n.btn-danger.focus {\n color: #ffffff;\n background-color: #c9302c;\n border-color: #761c19;\n}\n.btn-danger:hover {\n color: #ffffff;\n background-color: #c9302c;\n border-color: #ac2925;\n}\n.btn-danger:active,\n.btn-danger.active,\n.open > .dropdown-toggle.btn-danger {\n color: #ffffff;\n background-color: #c9302c;\n border-color: #ac2925;\n}\n.btn-danger:active:hover,\n.btn-danger.active:hover,\n.open > .dropdown-toggle.btn-danger:hover,\n.btn-danger:active:focus,\n.btn-danger.active:focus,\n.open > .dropdown-toggle.btn-danger:focus,\n.btn-danger:active.focus,\n.btn-danger.active.focus,\n.open > .dropdown-toggle.btn-danger.focus {\n color: #ffffff;\n background-color: #ac2925;\n border-color: #761c19;\n}\n.btn-danger:active,\n.btn-danger.active,\n.open > .dropdown-toggle.btn-danger {\n background-image: none;\n}\n.btn-danger.disabled,\n.btn-danger[disabled],\nfieldset[disabled] .btn-danger,\n.btn-danger.disabled:hover,\n.btn-danger[disabled]:hover,\nfieldset[disabled] .btn-danger:hover,\n.btn-danger.disabled:focus,\n.btn-danger[disabled]:focus,\nfieldset[disabled] .btn-danger:focus,\n.btn-danger.disabled.focus,\n.btn-danger[disabled].focus,\nfieldset[disabled] .btn-danger.focus,\n.btn-danger.disabled:active,\n.btn-danger[disabled]:active,\nfieldset[disabled] .btn-danger:active,\n.btn-danger.disabled.active,\n.btn-danger[disabled].active,\nfieldset[disabled] .btn-danger.active {\n background-color: #d9534f;\n border-color: #d43f3a;\n}\n.btn-danger .badge {\n color: #d9534f;\n background-color: #ffffff;\n}\n.btn-link {\n color: #337ab7;\n font-weight: normal;\n border-radius: 0;\n}\n.btn-link,\n.btn-link:active,\n.btn-link.active,\n.btn-link[disabled],\nfieldset[disabled] .btn-link {\n background-color: transparent;\n -webkit-box-shadow: none;\n box-shadow: none;\n}\n.btn-link,\n.btn-link:hover,\n.btn-link:focus,\n.btn-link:active {\n border-color: transparent;\n}\n.btn-link:hover,\n.btn-link:focus {\n color: #23527c;\n text-decoration: underline;\n background-color: transparent;\n}\n.btn-link[disabled]:hover,\nfieldset[disabled] .btn-link:hover,\n.btn-link[disabled]:focus,\nfieldset[disabled] .btn-link:focus {\n color: #777777;\n text-decoration: none;\n}\n.btn-lg,\n.btn-group-lg > .btn {\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n border-radius: 6px;\n}\n.btn-sm,\n.btn-group-sm > .btn {\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\n.btn-xs,\n.btn-group-xs > .btn {\n padding: 1px 5px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\n.btn-block {\n display: block;\n width: 100%;\n}\n.btn-block + .btn-block {\n margin-top: 5px;\n}\ninput[type=\"submit\"].btn-block,\ninput[type=\"reset\"].btn-block,\ninput[type=\"button\"].btn-block {\n width: 100%;\n}\n.fade {\n opacity: 0;\n -webkit-transition: opacity 0.15s linear;\n -o-transition: opacity 0.15s linear;\n transition: opacity 0.15s linear;\n}\n.fade.in {\n opacity: 1;\n}\n.collapse {\n display: none;\n}\n.collapse.in {\n display: block;\n}\ntr.collapse.in {\n display: table-row;\n}\ntbody.collapse.in {\n display: table-row-group;\n}\n.collapsing {\n position: relative;\n height: 0;\n overflow: hidden;\n -webkit-transition-property: height, visibility;\n transition-property: height, visibility;\n -webkit-transition-duration: 0.35s;\n transition-duration: 0.35s;\n -webkit-transition-timing-function: ease;\n transition-timing-function: ease;\n}\n.caret {\n display: inline-block;\n width: 0;\n height: 0;\n margin-left: 2px;\n vertical-align: middle;\n border-top: 4px dashed;\n border-top: 4px solid \\9;\n border-right: 4px solid transparent;\n border-left: 4px solid transparent;\n}\n.dropup,\n.dropdown {\n position: relative;\n}\n.dropdown-toggle:focus {\n outline: 0;\n}\n.dropdown-menu {\n position: absolute;\n top: 100%;\n left: 0;\n z-index: 1000;\n display: none;\n float: left;\n min-width: 160px;\n padding: 5px 0;\n margin: 2px 0 0;\n list-style: none;\n font-size: 14px;\n text-align: left;\n background-color: #ffffff;\n border: 1px solid #cccccc;\n border: 1px solid rgba(0, 0, 0, 0.15);\n border-radius: 4px;\n -webkit-box-shadow: 0 6px 12px rgba(0, 0, 0, 0.175);\n box-shadow: 0 6px 12px rgba(0, 0, 0, 0.175);\n background-clip: padding-box;\n}\n.dropdown-menu.pull-right {\n right: 0;\n left: auto;\n}\n.dropdown-menu .divider {\n height: 1px;\n margin: 9px 0;\n overflow: hidden;\n background-color: #e5e5e5;\n}\n.dropdown-menu > li > a {\n display: block;\n padding: 3px 20px;\n clear: both;\n font-weight: normal;\n line-height: 1.42857143;\n color: #333333;\n white-space: nowrap;\n}\n.dropdown-menu > li > a:hover,\n.dropdown-menu > li > a:focus {\n text-decoration: none;\n color: #262626;\n background-color: #f5f5f5;\n}\n.dropdown-menu > .active > a,\n.dropdown-menu > .active > a:hover,\n.dropdown-menu > .active > a:focus {\n color: #ffffff;\n text-decoration: none;\n outline: 0;\n background-color: #337ab7;\n}\n.dropdown-menu > .disabled > a,\n.dropdown-menu > .disabled > a:hover,\n.dropdown-menu > .disabled > a:focus {\n color: #777777;\n}\n.dropdown-menu > .disabled > a:hover,\n.dropdown-menu > .disabled > a:focus {\n text-decoration: none;\n background-color: transparent;\n background-image: none;\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n cursor: not-allowed;\n}\n.open > .dropdown-menu {\n display: block;\n}\n.open > a {\n outline: 0;\n}\n.dropdown-menu-right {\n left: auto;\n right: 0;\n}\n.dropdown-menu-left {\n left: 0;\n right: auto;\n}\n.dropdown-header {\n display: block;\n padding: 3px 20px;\n font-size: 12px;\n line-height: 1.42857143;\n color: #777777;\n white-space: nowrap;\n}\n.dropdown-backdrop {\n position: fixed;\n left: 0;\n right: 0;\n bottom: 0;\n top: 0;\n z-index: 990;\n}\n.pull-right > .dropdown-menu {\n right: 0;\n left: auto;\n}\n.dropup .caret,\n.navbar-fixed-bottom .dropdown .caret {\n border-top: 0;\n border-bottom: 4px dashed;\n border-bottom: 4px solid \\9;\n content: \"\";\n}\n.dropup .dropdown-menu,\n.navbar-fixed-bottom .dropdown .dropdown-menu {\n top: auto;\n bottom: 100%;\n margin-bottom: 2px;\n}\n@media (min-width: 768px) {\n .navbar-right .dropdown-menu {\n left: auto;\n right: 0;\n }\n .navbar-right .dropdown-menu-left {\n left: 0;\n right: auto;\n }\n}\n.btn-group,\n.btn-group-vertical {\n position: relative;\n display: inline-block;\n vertical-align: middle;\n}\n.btn-group > .btn,\n.btn-group-vertical > .btn {\n position: relative;\n float: left;\n}\n.btn-group > .btn:hover,\n.btn-group-vertical > .btn:hover,\n.btn-group > .btn:focus,\n.btn-group-vertical > .btn:focus,\n.btn-group > .btn:active,\n.btn-group-vertical > .btn:active,\n.btn-group > .btn.active,\n.btn-group-vertical > .btn.active {\n z-index: 2;\n}\n.btn-group .btn + .btn,\n.btn-group .btn + .btn-group,\n.btn-group .btn-group + .btn,\n.btn-group .btn-group + .btn-group {\n margin-left: -1px;\n}\n.btn-toolbar {\n margin-left: -5px;\n}\n.btn-toolbar .btn,\n.btn-toolbar .btn-group,\n.btn-toolbar .input-group {\n float: left;\n}\n.btn-toolbar > .btn,\n.btn-toolbar > .btn-group,\n.btn-toolbar > .input-group {\n margin-left: 5px;\n}\n.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) {\n border-radius: 0;\n}\n.btn-group > .btn:first-child {\n margin-left: 0;\n}\n.btn-group > .btn:first-child:not(:last-child):not(.dropdown-toggle) {\n border-bottom-right-radius: 0;\n border-top-right-radius: 0;\n}\n.btn-group > .btn:last-child:not(:first-child),\n.btn-group > .dropdown-toggle:not(:first-child) {\n border-bottom-left-radius: 0;\n border-top-left-radius: 0;\n}\n.btn-group > .btn-group {\n float: left;\n}\n.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn {\n border-radius: 0;\n}\n.btn-group > .btn-group:first-child:not(:last-child) > .btn:last-child,\n.btn-group > .btn-group:first-child:not(:last-child) > .dropdown-toggle {\n border-bottom-right-radius: 0;\n border-top-right-radius: 0;\n}\n.btn-group > .btn-group:last-child:not(:first-child) > .btn:first-child {\n border-bottom-left-radius: 0;\n border-top-left-radius: 0;\n}\n.btn-group .dropdown-toggle:active,\n.btn-group.open .dropdown-toggle {\n outline: 0;\n}\n.btn-group > .btn + .dropdown-toggle {\n padding-left: 8px;\n padding-right: 8px;\n}\n.btn-group > .btn-lg + .dropdown-toggle {\n padding-left: 12px;\n padding-right: 12px;\n}\n.btn-group.open .dropdown-toggle {\n -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n}\n.btn-group.open .dropdown-toggle.btn-link {\n -webkit-box-shadow: none;\n box-shadow: none;\n}\n.btn .caret {\n margin-left: 0;\n}\n.btn-lg .caret {\n border-width: 5px 5px 0;\n border-bottom-width: 0;\n}\n.dropup .btn-lg .caret {\n border-width: 0 5px 5px;\n}\n.btn-group-vertical > .btn,\n.btn-group-vertical > .btn-group,\n.btn-group-vertical > .btn-group > .btn {\n display: block;\n float: none;\n width: 100%;\n max-width: 100%;\n}\n.btn-group-vertical > .btn-group > .btn {\n float: none;\n}\n.btn-group-vertical > .btn + .btn,\n.btn-group-vertical > .btn + .btn-group,\n.btn-group-vertical > .btn-group + .btn,\n.btn-group-vertical > .btn-group + .btn-group {\n margin-top: -1px;\n margin-left: 0;\n}\n.btn-group-vertical > .btn:not(:first-child):not(:last-child) {\n border-radius: 0;\n}\n.btn-group-vertical > .btn:first-child:not(:last-child) {\n border-top-right-radius: 4px;\n border-bottom-right-radius: 0;\n border-bottom-left-radius: 0;\n}\n.btn-group-vertical > .btn:last-child:not(:first-child) {\n border-bottom-left-radius: 4px;\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn {\n border-radius: 0;\n}\n.btn-group-vertical > .btn-group:first-child:not(:last-child) > .btn:last-child,\n.btn-group-vertical > .btn-group:first-child:not(:last-child) > .dropdown-toggle {\n border-bottom-right-radius: 0;\n border-bottom-left-radius: 0;\n}\n.btn-group-vertical > .btn-group:last-child:not(:first-child) > .btn:first-child {\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.btn-group-justified {\n display: table;\n width: 100%;\n table-layout: fixed;\n border-collapse: separate;\n}\n.btn-group-justified > .btn,\n.btn-group-justified > .btn-group {\n float: none;\n display: table-cell;\n width: 1%;\n}\n.btn-group-justified > .btn-group .btn {\n width: 100%;\n}\n.btn-group-justified > .btn-group .dropdown-menu {\n left: auto;\n}\n[data-toggle=\"buttons\"] > .btn input[type=\"radio\"],\n[data-toggle=\"buttons\"] > .btn-group > .btn input[type=\"radio\"],\n[data-toggle=\"buttons\"] > .btn input[type=\"checkbox\"],\n[data-toggle=\"buttons\"] > .btn-group > .btn input[type=\"checkbox\"] {\n position: absolute;\n clip: rect(0, 0, 0, 0);\n pointer-events: none;\n}\n.input-group {\n position: relative;\n display: table;\n border-collapse: separate;\n}\n.input-group[class*=\"col-\"] {\n float: none;\n padding-left: 0;\n padding-right: 0;\n}\n.input-group .form-control {\n position: relative;\n z-index: 2;\n float: left;\n width: 100%;\n margin-bottom: 0;\n}\n.input-group-lg > .form-control,\n.input-group-lg > .input-group-addon,\n.input-group-lg > .input-group-btn > .btn {\n height: 46px;\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n border-radius: 6px;\n}\nselect.input-group-lg > .form-control,\nselect.input-group-lg > .input-group-addon,\nselect.input-group-lg > .input-group-btn > .btn {\n height: 46px;\n line-height: 46px;\n}\ntextarea.input-group-lg > .form-control,\ntextarea.input-group-lg > .input-group-addon,\ntextarea.input-group-lg > .input-group-btn > .btn,\nselect[multiple].input-group-lg > .form-control,\nselect[multiple].input-group-lg > .input-group-addon,\nselect[multiple].input-group-lg > .input-group-btn > .btn {\n height: auto;\n}\n.input-group-sm > .form-control,\n.input-group-sm > .input-group-addon,\n.input-group-sm > .input-group-btn > .btn {\n height: 30px;\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\nselect.input-group-sm > .form-control,\nselect.input-group-sm > .input-group-addon,\nselect.input-group-sm > .input-group-btn > .btn {\n height: 30px;\n line-height: 30px;\n}\ntextarea.input-group-sm > .form-control,\ntextarea.input-group-sm > .input-group-addon,\ntextarea.input-group-sm > .input-group-btn > .btn,\nselect[multiple].input-group-sm > .form-control,\nselect[multiple].input-group-sm > .input-group-addon,\nselect[multiple].input-group-sm > .input-group-btn > .btn {\n height: auto;\n}\n.input-group-addon,\n.input-group-btn,\n.input-group .form-control {\n display: table-cell;\n}\n.input-group-addon:not(:first-child):not(:last-child),\n.input-group-btn:not(:first-child):not(:last-child),\n.input-group .form-control:not(:first-child):not(:last-child) {\n border-radius: 0;\n}\n.input-group-addon,\n.input-group-btn {\n width: 1%;\n white-space: nowrap;\n vertical-align: middle;\n}\n.input-group-addon {\n padding: 6px 12px;\n font-size: 14px;\n font-weight: normal;\n line-height: 1;\n color: #555555;\n text-align: center;\n background-color: #eeeeee;\n border: 1px solid #cccccc;\n border-radius: 4px;\n}\n.input-group-addon.input-sm {\n padding: 5px 10px;\n font-size: 12px;\n border-radius: 3px;\n}\n.input-group-addon.input-lg {\n padding: 10px 16px;\n font-size: 18px;\n border-radius: 6px;\n}\n.input-group-addon input[type=\"radio\"],\n.input-group-addon input[type=\"checkbox\"] {\n margin-top: 0;\n}\n.input-group .form-control:first-child,\n.input-group-addon:first-child,\n.input-group-btn:first-child > .btn,\n.input-group-btn:first-child > .btn-group > .btn,\n.input-group-btn:first-child > .dropdown-toggle,\n.input-group-btn:last-child > .btn:not(:last-child):not(.dropdown-toggle),\n.input-group-btn:last-child > .btn-group:not(:last-child) > .btn {\n border-bottom-right-radius: 0;\n border-top-right-radius: 0;\n}\n.input-group-addon:first-child {\n border-right: 0;\n}\n.input-group .form-control:last-child,\n.input-group-addon:last-child,\n.input-group-btn:last-child > .btn,\n.input-group-btn:last-child > .btn-group > .btn,\n.input-group-btn:last-child > .dropdown-toggle,\n.input-group-btn:first-child > .btn:not(:first-child),\n.input-group-btn:first-child > .btn-group:not(:first-child) > .btn {\n border-bottom-left-radius: 0;\n border-top-left-radius: 0;\n}\n.input-group-addon:last-child {\n border-left: 0;\n}\n.input-group-btn {\n position: relative;\n font-size: 0;\n white-space: nowrap;\n}\n.input-group-btn > .btn {\n position: relative;\n}\n.input-group-btn > .btn + .btn {\n margin-left: -1px;\n}\n.input-group-btn > .btn:hover,\n.input-group-btn > .btn:focus,\n.input-group-btn > .btn:active {\n z-index: 2;\n}\n.input-group-btn:first-child > .btn,\n.input-group-btn:first-child > .btn-group {\n margin-right: -1px;\n}\n.input-group-btn:last-child > .btn,\n.input-group-btn:last-child > .btn-group {\n z-index: 2;\n margin-left: -1px;\n}\n.nav {\n margin-bottom: 0;\n padding-left: 0;\n list-style: none;\n}\n.nav > li {\n position: relative;\n display: block;\n}\n.nav > li > a {\n position: relative;\n display: block;\n padding: 10px 15px;\n}\n.nav > li > a:hover,\n.nav > li > a:focus {\n text-decoration: none;\n background-color: #eeeeee;\n}\n.nav > li.disabled > a {\n color: #777777;\n}\n.nav > li.disabled > a:hover,\n.nav > li.disabled > a:focus {\n color: #777777;\n text-decoration: none;\n background-color: transparent;\n cursor: not-allowed;\n}\n.nav .open > a,\n.nav .open > a:hover,\n.nav .open > a:focus {\n background-color: #eeeeee;\n border-color: #337ab7;\n}\n.nav .nav-divider {\n height: 1px;\n margin: 9px 0;\n overflow: hidden;\n background-color: #e5e5e5;\n}\n.nav > li > a > img {\n max-width: none;\n}\n.nav-tabs {\n border-bottom: 1px solid #dddddd;\n}\n.nav-tabs > li {\n float: left;\n margin-bottom: -1px;\n}\n.nav-tabs > li > a {\n margin-right: 2px;\n line-height: 1.42857143;\n border: 1px solid transparent;\n border-radius: 4px 4px 0 0;\n}\n.nav-tabs > li > a:hover {\n border-color: #eeeeee #eeeeee #dddddd;\n}\n.nav-tabs > li.active > a,\n.nav-tabs > li.active > a:hover,\n.nav-tabs > li.active > a:focus {\n color: #555555;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n border-bottom-color: transparent;\n cursor: default;\n}\n.nav-tabs.nav-justified {\n width: 100%;\n border-bottom: 0;\n}\n.nav-tabs.nav-justified > li {\n float: none;\n}\n.nav-tabs.nav-justified > li > a {\n text-align: center;\n margin-bottom: 5px;\n}\n.nav-tabs.nav-justified > .dropdown .dropdown-menu {\n top: auto;\n left: auto;\n}\n@media (min-width: 768px) {\n .nav-tabs.nav-justified > li {\n display: table-cell;\n width: 1%;\n }\n .nav-tabs.nav-justified > li > a {\n margin-bottom: 0;\n }\n}\n.nav-tabs.nav-justified > li > a {\n margin-right: 0;\n border-radius: 4px;\n}\n.nav-tabs.nav-justified > .active > a,\n.nav-tabs.nav-justified > .active > a:hover,\n.nav-tabs.nav-justified > .active > a:focus {\n border: 1px solid #dddddd;\n}\n@media (min-width: 768px) {\n .nav-tabs.nav-justified > li > a {\n border-bottom: 1px solid #dddddd;\n border-radius: 4px 4px 0 0;\n }\n .nav-tabs.nav-justified > .active > a,\n .nav-tabs.nav-justified > .active > a:hover,\n .nav-tabs.nav-justified > .active > a:focus {\n border-bottom-color: #ffffff;\n }\n}\n.nav-pills > li {\n float: left;\n}\n.nav-pills > li > a {\n border-radius: 4px;\n}\n.nav-pills > li + li {\n margin-left: 2px;\n}\n.nav-pills > li.active > a,\n.nav-pills > li.active > a:hover,\n.nav-pills > li.active > a:focus {\n color: #ffffff;\n background-color: #337ab7;\n}\n.nav-stacked > li {\n float: none;\n}\n.nav-stacked > li + li {\n margin-top: 2px;\n margin-left: 0;\n}\n.nav-justified {\n width: 100%;\n}\n.nav-justified > li {\n float: none;\n}\n.nav-justified > li > a {\n text-align: center;\n margin-bottom: 5px;\n}\n.nav-justified > .dropdown .dropdown-menu {\n top: auto;\n left: auto;\n}\n@media (min-width: 768px) {\n .nav-justified > li {\n display: table-cell;\n width: 1%;\n }\n .nav-justified > li > a {\n margin-bottom: 0;\n }\n}\n.nav-tabs-justified {\n border-bottom: 0;\n}\n.nav-tabs-justified > li > a {\n margin-right: 0;\n border-radius: 4px;\n}\n.nav-tabs-justified > .active > a,\n.nav-tabs-justified > .active > a:hover,\n.nav-tabs-justified > .active > a:focus {\n border: 1px solid #dddddd;\n}\n@media (min-width: 768px) {\n .nav-tabs-justified > li > a {\n border-bottom: 1px solid #dddddd;\n border-radius: 4px 4px 0 0;\n }\n .nav-tabs-justified > .active > a,\n .nav-tabs-justified > .active > a:hover,\n .nav-tabs-justified > .active > a:focus {\n border-bottom-color: #ffffff;\n }\n}\n.tab-content > .tab-pane {\n display: none;\n}\n.tab-content > .active {\n display: block;\n}\n.nav-tabs .dropdown-menu {\n margin-top: -1px;\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.navbar {\n position: relative;\n min-height: 50px;\n margin-bottom: 20px;\n border: 1px solid transparent;\n}\n@media (min-width: 768px) {\n .navbar {\n border-radius: 4px;\n }\n}\n@media (min-width: 768px) {\n .navbar-header {\n float: left;\n }\n}\n.navbar-collapse {\n overflow-x: visible;\n padding-right: 15px;\n padding-left: 15px;\n border-top: 1px solid transparent;\n box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1);\n -webkit-overflow-scrolling: touch;\n}\n.navbar-collapse.in {\n overflow-y: auto;\n}\n@media (min-width: 768px) {\n .navbar-collapse {\n width: auto;\n border-top: 0;\n box-shadow: none;\n }\n .navbar-collapse.collapse {\n display: block !important;\n height: auto !important;\n padding-bottom: 0;\n overflow: visible !important;\n }\n .navbar-collapse.in {\n overflow-y: visible;\n }\n .navbar-fixed-top .navbar-collapse,\n .navbar-static-top .navbar-collapse,\n .navbar-fixed-bottom .navbar-collapse {\n padding-left: 0;\n padding-right: 0;\n }\n}\n.navbar-fixed-top .navbar-collapse,\n.navbar-fixed-bottom .navbar-collapse {\n max-height: 340px;\n}\n@media (max-device-width: 480px) and (orientation: landscape) {\n .navbar-fixed-top .navbar-collapse,\n .navbar-fixed-bottom .navbar-collapse {\n max-height: 200px;\n }\n}\n.container > .navbar-header,\n.container-fluid > .navbar-header,\n.container > .navbar-collapse,\n.container-fluid > .navbar-collapse {\n margin-right: -15px;\n margin-left: -15px;\n}\n@media (min-width: 768px) {\n .container > .navbar-header,\n .container-fluid > .navbar-header,\n .container > .navbar-collapse,\n .container-fluid > .navbar-collapse {\n margin-right: 0;\n margin-left: 0;\n }\n}\n.navbar-static-top {\n z-index: 1000;\n border-width: 0 0 1px;\n}\n@media (min-width: 768px) {\n .navbar-static-top {\n border-radius: 0;\n }\n}\n.navbar-fixed-top,\n.navbar-fixed-bottom {\n position: fixed;\n right: 0;\n left: 0;\n z-index: 1030;\n}\n@media (min-width: 768px) {\n .navbar-fixed-top,\n .navbar-fixed-bottom {\n border-radius: 0;\n }\n}\n.navbar-fixed-top {\n top: 0;\n border-width: 0 0 1px;\n}\n.navbar-fixed-bottom {\n bottom: 0;\n margin-bottom: 0;\n border-width: 1px 0 0;\n}\n.navbar-brand {\n float: left;\n padding: 15px 15px;\n font-size: 18px;\n line-height: 20px;\n height: 50px;\n}\n.navbar-brand:hover,\n.navbar-brand:focus {\n text-decoration: none;\n}\n.navbar-brand > img {\n display: block;\n}\n@media (min-width: 768px) {\n .navbar > .container .navbar-brand,\n .navbar > .container-fluid .navbar-brand {\n margin-left: -15px;\n }\n}\n.navbar-toggle {\n position: relative;\n float: right;\n margin-right: 15px;\n padding: 9px 10px;\n margin-top: 8px;\n margin-bottom: 8px;\n background-color: transparent;\n background-image: none;\n border: 1px solid transparent;\n border-radius: 4px;\n}\n.navbar-toggle:focus {\n outline: 0;\n}\n.navbar-toggle .icon-bar {\n display: block;\n width: 22px;\n height: 2px;\n border-radius: 1px;\n}\n.navbar-toggle .icon-bar + .icon-bar {\n margin-top: 4px;\n}\n@media (min-width: 768px) {\n .navbar-toggle {\n display: none;\n }\n}\n.navbar-nav {\n margin: 7.5px -15px;\n}\n.navbar-nav > li > a {\n padding-top: 10px;\n padding-bottom: 10px;\n line-height: 20px;\n}\n@media (max-width: 767px) {\n .navbar-nav .open .dropdown-menu {\n position: static;\n float: none;\n width: auto;\n margin-top: 0;\n background-color: transparent;\n border: 0;\n box-shadow: none;\n }\n .navbar-nav .open .dropdown-menu > li > a,\n .navbar-nav .open .dropdown-menu .dropdown-header {\n padding: 5px 15px 5px 25px;\n }\n .navbar-nav .open .dropdown-menu > li > a {\n line-height: 20px;\n }\n .navbar-nav .open .dropdown-menu > li > a:hover,\n .navbar-nav .open .dropdown-menu > li > a:focus {\n background-image: none;\n }\n}\n@media (min-width: 768px) {\n .navbar-nav {\n float: left;\n margin: 0;\n }\n .navbar-nav > li {\n float: left;\n }\n .navbar-nav > li > a {\n padding-top: 15px;\n padding-bottom: 15px;\n }\n}\n.navbar-form {\n margin-left: -15px;\n margin-right: -15px;\n padding: 10px 15px;\n border-top: 1px solid transparent;\n border-bottom: 1px solid transparent;\n -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);\n box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);\n margin-top: 8px;\n margin-bottom: 8px;\n}\n@media (min-width: 768px) {\n .navbar-form .form-group {\n display: inline-block;\n margin-bottom: 0;\n vertical-align: middle;\n }\n .navbar-form .form-control {\n display: inline-block;\n width: auto;\n vertical-align: middle;\n }\n .navbar-form .form-control-static {\n display: inline-block;\n }\n .navbar-form .input-group {\n display: inline-table;\n vertical-align: middle;\n }\n .navbar-form .input-group .input-group-addon,\n .navbar-form .input-group .input-group-btn,\n .navbar-form .input-group .form-control {\n width: auto;\n }\n .navbar-form .input-group > .form-control {\n width: 100%;\n }\n .navbar-form .control-label {\n margin-bottom: 0;\n vertical-align: middle;\n }\n .navbar-form .radio,\n .navbar-form .checkbox {\n display: inline-block;\n margin-top: 0;\n margin-bottom: 0;\n vertical-align: middle;\n }\n .navbar-form .radio label,\n .navbar-form .checkbox label {\n padding-left: 0;\n }\n .navbar-form .radio input[type=\"radio\"],\n .navbar-form .checkbox input[type=\"checkbox\"] {\n position: relative;\n margin-left: 0;\n }\n .navbar-form .has-feedback .form-control-feedback {\n top: 0;\n }\n}\n@media (max-width: 767px) {\n .navbar-form .form-group {\n margin-bottom: 5px;\n }\n .navbar-form .form-group:last-child {\n margin-bottom: 0;\n }\n}\n@media (min-width: 768px) {\n .navbar-form {\n width: auto;\n border: 0;\n margin-left: 0;\n margin-right: 0;\n padding-top: 0;\n padding-bottom: 0;\n -webkit-box-shadow: none;\n box-shadow: none;\n }\n}\n.navbar-nav > li > .dropdown-menu {\n margin-top: 0;\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.navbar-fixed-bottom .navbar-nav > li > .dropdown-menu {\n margin-bottom: 0;\n border-top-right-radius: 4px;\n border-top-left-radius: 4px;\n border-bottom-right-radius: 0;\n border-bottom-left-radius: 0;\n}\n.navbar-btn {\n margin-top: 8px;\n margin-bottom: 8px;\n}\n.navbar-btn.btn-sm {\n margin-top: 10px;\n margin-bottom: 10px;\n}\n.navbar-btn.btn-xs {\n margin-top: 14px;\n margin-bottom: 14px;\n}\n.navbar-text {\n margin-top: 15px;\n margin-bottom: 15px;\n}\n@media (min-width: 768px) {\n .navbar-text {\n float: left;\n margin-left: 15px;\n margin-right: 15px;\n }\n}\n@media (min-width: 768px) {\n .navbar-left {\n float: left !important;\n }\n .navbar-right {\n float: right !important;\n margin-right: -15px;\n }\n .navbar-right ~ .navbar-right {\n margin-right: 0;\n }\n}\n.navbar-default {\n background-color: #f8f8f8;\n border-color: #e7e7e7;\n}\n.navbar-default .navbar-brand {\n color: #777777;\n}\n.navbar-default .navbar-brand:hover,\n.navbar-default .navbar-brand:focus {\n color: #5e5e5e;\n background-color: transparent;\n}\n.navbar-default .navbar-text {\n color: #777777;\n}\n.navbar-default .navbar-nav > li > a {\n color: #777777;\n}\n.navbar-default .navbar-nav > li > a:hover,\n.navbar-default .navbar-nav > li > a:focus {\n color: #333333;\n background-color: transparent;\n}\n.navbar-default .navbar-nav > .active > a,\n.navbar-default .navbar-nav > .active > a:hover,\n.navbar-default .navbar-nav > .active > a:focus {\n color: #555555;\n background-color: #e7e7e7;\n}\n.navbar-default .navbar-nav > .disabled > a,\n.navbar-default .navbar-nav > .disabled > a:hover,\n.navbar-default .navbar-nav > .disabled > a:focus {\n color: #cccccc;\n background-color: transparent;\n}\n.navbar-default .navbar-toggle {\n border-color: #dddddd;\n}\n.navbar-default .navbar-toggle:hover,\n.navbar-default .navbar-toggle:focus {\n background-color: #dddddd;\n}\n.navbar-default .navbar-toggle .icon-bar {\n background-color: #888888;\n}\n.navbar-default .navbar-collapse,\n.navbar-default .navbar-form {\n border-color: #e7e7e7;\n}\n.navbar-default .navbar-nav > .open > a,\n.navbar-default .navbar-nav > .open > a:hover,\n.navbar-default .navbar-nav > .open > a:focus {\n background-color: #e7e7e7;\n color: #555555;\n}\n@media (max-width: 767px) {\n .navbar-default .navbar-nav .open .dropdown-menu > li > a {\n color: #777777;\n }\n .navbar-default .navbar-nav .open .dropdown-menu > li > a:hover,\n .navbar-default .navbar-nav .open .dropdown-menu > li > a:focus {\n color: #333333;\n background-color: transparent;\n }\n .navbar-default .navbar-nav .open .dropdown-menu > .active > a,\n .navbar-default .navbar-nav .open .dropdown-menu > .active > a:hover,\n .navbar-default .navbar-nav .open .dropdown-menu > .active > a:focus {\n color: #555555;\n background-color: #e7e7e7;\n }\n .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a,\n .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:hover,\n .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:focus {\n color: #cccccc;\n background-color: transparent;\n }\n}\n.navbar-default .navbar-link {\n color: #777777;\n}\n.navbar-default .navbar-link:hover {\n color: #333333;\n}\n.navbar-default .btn-link {\n color: #777777;\n}\n.navbar-default .btn-link:hover,\n.navbar-default .btn-link:focus {\n color: #333333;\n}\n.navbar-default .btn-link[disabled]:hover,\nfieldset[disabled] .navbar-default .btn-link:hover,\n.navbar-default .btn-link[disabled]:focus,\nfieldset[disabled] .navbar-default .btn-link:focus {\n color: #cccccc;\n}\n.navbar-inverse {\n background-color: #222222;\n border-color: #080808;\n}\n.navbar-inverse .navbar-brand {\n color: #9d9d9d;\n}\n.navbar-inverse .navbar-brand:hover,\n.navbar-inverse .navbar-brand:focus {\n color: #ffffff;\n background-color: transparent;\n}\n.navbar-inverse .navbar-text {\n color: #9d9d9d;\n}\n.navbar-inverse .navbar-nav > li > a {\n color: #9d9d9d;\n}\n.navbar-inverse .navbar-nav > li > a:hover,\n.navbar-inverse .navbar-nav > li > a:focus {\n color: #ffffff;\n background-color: transparent;\n}\n.navbar-inverse .navbar-nav > .active > a,\n.navbar-inverse .navbar-nav > .active > a:hover,\n.navbar-inverse .navbar-nav > .active > a:focus {\n color: #ffffff;\n background-color: #080808;\n}\n.navbar-inverse .navbar-nav > .disabled > a,\n.navbar-inverse .navbar-nav > .disabled > a:hover,\n.navbar-inverse .navbar-nav > .disabled > a:focus {\n color: #444444;\n background-color: transparent;\n}\n.navbar-inverse .navbar-toggle {\n border-color: #333333;\n}\n.navbar-inverse .navbar-toggle:hover,\n.navbar-inverse .navbar-toggle:focus {\n background-color: #333333;\n}\n.navbar-inverse .navbar-toggle .icon-bar {\n background-color: #ffffff;\n}\n.navbar-inverse .navbar-collapse,\n.navbar-inverse .navbar-form {\n border-color: #101010;\n}\n.navbar-inverse .navbar-nav > .open > a,\n.navbar-inverse .navbar-nav > .open > a:hover,\n.navbar-inverse .navbar-nav > .open > a:focus {\n background-color: #080808;\n color: #ffffff;\n}\n@media (max-width: 767px) {\n .navbar-inverse .navbar-nav .open .dropdown-menu > .dropdown-header {\n border-color: #080808;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu .divider {\n background-color: #080808;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu > li > a {\n color: #9d9d9d;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:hover,\n .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:focus {\n color: #ffffff;\n background-color: transparent;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a,\n .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:hover,\n .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:focus {\n color: #ffffff;\n background-color: #080808;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a,\n .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:hover,\n .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:focus {\n color: #444444;\n background-color: transparent;\n }\n}\n.navbar-inverse .navbar-link {\n color: #9d9d9d;\n}\n.navbar-inverse .navbar-link:hover {\n color: #ffffff;\n}\n.navbar-inverse .btn-link {\n color: #9d9d9d;\n}\n.navbar-inverse .btn-link:hover,\n.navbar-inverse .btn-link:focus {\n color: #ffffff;\n}\n.navbar-inverse .btn-link[disabled]:hover,\nfieldset[disabled] .navbar-inverse .btn-link:hover,\n.navbar-inverse .btn-link[disabled]:focus,\nfieldset[disabled] .navbar-inverse .btn-link:focus {\n color: #444444;\n}\n.breadcrumb {\n padding: 8px 15px;\n margin-bottom: 20px;\n list-style: none;\n background-color: #f5f5f5;\n border-radius: 4px;\n}\n.breadcrumb > li {\n display: inline-block;\n}\n.breadcrumb > li + li:before {\n content: \"/\\00a0\";\n padding: 0 5px;\n color: #cccccc;\n}\n.breadcrumb > .active {\n color: #777777;\n}\n.pagination {\n display: inline-block;\n padding-left: 0;\n margin: 20px 0;\n border-radius: 4px;\n}\n.pagination > li {\n display: inline;\n}\n.pagination > li > a,\n.pagination > li > span {\n position: relative;\n float: left;\n padding: 6px 12px;\n line-height: 1.42857143;\n text-decoration: none;\n color: #337ab7;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n margin-left: -1px;\n}\n.pagination > li:first-child > a,\n.pagination > li:first-child > span {\n margin-left: 0;\n border-bottom-left-radius: 4px;\n border-top-left-radius: 4px;\n}\n.pagination > li:last-child > a,\n.pagination > li:last-child > span {\n border-bottom-right-radius: 4px;\n border-top-right-radius: 4px;\n}\n.pagination > li > a:hover,\n.pagination > li > span:hover,\n.pagination > li > a:focus,\n.pagination > li > span:focus {\n z-index: 3;\n color: #23527c;\n background-color: #eeeeee;\n border-color: #dddddd;\n}\n.pagination > .active > a,\n.pagination > .active > span,\n.pagination > .active > a:hover,\n.pagination > .active > span:hover,\n.pagination > .active > a:focus,\n.pagination > .active > span:focus {\n z-index: 2;\n color: #ffffff;\n background-color: #337ab7;\n border-color: #337ab7;\n cursor: default;\n}\n.pagination > .disabled > span,\n.pagination > .disabled > span:hover,\n.pagination > .disabled > span:focus,\n.pagination > .disabled > a,\n.pagination > .disabled > a:hover,\n.pagination > .disabled > a:focus {\n color: #777777;\n background-color: #ffffff;\n border-color: #dddddd;\n cursor: not-allowed;\n}\n.pagination-lg > li > a,\n.pagination-lg > li > span {\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n}\n.pagination-lg > li:first-child > a,\n.pagination-lg > li:first-child > span {\n border-bottom-left-radius: 6px;\n border-top-left-radius: 6px;\n}\n.pagination-lg > li:last-child > a,\n.pagination-lg > li:last-child > span {\n border-bottom-right-radius: 6px;\n border-top-right-radius: 6px;\n}\n.pagination-sm > li > a,\n.pagination-sm > li > span {\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n}\n.pagination-sm > li:first-child > a,\n.pagination-sm > li:first-child > span {\n border-bottom-left-radius: 3px;\n border-top-left-radius: 3px;\n}\n.pagination-sm > li:last-child > a,\n.pagination-sm > li:last-child > span {\n border-bottom-right-radius: 3px;\n border-top-right-radius: 3px;\n}\n.pager {\n padding-left: 0;\n margin: 20px 0;\n list-style: none;\n text-align: center;\n}\n.pager li {\n display: inline;\n}\n.pager li > a,\n.pager li > span {\n display: inline-block;\n padding: 5px 14px;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n border-radius: 15px;\n}\n.pager li > a:hover,\n.pager li > a:focus {\n text-decoration: none;\n background-color: #eeeeee;\n}\n.pager .next > a,\n.pager .next > span {\n float: right;\n}\n.pager .previous > a,\n.pager .previous > span {\n float: left;\n}\n.pager .disabled > a,\n.pager .disabled > a:hover,\n.pager .disabled > a:focus,\n.pager .disabled > span {\n color: #777777;\n background-color: #ffffff;\n cursor: not-allowed;\n}\n.label {\n display: inline;\n padding: .2em .6em .3em;\n font-size: 75%;\n font-weight: bold;\n line-height: 1;\n color: #ffffff;\n text-align: center;\n white-space: nowrap;\n vertical-align: baseline;\n border-radius: .25em;\n}\na.label:hover,\na.label:focus {\n color: #ffffff;\n text-decoration: none;\n cursor: pointer;\n}\n.label:empty {\n display: none;\n}\n.btn .label {\n position: relative;\n top: -1px;\n}\n.label-default {\n background-color: #777777;\n}\n.label-default[href]:hover,\n.label-default[href]:focus {\n background-color: #5e5e5e;\n}\n.label-primary {\n background-color: #337ab7;\n}\n.label-primary[href]:hover,\n.label-primary[href]:focus {\n background-color: #286090;\n}\n.label-success {\n background-color: #5cb85c;\n}\n.label-success[href]:hover,\n.label-success[href]:focus {\n background-color: #449d44;\n}\n.label-info {\n background-color: #5bc0de;\n}\n.label-info[href]:hover,\n.label-info[href]:focus {\n background-color: #31b0d5;\n}\n.label-warning {\n background-color: #f0ad4e;\n}\n.label-warning[href]:hover,\n.label-warning[href]:focus {\n background-color: #ec971f;\n}\n.label-danger {\n background-color: #d9534f;\n}\n.label-danger[href]:hover,\n.label-danger[href]:focus {\n background-color: #c9302c;\n}\n.badge {\n display: inline-block;\n min-width: 10px;\n padding: 3px 7px;\n font-size: 12px;\n font-weight: bold;\n color: #ffffff;\n line-height: 1;\n vertical-align: middle;\n white-space: nowrap;\n text-align: center;\n background-color: #777777;\n border-radius: 10px;\n}\n.badge:empty {\n display: none;\n}\n.btn .badge {\n position: relative;\n top: -1px;\n}\n.btn-xs .badge,\n.btn-group-xs > .btn .badge {\n top: 0;\n padding: 1px 5px;\n}\na.badge:hover,\na.badge:focus {\n color: #ffffff;\n text-decoration: none;\n cursor: pointer;\n}\n.list-group-item.active > .badge,\n.nav-pills > .active > a > .badge {\n color: #337ab7;\n background-color: #ffffff;\n}\n.list-group-item > .badge {\n float: right;\n}\n.list-group-item > .badge + .badge {\n margin-right: 5px;\n}\n.nav-pills > li > a > .badge {\n margin-left: 3px;\n}\n.jumbotron {\n padding-top: 30px;\n padding-bottom: 30px;\n margin-bottom: 30px;\n color: inherit;\n background-color: #eeeeee;\n}\n.jumbotron h1,\n.jumbotron .h1 {\n color: inherit;\n}\n.jumbotron p {\n margin-bottom: 15px;\n font-size: 21px;\n font-weight: 200;\n}\n.jumbotron > hr {\n border-top-color: #d5d5d5;\n}\n.container .jumbotron,\n.container-fluid .jumbotron {\n border-radius: 6px;\n}\n.jumbotron .container {\n max-width: 100%;\n}\n@media screen and (min-width: 768px) {\n .jumbotron {\n padding-top: 48px;\n padding-bottom: 48px;\n }\n .container .jumbotron,\n .container-fluid .jumbotron {\n padding-left: 60px;\n padding-right: 60px;\n }\n .jumbotron h1,\n .jumbotron .h1 {\n font-size: 63px;\n }\n}\n.thumbnail {\n display: block;\n padding: 4px;\n margin-bottom: 20px;\n line-height: 1.42857143;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n border-radius: 4px;\n -webkit-transition: border 0.2s ease-in-out;\n -o-transition: border 0.2s ease-in-out;\n transition: border 0.2s ease-in-out;\n}\n.thumbnail > img,\n.thumbnail a > img {\n margin-left: auto;\n margin-right: auto;\n}\na.thumbnail:hover,\na.thumbnail:focus,\na.thumbnail.active {\n border-color: #337ab7;\n}\n.thumbnail .caption {\n padding: 9px;\n color: #333333;\n}\n.alert {\n padding: 15px;\n margin-bottom: 20px;\n border: 1px solid transparent;\n border-radius: 4px;\n}\n.alert h4 {\n margin-top: 0;\n color: inherit;\n}\n.alert .alert-link {\n font-weight: bold;\n}\n.alert > p,\n.alert > ul {\n margin-bottom: 0;\n}\n.alert > p + p {\n margin-top: 5px;\n}\n.alert-dismissable,\n.alert-dismissible {\n padding-right: 35px;\n}\n.alert-dismissable .close,\n.alert-dismissible .close {\n position: relative;\n top: -2px;\n right: -21px;\n color: inherit;\n}\n.alert-success {\n background-color: #dff0d8;\n border-color: #d6e9c6;\n color: #3c763d;\n}\n.alert-success hr {\n border-top-color: #c9e2b3;\n}\n.alert-success .alert-link {\n color: #2b542c;\n}\n.alert-info {\n background-color: #d9edf7;\n border-color: #bce8f1;\n color: #31708f;\n}\n.alert-info hr {\n border-top-color: #a6e1ec;\n}\n.alert-info .alert-link {\n color: #245269;\n}\n.alert-warning {\n background-color: #fcf8e3;\n border-color: #faebcc;\n color: #8a6d3b;\n}\n.alert-warning hr {\n border-top-color: #f7e1b5;\n}\n.alert-warning .alert-link {\n color: #66512c;\n}\n.alert-danger {\n background-color: #f2dede;\n border-color: #ebccd1;\n color: #a94442;\n}\n.alert-danger hr {\n border-top-color: #e4b9c0;\n}\n.alert-danger .alert-link {\n color: #843534;\n}\n@-webkit-keyframes progress-bar-stripes {\n from {\n background-position: 40px 0;\n }\n to {\n background-position: 0 0;\n }\n}\n@keyframes progress-bar-stripes {\n from {\n background-position: 40px 0;\n }\n to {\n background-position: 0 0;\n }\n}\n.progress {\n overflow: hidden;\n height: 20px;\n margin-bottom: 20px;\n background-color: #f5f5f5;\n border-radius: 4px;\n -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1);\n box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1);\n}\n.progress-bar {\n float: left;\n width: 0%;\n height: 100%;\n font-size: 12px;\n line-height: 20px;\n color: #ffffff;\n text-align: center;\n background-color: #337ab7;\n -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15);\n box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15);\n -webkit-transition: width 0.6s ease;\n -o-transition: width 0.6s ease;\n transition: width 0.6s ease;\n}\n.progress-striped .progress-bar,\n.progress-bar-striped {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-size: 40px 40px;\n}\n.progress.active .progress-bar,\n.progress-bar.active {\n -webkit-animation: progress-bar-stripes 2s linear infinite;\n -o-animation: progress-bar-stripes 2s linear infinite;\n animation: progress-bar-stripes 2s linear infinite;\n}\n.progress-bar-success {\n background-color: #5cb85c;\n}\n.progress-striped .progress-bar-success {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.progress-bar-info {\n background-color: #5bc0de;\n}\n.progress-striped .progress-bar-info {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.progress-bar-warning {\n background-color: #f0ad4e;\n}\n.progress-striped .progress-bar-warning {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.progress-bar-danger {\n background-color: #d9534f;\n}\n.progress-striped .progress-bar-danger {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.media {\n margin-top: 15px;\n}\n.media:first-child {\n margin-top: 0;\n}\n.media,\n.media-body {\n zoom: 1;\n overflow: hidden;\n}\n.media-body {\n width: 10000px;\n}\n.media-object {\n display: block;\n}\n.media-object.img-thumbnail {\n max-width: none;\n}\n.media-right,\n.media > .pull-right {\n padding-left: 10px;\n}\n.media-left,\n.media > .pull-left {\n padding-right: 10px;\n}\n.media-left,\n.media-right,\n.media-body {\n display: table-cell;\n vertical-align: top;\n}\n.media-middle {\n vertical-align: middle;\n}\n.media-bottom {\n vertical-align: bottom;\n}\n.media-heading {\n margin-top: 0;\n margin-bottom: 5px;\n}\n.media-list {\n padding-left: 0;\n list-style: none;\n}\n.list-group {\n margin-bottom: 20px;\n padding-left: 0;\n}\n.list-group-item {\n position: relative;\n display: block;\n padding: 10px 15px;\n margin-bottom: -1px;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n}\n.list-group-item:first-child {\n border-top-right-radius: 4px;\n border-top-left-radius: 4px;\n}\n.list-group-item:last-child {\n margin-bottom: 0;\n border-bottom-right-radius: 4px;\n border-bottom-left-radius: 4px;\n}\na.list-group-item,\nbutton.list-group-item {\n color: #555555;\n}\na.list-group-item .list-group-item-heading,\nbutton.list-group-item .list-group-item-heading {\n color: #333333;\n}\na.list-group-item:hover,\nbutton.list-group-item:hover,\na.list-group-item:focus,\nbutton.list-group-item:focus {\n text-decoration: none;\n color: #555555;\n background-color: #f5f5f5;\n}\nbutton.list-group-item {\n width: 100%;\n text-align: left;\n}\n.list-group-item.disabled,\n.list-group-item.disabled:hover,\n.list-group-item.disabled:focus {\n background-color: #eeeeee;\n color: #777777;\n cursor: not-allowed;\n}\n.list-group-item.disabled .list-group-item-heading,\n.list-group-item.disabled:hover .list-group-item-heading,\n.list-group-item.disabled:focus .list-group-item-heading {\n color: inherit;\n}\n.list-group-item.disabled .list-group-item-text,\n.list-group-item.disabled:hover .list-group-item-text,\n.list-group-item.disabled:focus .list-group-item-text {\n color: #777777;\n}\n.list-group-item.active,\n.list-group-item.active:hover,\n.list-group-item.active:focus {\n z-index: 2;\n color: #ffffff;\n background-color: #337ab7;\n border-color: #337ab7;\n}\n.list-group-item.active .list-group-item-heading,\n.list-group-item.active:hover .list-group-item-heading,\n.list-group-item.active:focus .list-group-item-heading,\n.list-group-item.active .list-group-item-heading > small,\n.list-group-item.active:hover .list-group-item-heading > small,\n.list-group-item.active:focus .list-group-item-heading > small,\n.list-group-item.active .list-group-item-heading > .small,\n.list-group-item.active:hover .list-group-item-heading > .small,\n.list-group-item.active:focus .list-group-item-heading > .small {\n color: inherit;\n}\n.list-group-item.active .list-group-item-text,\n.list-group-item.active:hover .list-group-item-text,\n.list-group-item.active:focus .list-group-item-text {\n color: #c7ddef;\n}\n.list-group-item-success {\n color: #3c763d;\n background-color: #dff0d8;\n}\na.list-group-item-success,\nbutton.list-group-item-success {\n color: #3c763d;\n}\na.list-group-item-success .list-group-item-heading,\nbutton.list-group-item-success .list-group-item-heading {\n color: inherit;\n}\na.list-group-item-success:hover,\nbutton.list-group-item-success:hover,\na.list-group-item-success:focus,\nbutton.list-group-item-success:focus {\n color: #3c763d;\n background-color: #d0e9c6;\n}\na.list-group-item-success.active,\nbutton.list-group-item-success.active,\na.list-group-item-success.active:hover,\nbutton.list-group-item-success.active:hover,\na.list-group-item-success.active:focus,\nbutton.list-group-item-success.active:focus {\n color: #fff;\n background-color: #3c763d;\n border-color: #3c763d;\n}\n.list-group-item-info {\n color: #31708f;\n background-color: #d9edf7;\n}\na.list-group-item-info,\nbutton.list-group-item-info {\n color: #31708f;\n}\na.list-group-item-info .list-group-item-heading,\nbutton.list-group-item-info .list-group-item-heading {\n color: inherit;\n}\na.list-group-item-info:hover,\nbutton.list-group-item-info:hover,\na.list-group-item-info:focus,\nbutton.list-group-item-info:focus {\n color: #31708f;\n background-color: #c4e3f3;\n}\na.list-group-item-info.active,\nbutton.list-group-item-info.active,\na.list-group-item-info.active:hover,\nbutton.list-group-item-info.active:hover,\na.list-group-item-info.active:focus,\nbutton.list-group-item-info.active:focus {\n color: #fff;\n background-color: #31708f;\n border-color: #31708f;\n}\n.list-group-item-warning {\n color: #8a6d3b;\n background-color: #fcf8e3;\n}\na.list-group-item-warning,\nbutton.list-group-item-warning {\n color: #8a6d3b;\n}\na.list-group-item-warning .list-group-item-heading,\nbutton.list-group-item-warning .list-group-item-heading {\n color: inherit;\n}\na.list-group-item-warning:hover,\nbutton.list-group-item-warning:hover,\na.list-group-item-warning:focus,\nbutton.list-group-item-warning:focus {\n color: #8a6d3b;\n background-color: #faf2cc;\n}\na.list-group-item-warning.active,\nbutton.list-group-item-warning.active,\na.list-group-item-warning.active:hover,\nbutton.list-group-item-warning.active:hover,\na.list-group-item-warning.active:focus,\nbutton.list-group-item-warning.active:focus {\n color: #fff;\n background-color: #8a6d3b;\n border-color: #8a6d3b;\n}\n.list-group-item-danger {\n color: #a94442;\n background-color: #f2dede;\n}\na.list-group-item-danger,\nbutton.list-group-item-danger {\n color: #a94442;\n}\na.list-group-item-danger .list-group-item-heading,\nbutton.list-group-item-danger .list-group-item-heading {\n color: inherit;\n}\na.list-group-item-danger:hover,\nbutton.list-group-item-danger:hover,\na.list-group-item-danger:focus,\nbutton.list-group-item-danger:focus {\n color: #a94442;\n background-color: #ebcccc;\n}\na.list-group-item-danger.active,\nbutton.list-group-item-danger.active,\na.list-group-item-danger.active:hover,\nbutton.list-group-item-danger.active:hover,\na.list-group-item-danger.active:focus,\nbutton.list-group-item-danger.active:focus {\n color: #fff;\n background-color: #a94442;\n border-color: #a94442;\n}\n.list-group-item-heading {\n margin-top: 0;\n margin-bottom: 5px;\n}\n.list-group-item-text {\n margin-bottom: 0;\n line-height: 1.3;\n}\n.panel {\n margin-bottom: 20px;\n background-color: #ffffff;\n border: 1px solid transparent;\n border-radius: 4px;\n -webkit-box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);\n box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);\n}\n.panel-body {\n padding: 15px;\n}\n.panel-heading {\n padding: 10px 15px;\n border-bottom: 1px solid transparent;\n border-top-right-radius: 3px;\n border-top-left-radius: 3px;\n}\n.panel-heading > .dropdown .dropdown-toggle {\n color: inherit;\n}\n.panel-title {\n margin-top: 0;\n margin-bottom: 0;\n font-size: 16px;\n color: inherit;\n}\n.panel-title > a,\n.panel-title > small,\n.panel-title > .small,\n.panel-title > small > a,\n.panel-title > .small > a {\n color: inherit;\n}\n.panel-footer {\n padding: 10px 15px;\n background-color: #f5f5f5;\n border-top: 1px solid #dddddd;\n border-bottom-right-radius: 3px;\n border-bottom-left-radius: 3px;\n}\n.panel > .list-group,\n.panel > .panel-collapse > .list-group {\n margin-bottom: 0;\n}\n.panel > .list-group .list-group-item,\n.panel > .panel-collapse > .list-group .list-group-item {\n border-width: 1px 0;\n border-radius: 0;\n}\n.panel > .list-group:first-child .list-group-item:first-child,\n.panel > .panel-collapse > .list-group:first-child .list-group-item:first-child {\n border-top: 0;\n border-top-right-radius: 3px;\n border-top-left-radius: 3px;\n}\n.panel > .list-group:last-child .list-group-item:last-child,\n.panel > .panel-collapse > .list-group:last-child .list-group-item:last-child {\n border-bottom: 0;\n border-bottom-right-radius: 3px;\n border-bottom-left-radius: 3px;\n}\n.panel > .panel-heading + .panel-collapse > .list-group .list-group-item:first-child {\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.panel-heading + .list-group .list-group-item:first-child {\n border-top-width: 0;\n}\n.list-group + .panel-footer {\n border-top-width: 0;\n}\n.panel > .table,\n.panel > .table-responsive > .table,\n.panel > .panel-collapse > .table {\n margin-bottom: 0;\n}\n.panel > .table caption,\n.panel > .table-responsive > .table caption,\n.panel > .panel-collapse > .table caption {\n padding-left: 15px;\n padding-right: 15px;\n}\n.panel > .table:first-child,\n.panel > .table-responsive:first-child > .table:first-child {\n border-top-right-radius: 3px;\n border-top-left-radius: 3px;\n}\n.panel > .table:first-child > thead:first-child > tr:first-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child {\n border-top-left-radius: 3px;\n border-top-right-radius: 3px;\n}\n.panel > .table:first-child > thead:first-child > tr:first-child td:first-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:first-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child td:first-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:first-child,\n.panel > .table:first-child > thead:first-child > tr:first-child th:first-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:first-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child th:first-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:first-child {\n border-top-left-radius: 3px;\n}\n.panel > .table:first-child > thead:first-child > tr:first-child td:last-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:last-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child td:last-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:last-child,\n.panel > .table:first-child > thead:first-child > tr:first-child th:last-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:last-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child th:last-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:last-child {\n border-top-right-radius: 3px;\n}\n.panel > .table:last-child,\n.panel > .table-responsive:last-child > .table:last-child {\n border-bottom-right-radius: 3px;\n border-bottom-left-radius: 3px;\n}\n.panel > .table:last-child > tbody:last-child > tr:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child {\n border-bottom-left-radius: 3px;\n border-bottom-right-radius: 3px;\n}\n.panel > .table:last-child > tbody:last-child > tr:last-child td:first-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:first-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child td:first-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:first-child,\n.panel > .table:last-child > tbody:last-child > tr:last-child th:first-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:first-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child th:first-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:first-child {\n border-bottom-left-radius: 3px;\n}\n.panel > .table:last-child > tbody:last-child > tr:last-child td:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:last-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child td:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:last-child,\n.panel > .table:last-child > tbody:last-child > tr:last-child th:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:last-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child th:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:last-child {\n border-bottom-right-radius: 3px;\n}\n.panel > .panel-body + .table,\n.panel > .panel-body + .table-responsive,\n.panel > .table + .panel-body,\n.panel > .table-responsive + .panel-body {\n border-top: 1px solid #dddddd;\n}\n.panel > .table > tbody:first-child > tr:first-child th,\n.panel > .table > tbody:first-child > tr:first-child td {\n border-top: 0;\n}\n.panel > .table-bordered,\n.panel > .table-responsive > .table-bordered {\n border: 0;\n}\n.panel > .table-bordered > thead > tr > th:first-child,\n.panel > .table-responsive > .table-bordered > thead > tr > th:first-child,\n.panel > .table-bordered > tbody > tr > th:first-child,\n.panel > .table-responsive > .table-bordered > tbody > tr > th:first-child,\n.panel > .table-bordered > tfoot > tr > th:first-child,\n.panel > .table-responsive > .table-bordered > tfoot > tr > th:first-child,\n.panel > .table-bordered > thead > tr > td:first-child,\n.panel > .table-responsive > .table-bordered > thead > tr > td:first-child,\n.panel > .table-bordered > tbody > tr > td:first-child,\n.panel > .table-responsive > .table-bordered > tbody > tr > td:first-child,\n.panel > .table-bordered > tfoot > tr > td:first-child,\n.panel > .table-responsive > .table-bordered > tfoot > tr > td:first-child {\n border-left: 0;\n}\n.panel > .table-bordered > thead > tr > th:last-child,\n.panel > .table-responsive > .table-bordered > thead > tr > th:last-child,\n.panel > .table-bordered > tbody > tr > th:last-child,\n.panel > .table-responsive > .table-bordered > tbody > tr > th:last-child,\n.panel > .table-bordered > tfoot > tr > th:last-child,\n.panel > .table-responsive > .table-bordered > tfoot > tr > th:last-child,\n.panel > .table-bordered > thead > tr > td:last-child,\n.panel > .table-responsive > .table-bordered > thead > tr > td:last-child,\n.panel > .table-bordered > tbody > tr > td:last-child,\n.panel > .table-responsive > .table-bordered > tbody > tr > td:last-child,\n.panel > .table-bordered > tfoot > tr > td:last-child,\n.panel > .table-responsive > .table-bordered > tfoot > tr > td:last-child {\n border-right: 0;\n}\n.panel > .table-bordered > thead > tr:first-child > td,\n.panel > .table-responsive > .table-bordered > thead > tr:first-child > td,\n.panel > .table-bordered > tbody > tr:first-child > td,\n.panel > .table-responsive > .table-bordered > tbody > tr:first-child > td,\n.panel > .table-bordered > thead > tr:first-child > th,\n.panel > .table-responsive > .table-bordered > thead > tr:first-child > th,\n.panel > .table-bordered > tbody > tr:first-child > th,\n.panel > .table-responsive > .table-bordered > tbody > tr:first-child > th {\n border-bottom: 0;\n}\n.panel > .table-bordered > tbody > tr:last-child > td,\n.panel > .table-responsive > .table-bordered > tbody > tr:last-child > td,\n.panel > .table-bordered > tfoot > tr:last-child > td,\n.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > td,\n.panel > .table-bordered > tbody > tr:last-child > th,\n.panel > .table-responsive > .table-bordered > tbody > tr:last-child > th,\n.panel > .table-bordered > tfoot > tr:last-child > th,\n.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > th {\n border-bottom: 0;\n}\n.panel > .table-responsive {\n border: 0;\n margin-bottom: 0;\n}\n.panel-group {\n margin-bottom: 20px;\n}\n.panel-group .panel {\n margin-bottom: 0;\n border-radius: 4px;\n}\n.panel-group .panel + .panel {\n margin-top: 5px;\n}\n.panel-group .panel-heading {\n border-bottom: 0;\n}\n.panel-group .panel-heading + .panel-collapse > .panel-body,\n.panel-group .panel-heading + .panel-collapse > .list-group {\n border-top: 1px solid #dddddd;\n}\n.panel-group .panel-footer {\n border-top: 0;\n}\n.panel-group .panel-footer + .panel-collapse .panel-body {\n border-bottom: 1px solid #dddddd;\n}\n.panel-default {\n border-color: #dddddd;\n}\n.panel-default > .panel-heading {\n color: #333333;\n background-color: #f5f5f5;\n border-color: #dddddd;\n}\n.panel-default > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #dddddd;\n}\n.panel-default > .panel-heading .badge {\n color: #f5f5f5;\n background-color: #333333;\n}\n.panel-default > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #dddddd;\n}\n.panel-primary {\n border-color: #337ab7;\n}\n.panel-primary > .panel-heading {\n color: #ffffff;\n background-color: #337ab7;\n border-color: #337ab7;\n}\n.panel-primary > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #337ab7;\n}\n.panel-primary > .panel-heading .badge {\n color: #337ab7;\n background-color: #ffffff;\n}\n.panel-primary > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #337ab7;\n}\n.panel-success {\n border-color: #d6e9c6;\n}\n.panel-success > .panel-heading {\n color: #3c763d;\n background-color: #dff0d8;\n border-color: #d6e9c6;\n}\n.panel-success > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #d6e9c6;\n}\n.panel-success > .panel-heading .badge {\n color: #dff0d8;\n background-color: #3c763d;\n}\n.panel-success > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #d6e9c6;\n}\n.panel-info {\n border-color: #bce8f1;\n}\n.panel-info > .panel-heading {\n color: #31708f;\n background-color: #d9edf7;\n border-color: #bce8f1;\n}\n.panel-info > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #bce8f1;\n}\n.panel-info > .panel-heading .badge {\n color: #d9edf7;\n background-color: #31708f;\n}\n.panel-info > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #bce8f1;\n}\n.panel-warning {\n border-color: #faebcc;\n}\n.panel-warning > .panel-heading {\n color: #8a6d3b;\n background-color: #fcf8e3;\n border-color: #faebcc;\n}\n.panel-warning > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #faebcc;\n}\n.panel-warning > .panel-heading .badge {\n color: #fcf8e3;\n background-color: #8a6d3b;\n}\n.panel-warning > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #faebcc;\n}\n.panel-danger {\n border-color: #ebccd1;\n}\n.panel-danger > .panel-heading {\n color: #a94442;\n background-color: #f2dede;\n border-color: #ebccd1;\n}\n.panel-danger > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #ebccd1;\n}\n.panel-danger > .panel-heading .badge {\n color: #f2dede;\n background-color: #a94442;\n}\n.panel-danger > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #ebccd1;\n}\n.embed-responsive {\n position: relative;\n display: block;\n height: 0;\n padding: 0;\n overflow: hidden;\n}\n.embed-responsive .embed-responsive-item,\n.embed-responsive iframe,\n.embed-responsive embed,\n.embed-responsive object,\n.embed-responsive video {\n position: absolute;\n top: 0;\n left: 0;\n bottom: 0;\n height: 100%;\n width: 100%;\n border: 0;\n}\n.embed-responsive-16by9 {\n padding-bottom: 56.25%;\n}\n.embed-responsive-4by3 {\n padding-bottom: 75%;\n}\n.well {\n min-height: 20px;\n padding: 19px;\n margin-bottom: 20px;\n background-color: #f5f5f5;\n border: 1px solid #e3e3e3;\n border-radius: 4px;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05);\n}\n.well blockquote {\n border-color: #ddd;\n border-color: rgba(0, 0, 0, 0.15);\n}\n.well-lg {\n padding: 24px;\n border-radius: 6px;\n}\n.well-sm {\n padding: 9px;\n border-radius: 3px;\n}\n.close {\n float: right;\n font-size: 21px;\n font-weight: bold;\n line-height: 1;\n color: #000000;\n text-shadow: 0 1px 0 #ffffff;\n opacity: 0.2;\n filter: alpha(opacity=20);\n}\n.close:hover,\n.close:focus {\n color: #000000;\n text-decoration: none;\n cursor: pointer;\n opacity: 0.5;\n filter: alpha(opacity=50);\n}\nbutton.close {\n padding: 0;\n cursor: pointer;\n background: transparent;\n border: 0;\n -webkit-appearance: none;\n}\n.modal-open {\n overflow: hidden;\n}\n.modal {\n display: none;\n overflow: hidden;\n position: fixed;\n top: 0;\n right: 0;\n bottom: 0;\n left: 0;\n z-index: 1050;\n -webkit-overflow-scrolling: touch;\n outline: 0;\n}\n.modal.fade .modal-dialog {\n -webkit-transform: translate(0, -25%);\n -ms-transform: translate(0, -25%);\n -o-transform: translate(0, -25%);\n transform: translate(0, -25%);\n -webkit-transition: -webkit-transform 0.3s ease-out;\n -moz-transition: -moz-transform 0.3s ease-out;\n -o-transition: -o-transform 0.3s ease-out;\n transition: transform 0.3s ease-out;\n}\n.modal.in .modal-dialog {\n -webkit-transform: translate(0, 0);\n -ms-transform: translate(0, 0);\n -o-transform: translate(0, 0);\n transform: translate(0, 0);\n}\n.modal-open .modal {\n overflow-x: hidden;\n overflow-y: auto;\n}\n.modal-dialog {\n position: relative;\n width: auto;\n margin: 10px;\n}\n.modal-content {\n position: relative;\n background-color: #ffffff;\n border: 1px solid #999999;\n border: 1px solid rgba(0, 0, 0, 0.2);\n border-radius: 6px;\n -webkit-box-shadow: 0 3px 9px rgba(0, 0, 0, 0.5);\n box-shadow: 0 3px 9px rgba(0, 0, 0, 0.5);\n background-clip: padding-box;\n outline: 0;\n}\n.modal-backdrop {\n position: fixed;\n top: 0;\n right: 0;\n bottom: 0;\n left: 0;\n z-index: 1040;\n background-color: #000000;\n}\n.modal-backdrop.fade {\n opacity: 0;\n filter: alpha(opacity=0);\n}\n.modal-backdrop.in {\n opacity: 0.5;\n filter: alpha(opacity=50);\n}\n.modal-header {\n padding: 15px;\n border-bottom: 1px solid #e5e5e5;\n min-height: 16.42857143px;\n}\n.modal-header .close {\n margin-top: -2px;\n}\n.modal-title {\n margin: 0;\n line-height: 1.42857143;\n}\n.modal-body {\n position: relative;\n padding: 15px;\n}\n.modal-footer {\n padding: 15px;\n text-align: right;\n border-top: 1px solid #e5e5e5;\n}\n.modal-footer .btn + .btn {\n margin-left: 5px;\n margin-bottom: 0;\n}\n.modal-footer .btn-group .btn + .btn {\n margin-left: -1px;\n}\n.modal-footer .btn-block + .btn-block {\n margin-left: 0;\n}\n.modal-scrollbar-measure {\n position: absolute;\n top: -9999px;\n width: 50px;\n height: 50px;\n overflow: scroll;\n}\n@media (min-width: 768px) {\n .modal-dialog {\n width: 600px;\n margin: 30px auto;\n }\n .modal-content {\n -webkit-box-shadow: 0 5px 15px rgba(0, 0, 0, 0.5);\n box-shadow: 0 5px 15px rgba(0, 0, 0, 0.5);\n }\n .modal-sm {\n width: 300px;\n }\n}\n@media (min-width: 992px) {\n .modal-lg {\n width: 900px;\n }\n}\n.tooltip {\n position: absolute;\n z-index: 1070;\n display: block;\n font-family: \"Helvetica Neue\", Helvetica, Arial, sans-serif;\n font-style: normal;\n font-weight: normal;\n letter-spacing: normal;\n line-break: auto;\n line-height: 1.42857143;\n text-align: left;\n text-align: start;\n text-decoration: none;\n text-shadow: none;\n text-transform: none;\n white-space: normal;\n word-break: normal;\n word-spacing: normal;\n word-wrap: normal;\n font-size: 12px;\n opacity: 0;\n filter: alpha(opacity=0);\n}\n.tooltip.in {\n opacity: 0.9;\n filter: alpha(opacity=90);\n}\n.tooltip.top {\n margin-top: -3px;\n padding: 5px 0;\n}\n.tooltip.right {\n margin-left: 3px;\n padding: 0 5px;\n}\n.tooltip.bottom {\n margin-top: 3px;\n padding: 5px 0;\n}\n.tooltip.left {\n margin-left: -3px;\n padding: 0 5px;\n}\n.tooltip-inner {\n max-width: 200px;\n padding: 3px 8px;\n color: #ffffff;\n text-align: center;\n background-color: #000000;\n border-radius: 4px;\n}\n.tooltip-arrow {\n position: absolute;\n width: 0;\n height: 0;\n border-color: transparent;\n border-style: solid;\n}\n.tooltip.top .tooltip-arrow {\n bottom: 0;\n left: 50%;\n margin-left: -5px;\n border-width: 5px 5px 0;\n border-top-color: #000000;\n}\n.tooltip.top-left .tooltip-arrow {\n bottom: 0;\n right: 5px;\n margin-bottom: -5px;\n border-width: 5px 5px 0;\n border-top-color: #000000;\n}\n.tooltip.top-right .tooltip-arrow {\n bottom: 0;\n left: 5px;\n margin-bottom: -5px;\n border-width: 5px 5px 0;\n border-top-color: #000000;\n}\n.tooltip.right .tooltip-arrow {\n top: 50%;\n left: 0;\n margin-top: -5px;\n border-width: 5px 5px 5px 0;\n border-right-color: #000000;\n}\n.tooltip.left .tooltip-arrow {\n top: 50%;\n right: 0;\n margin-top: -5px;\n border-width: 5px 0 5px 5px;\n border-left-color: #000000;\n}\n.tooltip.bottom .tooltip-arrow {\n top: 0;\n left: 50%;\n margin-left: -5px;\n border-width: 0 5px 5px;\n border-bottom-color: #000000;\n}\n.tooltip.bottom-left .tooltip-arrow {\n top: 0;\n right: 5px;\n margin-top: -5px;\n border-width: 0 5px 5px;\n border-bottom-color: #000000;\n}\n.tooltip.bottom-right .tooltip-arrow {\n top: 0;\n left: 5px;\n margin-top: -5px;\n border-width: 0 5px 5px;\n border-bottom-color: #000000;\n}\n.popover {\n position: absolute;\n top: 0;\n left: 0;\n z-index: 1060;\n display: none;\n max-width: 276px;\n padding: 1px;\n font-family: \"Helvetica Neue\", Helvetica, Arial, sans-serif;\n font-style: normal;\n font-weight: normal;\n letter-spacing: normal;\n line-break: auto;\n line-height: 1.42857143;\n text-align: left;\n text-align: start;\n text-decoration: none;\n text-shadow: none;\n text-transform: none;\n white-space: normal;\n word-break: normal;\n word-spacing: normal;\n word-wrap: normal;\n font-size: 14px;\n background-color: #ffffff;\n background-clip: padding-box;\n border: 1px solid #cccccc;\n border: 1px solid rgba(0, 0, 0, 0.2);\n border-radius: 6px;\n -webkit-box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2);\n box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2);\n}\n.popover.top {\n margin-top: -10px;\n}\n.popover.right {\n margin-left: 10px;\n}\n.popover.bottom {\n margin-top: 10px;\n}\n.popover.left {\n margin-left: -10px;\n}\n.popover-title {\n margin: 0;\n padding: 8px 14px;\n font-size: 14px;\n background-color: #f7f7f7;\n border-bottom: 1px solid #ebebeb;\n border-radius: 5px 5px 0 0;\n}\n.popover-content {\n padding: 9px 14px;\n}\n.popover > .arrow,\n.popover > .arrow:after {\n position: absolute;\n display: block;\n width: 0;\n height: 0;\n border-color: transparent;\n border-style: solid;\n}\n.popover > .arrow {\n border-width: 11px;\n}\n.popover > .arrow:after {\n border-width: 10px;\n content: \"\";\n}\n.popover.top > .arrow {\n left: 50%;\n margin-left: -11px;\n border-bottom-width: 0;\n border-top-color: #999999;\n border-top-color: rgba(0, 0, 0, 0.25);\n bottom: -11px;\n}\n.popover.top > .arrow:after {\n content: \" \";\n bottom: 1px;\n margin-left: -10px;\n border-bottom-width: 0;\n border-top-color: #ffffff;\n}\n.popover.right > .arrow {\n top: 50%;\n left: -11px;\n margin-top: -11px;\n border-left-width: 0;\n border-right-color: #999999;\n border-right-color: rgba(0, 0, 0, 0.25);\n}\n.popover.right > .arrow:after {\n content: \" \";\n left: 1px;\n bottom: -10px;\n border-left-width: 0;\n border-right-color: #ffffff;\n}\n.popover.bottom > .arrow {\n left: 50%;\n margin-left: -11px;\n border-top-width: 0;\n border-bottom-color: #999999;\n border-bottom-color: rgba(0, 0, 0, 0.25);\n top: -11px;\n}\n.popover.bottom > .arrow:after {\n content: \" \";\n top: 1px;\n margin-left: -10px;\n border-top-width: 0;\n border-bottom-color: #ffffff;\n}\n.popover.left > .arrow {\n top: 50%;\n right: -11px;\n margin-top: -11px;\n border-right-width: 0;\n border-left-color: #999999;\n border-left-color: rgba(0, 0, 0, 0.25);\n}\n.popover.left > .arrow:after {\n content: \" \";\n right: 1px;\n border-right-width: 0;\n border-left-color: #ffffff;\n bottom: -10px;\n}\n.carousel {\n position: relative;\n}\n.carousel-inner {\n position: relative;\n overflow: hidden;\n width: 100%;\n}\n.carousel-inner > .item {\n display: none;\n position: relative;\n -webkit-transition: 0.6s ease-in-out left;\n -o-transition: 0.6s ease-in-out left;\n transition: 0.6s ease-in-out left;\n}\n.carousel-inner > .item > img,\n.carousel-inner > .item > a > img {\n line-height: 1;\n}\n@media all and (transform-3d), (-webkit-transform-3d) {\n .carousel-inner > .item {\n -webkit-transition: -webkit-transform 0.6s ease-in-out;\n -moz-transition: -moz-transform 0.6s ease-in-out;\n -o-transition: -o-transform 0.6s ease-in-out;\n transition: transform 0.6s ease-in-out;\n -webkit-backface-visibility: hidden;\n -moz-backface-visibility: hidden;\n backface-visibility: hidden;\n -webkit-perspective: 1000px;\n -moz-perspective: 1000px;\n perspective: 1000px;\n }\n .carousel-inner > .item.next,\n .carousel-inner > .item.active.right {\n -webkit-transform: translate3d(100%, 0, 0);\n transform: translate3d(100%, 0, 0);\n left: 0;\n }\n .carousel-inner > .item.prev,\n .carousel-inner > .item.active.left {\n -webkit-transform: translate3d(-100%, 0, 0);\n transform: translate3d(-100%, 0, 0);\n left: 0;\n }\n .carousel-inner > .item.next.left,\n .carousel-inner > .item.prev.right,\n .carousel-inner > .item.active {\n -webkit-transform: translate3d(0, 0, 0);\n transform: translate3d(0, 0, 0);\n left: 0;\n }\n}\n.carousel-inner > .active,\n.carousel-inner > .next,\n.carousel-inner > .prev {\n display: block;\n}\n.carousel-inner > .active {\n left: 0;\n}\n.carousel-inner > .next,\n.carousel-inner > .prev {\n position: absolute;\n top: 0;\n width: 100%;\n}\n.carousel-inner > .next {\n left: 100%;\n}\n.carousel-inner > .prev {\n left: -100%;\n}\n.carousel-inner > .next.left,\n.carousel-inner > .prev.right {\n left: 0;\n}\n.carousel-inner > .active.left {\n left: -100%;\n}\n.carousel-inner > .active.right {\n left: 100%;\n}\n.carousel-control {\n position: absolute;\n top: 0;\n left: 0;\n bottom: 0;\n width: 15%;\n opacity: 0.5;\n filter: alpha(opacity=50);\n font-size: 20px;\n color: #ffffff;\n text-align: center;\n text-shadow: 0 1px 2px rgba(0, 0, 0, 0.6);\n}\n.carousel-control.left {\n background-image: -webkit-linear-gradient(left, rgba(0, 0, 0, 0.5) 0%, rgba(0, 0, 0, 0.0001) 100%);\n background-image: -o-linear-gradient(left, rgba(0, 0, 0, 0.5) 0%, rgba(0, 0, 0, 0.0001) 100%);\n background-image: linear-gradient(to right, rgba(0, 0, 0, 0.5) 0%, rgba(0, 0, 0, 0.0001) 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#80000000', endColorstr='#00000000', GradientType=1);\n}\n.carousel-control.right {\n left: auto;\n right: 0;\n background-image: -webkit-linear-gradient(left, rgba(0, 0, 0, 0.0001) 0%, rgba(0, 0, 0, 0.5) 100%);\n background-image: -o-linear-gradient(left, rgba(0, 0, 0, 0.0001) 0%, rgba(0, 0, 0, 0.5) 100%);\n background-image: linear-gradient(to right, rgba(0, 0, 0, 0.0001) 0%, rgba(0, 0, 0, 0.5) 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#80000000', GradientType=1);\n}\n.carousel-control:hover,\n.carousel-control:focus {\n outline: 0;\n color: #ffffff;\n text-decoration: none;\n opacity: 0.9;\n filter: alpha(opacity=90);\n}\n.carousel-control .icon-prev,\n.carousel-control .icon-next,\n.carousel-control .glyphicon-chevron-left,\n.carousel-control .glyphicon-chevron-right {\n position: absolute;\n top: 50%;\n margin-top: -10px;\n z-index: 5;\n display: inline-block;\n}\n.carousel-control .icon-prev,\n.carousel-control .glyphicon-chevron-left {\n left: 50%;\n margin-left: -10px;\n}\n.carousel-control .icon-next,\n.carousel-control .glyphicon-chevron-right {\n right: 50%;\n margin-right: -10px;\n}\n.carousel-control .icon-prev,\n.carousel-control .icon-next {\n width: 20px;\n height: 20px;\n line-height: 1;\n font-family: serif;\n}\n.carousel-control .icon-prev:before {\n content: '\\2039';\n}\n.carousel-control .icon-next:before {\n content: '\\203a';\n}\n.carousel-indicators {\n position: absolute;\n bottom: 10px;\n left: 50%;\n z-index: 15;\n width: 60%;\n margin-left: -30%;\n padding-left: 0;\n list-style: none;\n text-align: center;\n}\n.carousel-indicators li {\n display: inline-block;\n width: 10px;\n height: 10px;\n margin: 1px;\n text-indent: -999px;\n border: 1px solid #ffffff;\n border-radius: 10px;\n cursor: pointer;\n background-color: #000 \\9;\n background-color: rgba(0, 0, 0, 0);\n}\n.carousel-indicators .active {\n margin: 0;\n width: 12px;\n height: 12px;\n background-color: #ffffff;\n}\n.carousel-caption {\n position: absolute;\n left: 15%;\n right: 15%;\n bottom: 20px;\n z-index: 10;\n padding-top: 20px;\n padding-bottom: 20px;\n color: #ffffff;\n text-align: center;\n text-shadow: 0 1px 2px rgba(0, 0, 0, 0.6);\n}\n.carousel-caption .btn {\n text-shadow: none;\n}\n@media screen and (min-width: 768px) {\n .carousel-control .glyphicon-chevron-left,\n .carousel-control .glyphicon-chevron-right,\n .carousel-control .icon-prev,\n .carousel-control .icon-next {\n width: 30px;\n height: 30px;\n margin-top: -15px;\n font-size: 30px;\n }\n .carousel-control .glyphicon-chevron-left,\n .carousel-control .icon-prev {\n margin-left: -15px;\n }\n .carousel-control .glyphicon-chevron-right,\n .carousel-control .icon-next {\n margin-right: -15px;\n }\n .carousel-caption {\n left: 20%;\n right: 20%;\n padding-bottom: 30px;\n }\n .carousel-indicators {\n bottom: 20px;\n }\n}\n.clearfix:before,\n.clearfix:after,\n.dl-horizontal dd:before,\n.dl-horizontal dd:after,\n.container:before,\n.container:after,\n.container-fluid:before,\n.container-fluid:after,\n.row:before,\n.row:after,\n.form-horizontal .form-group:before,\n.form-horizontal .form-group:after,\n.btn-toolbar:before,\n.btn-toolbar:after,\n.btn-group-vertical > .btn-group:before,\n.btn-group-vertical > .btn-group:after,\n.nav:before,\n.nav:after,\n.navbar:before,\n.navbar:after,\n.navbar-header:before,\n.navbar-header:after,\n.navbar-collapse:before,\n.navbar-collapse:after,\n.pager:before,\n.pager:after,\n.panel-body:before,\n.panel-body:after,\n.modal-footer:before,\n.modal-footer:after {\n content: \" \";\n display: table;\n}\n.clearfix:after,\n.dl-horizontal dd:after,\n.container:after,\n.container-fluid:after,\n.row:after,\n.form-horizontal .form-group:after,\n.btn-toolbar:after,\n.btn-group-vertical > .btn-group:after,\n.nav:after,\n.navbar:after,\n.navbar-header:after,\n.navbar-collapse:after,\n.pager:after,\n.panel-body:after,\n.modal-footer:after {\n clear: both;\n}\n.center-block {\n display: block;\n margin-left: auto;\n margin-right: auto;\n}\n.pull-right {\n float: right !important;\n}\n.pull-left {\n float: left !important;\n}\n.hide {\n display: none !important;\n}\n.show {\n display: block !important;\n}\n.invisible {\n visibility: hidden;\n}\n.text-hide {\n font: 0/0 a;\n color: transparent;\n text-shadow: none;\n background-color: transparent;\n border: 0;\n}\n.hidden {\n display: none !important;\n}\n.affix {\n position: fixed;\n}\n@-ms-viewport {\n width: device-width;\n}\n.visible-xs,\n.visible-sm,\n.visible-md,\n.visible-lg {\n display: none !important;\n}\n.visible-xs-block,\n.visible-xs-inline,\n.visible-xs-inline-block,\n.visible-sm-block,\n.visible-sm-inline,\n.visible-sm-inline-block,\n.visible-md-block,\n.visible-md-inline,\n.visible-md-inline-block,\n.visible-lg-block,\n.visible-lg-inline,\n.visible-lg-inline-block {\n display: none !important;\n}\n@media (max-width: 767px) {\n .visible-xs {\n display: block !important;\n }\n table.visible-xs {\n display: table !important;\n }\n tr.visible-xs {\n display: table-row !important;\n }\n th.visible-xs,\n td.visible-xs {\n display: table-cell !important;\n }\n}\n@media (max-width: 767px) {\n .visible-xs-block {\n display: block !important;\n }\n}\n@media (max-width: 767px) {\n .visible-xs-inline {\n display: inline !important;\n }\n}\n@media (max-width: 767px) {\n .visible-xs-inline-block {\n display: inline-block !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .visible-sm {\n display: block !important;\n }\n table.visible-sm {\n display: table !important;\n }\n tr.visible-sm {\n display: table-row !important;\n }\n th.visible-sm,\n td.visible-sm {\n display: table-cell !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .visible-sm-block {\n display: block !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .visible-sm-inline {\n display: inline !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .visible-sm-inline-block {\n display: inline-block !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .visible-md {\n display: block !important;\n }\n table.visible-md {\n display: table !important;\n }\n tr.visible-md {\n display: table-row !important;\n }\n th.visible-md,\n td.visible-md {\n display: table-cell !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .visible-md-block {\n display: block !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .visible-md-inline {\n display: inline !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .visible-md-inline-block {\n display: inline-block !important;\n }\n}\n@media (min-width: 1200px) {\n .visible-lg {\n display: block !important;\n }\n table.visible-lg {\n display: table !important;\n }\n tr.visible-lg {\n display: table-row !important;\n }\n th.visible-lg,\n td.visible-lg {\n display: table-cell !important;\n }\n}\n@media (min-width: 1200px) {\n .visible-lg-block {\n display: block !important;\n }\n}\n@media (min-width: 1200px) {\n .visible-lg-inline {\n display: inline !important;\n }\n}\n@media (min-width: 1200px) {\n .visible-lg-inline-block {\n display: inline-block !important;\n }\n}\n@media (max-width: 767px) {\n .hidden-xs {\n display: none !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .hidden-sm {\n display: none !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .hidden-md {\n display: none !important;\n }\n}\n@media (min-width: 1200px) {\n .hidden-lg {\n display: none !important;\n }\n}\n.visible-print {\n display: none !important;\n}\n@media print {\n .visible-print {\n display: block !important;\n }\n table.visible-print {\n display: table !important;\n }\n tr.visible-print {\n display: table-row !important;\n }\n th.visible-print,\n td.visible-print {\n display: table-cell !important;\n }\n}\n.visible-print-block {\n display: none !important;\n}\n@media print {\n .visible-print-block {\n display: block !important;\n }\n}\n.visible-print-inline {\n display: none !important;\n}\n@media print {\n .visible-print-inline {\n display: inline !important;\n }\n}\n.visible-print-inline-block {\n display: none !important;\n}\n@media print {\n .visible-print-inline-block {\n display: inline-block !important;\n }\n}\n@media print {\n .hidden-print {\n display: none !important;\n }\n}\n/*# sourceMappingURL=bootstrap.css.map */","/*! normalize.css v3.0.3 | MIT License | github.com/necolas/normalize.css */\n\n//\n// 1. Set default font family to sans-serif.\n// 2. Prevent iOS and IE text size adjust after device orientation change,\n// without disabling user zoom.\n//\n\nhtml {\n font-family: sans-serif; // 1\n -ms-text-size-adjust: 100%; // 2\n -webkit-text-size-adjust: 100%; // 2\n}\n\n//\n// Remove default margin.\n//\n\nbody {\n margin: 0;\n}\n\n// HTML5 display definitions\n// ==========================================================================\n\n//\n// Correct `block` display not defined for any HTML5 element in IE 8/9.\n// Correct `block` display not defined for `details` or `summary` in IE 10/11\n// and Firefox.\n// Correct `block` display not defined for `main` in IE 11.\n//\n\narticle,\naside,\ndetails,\nfigcaption,\nfigure,\nfooter,\nheader,\nhgroup,\nmain,\nmenu,\nnav,\nsection,\nsummary {\n display: block;\n}\n\n//\n// 1. Correct `inline-block` display not defined in IE 8/9.\n// 2. Normalize vertical alignment of `progress` in Chrome, Firefox, and Opera.\n//\n\naudio,\ncanvas,\nprogress,\nvideo {\n display: inline-block; // 1\n vertical-align: baseline; // 2\n}\n\n//\n// Prevent modern browsers from displaying `audio` without controls.\n// Remove excess height in iOS 5 devices.\n//\n\naudio:not([controls]) {\n display: none;\n height: 0;\n}\n\n//\n// Address `[hidden]` styling not present in IE 8/9/10.\n// Hide the `template` element in IE 8/9/10/11, Safari, and Firefox < 22.\n//\n\n[hidden],\ntemplate {\n display: none;\n}\n\n// Links\n// ==========================================================================\n\n//\n// Remove the gray background color from active links in IE 10.\n//\n\na {\n background-color: transparent;\n}\n\n//\n// Improve readability of focused elements when they are also in an\n// active/hover state.\n//\n\na:active,\na:hover {\n outline: 0;\n}\n\n// Text-level semantics\n// ==========================================================================\n\n//\n// Address styling not present in IE 8/9/10/11, Safari, and Chrome.\n//\n\nabbr[title] {\n border-bottom: 1px dotted;\n}\n\n//\n// Address style set to `bolder` in Firefox 4+, Safari, and Chrome.\n//\n\nb,\nstrong {\n font-weight: bold;\n}\n\n//\n// Address styling not present in Safari and Chrome.\n//\n\ndfn {\n font-style: italic;\n}\n\n//\n// Address variable `h1` font-size and margin within `section` and `article`\n// contexts in Firefox 4+, Safari, and Chrome.\n//\n\nh1 {\n font-size: 2em;\n margin: 0.67em 0;\n}\n\n//\n// Address styling not present in IE 8/9.\n//\n\nmark {\n background: #ff0;\n color: #000;\n}\n\n//\n// Address inconsistent and variable font size in all browsers.\n//\n\nsmall {\n font-size: 80%;\n}\n\n//\n// Prevent `sub` and `sup` affecting `line-height` in all browsers.\n//\n\nsub,\nsup {\n font-size: 75%;\n line-height: 0;\n position: relative;\n vertical-align: baseline;\n}\n\nsup {\n top: -0.5em;\n}\n\nsub {\n bottom: -0.25em;\n}\n\n// Embedded content\n// ==========================================================================\n\n//\n// Remove border when inside `a` element in IE 8/9/10.\n//\n\nimg {\n border: 0;\n}\n\n//\n// Correct overflow not hidden in IE 9/10/11.\n//\n\nsvg:not(:root) {\n overflow: hidden;\n}\n\n// Grouping content\n// ==========================================================================\n\n//\n// Address margin not present in IE 8/9 and Safari.\n//\n\nfigure {\n margin: 1em 40px;\n}\n\n//\n// Address differences between Firefox and other browsers.\n//\n\nhr {\n box-sizing: content-box;\n height: 0;\n}\n\n//\n// Contain overflow in all browsers.\n//\n\npre {\n overflow: auto;\n}\n\n//\n// Address odd `em`-unit font size rendering in all browsers.\n//\n\ncode,\nkbd,\npre,\nsamp {\n font-family: monospace, monospace;\n font-size: 1em;\n}\n\n// Forms\n// ==========================================================================\n\n//\n// Known limitation: by default, Chrome and Safari on OS X allow very limited\n// styling of `select`, unless a `border` property is set.\n//\n\n//\n// 1. Correct color not being inherited.\n// Known issue: affects color of disabled elements.\n// 2. Correct font properties not being inherited.\n// 3. Address margins set differently in Firefox 4+, Safari, and Chrome.\n//\n\nbutton,\ninput,\noptgroup,\nselect,\ntextarea {\n color: inherit; // 1\n font: inherit; // 2\n margin: 0; // 3\n}\n\n//\n// Address `overflow` set to `hidden` in IE 8/9/10/11.\n//\n\nbutton {\n overflow: visible;\n}\n\n//\n// Address inconsistent `text-transform` inheritance for `button` and `select`.\n// All other form control elements do not inherit `text-transform` values.\n// Correct `button` style inheritance in Firefox, IE 8/9/10/11, and Opera.\n// Correct `select` style inheritance in Firefox.\n//\n\nbutton,\nselect {\n text-transform: none;\n}\n\n//\n// 1. Avoid the WebKit bug in Android 4.0.* where (2) destroys native `audio`\n// and `video` controls.\n// 2. Correct inability to style clickable `input` types in iOS.\n// 3. Improve usability and consistency of cursor style between image-type\n// `input` and others.\n//\n\nbutton,\nhtml input[type=\"button\"], // 1\ninput[type=\"reset\"],\ninput[type=\"submit\"] {\n -webkit-appearance: button; // 2\n cursor: pointer; // 3\n}\n\n//\n// Re-set default cursor for disabled elements.\n//\n\nbutton[disabled],\nhtml input[disabled] {\n cursor: default;\n}\n\n//\n// Remove inner padding and border in Firefox 4+.\n//\n\nbutton::-moz-focus-inner,\ninput::-moz-focus-inner {\n border: 0;\n padding: 0;\n}\n\n//\n// Address Firefox 4+ setting `line-height` on `input` using `!important` in\n// the UA stylesheet.\n//\n\ninput {\n line-height: normal;\n}\n\n//\n// It's recommended that you don't attempt to style these elements.\n// Firefox's implementation doesn't respect box-sizing, padding, or width.\n//\n// 1. Address box sizing set to `content-box` in IE 8/9/10.\n// 2. Remove excess padding in IE 8/9/10.\n//\n\ninput[type=\"checkbox\"],\ninput[type=\"radio\"] {\n box-sizing: border-box; // 1\n padding: 0; // 2\n}\n\n//\n// Fix the cursor style for Chrome's increment/decrement buttons. For certain\n// `font-size` values of the `input`, it causes the cursor style of the\n// decrement button to change from `default` to `text`.\n//\n\ninput[type=\"number\"]::-webkit-inner-spin-button,\ninput[type=\"number\"]::-webkit-outer-spin-button {\n height: auto;\n}\n\n//\n// 1. Address `appearance` set to `searchfield` in Safari and Chrome.\n// 2. Address `box-sizing` set to `border-box` in Safari and Chrome.\n//\n\ninput[type=\"search\"] {\n -webkit-appearance: textfield; // 1\n box-sizing: content-box; //2\n}\n\n//\n// Remove inner padding and search cancel button in Safari and Chrome on OS X.\n// Safari (but not Chrome) clips the cancel button when the search input has\n// padding (and `textfield` appearance).\n//\n\ninput[type=\"search\"]::-webkit-search-cancel-button,\ninput[type=\"search\"]::-webkit-search-decoration {\n -webkit-appearance: none;\n}\n\n//\n// Define consistent border, margin, and padding.\n//\n\nfieldset {\n border: 1px solid #c0c0c0;\n margin: 0 2px;\n padding: 0.35em 0.625em 0.75em;\n}\n\n//\n// 1. Correct `color` not being inherited in IE 8/9/10/11.\n// 2. Remove padding so people aren't caught out if they zero out fieldsets.\n//\n\nlegend {\n border: 0; // 1\n padding: 0; // 2\n}\n\n//\n// Remove default vertical scrollbar in IE 8/9/10/11.\n//\n\ntextarea {\n overflow: auto;\n}\n\n//\n// Don't inherit the `font-weight` (applied by a rule above).\n// NOTE: the default cannot safely be changed in Chrome and Safari on OS X.\n//\n\noptgroup {\n font-weight: bold;\n}\n\n// Tables\n// ==========================================================================\n\n//\n// Remove most spacing between table cells.\n//\n\ntable {\n border-collapse: collapse;\n border-spacing: 0;\n}\n\ntd,\nth {\n padding: 0;\n}\n","/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */\n\n// ==========================================================================\n// Print styles.\n// Inlined to avoid the additional HTTP request: h5bp.com/r\n// ==========================================================================\n\n@media print {\n *,\n *:before,\n *:after {\n background: transparent !important;\n color: #000 !important; // Black prints faster: h5bp.com/s\n box-shadow: none !important;\n text-shadow: none !important;\n }\n\n a,\n a:visited {\n text-decoration: underline;\n }\n\n a[href]:after {\n content: \" (\" attr(href) \")\";\n }\n\n abbr[title]:after {\n content: \" (\" attr(title) \")\";\n }\n\n // Don't show links that are fragment identifiers,\n // or use the `javascript:` pseudo protocol\n a[href^=\"#\"]:after,\n a[href^=\"javascript:\"]:after {\n content: \"\";\n }\n\n pre,\n blockquote {\n border: 1px solid #999;\n page-break-inside: avoid;\n }\n\n thead {\n display: table-header-group; // h5bp.com/t\n }\n\n tr,\n img {\n page-break-inside: avoid;\n }\n\n img {\n max-width: 100% !important;\n }\n\n p,\n h2,\n h3 {\n orphans: 3;\n widows: 3;\n }\n\n h2,\n h3 {\n page-break-after: avoid;\n }\n\n // Bootstrap specific changes start\n\n // Bootstrap components\n .navbar {\n display: none;\n }\n .btn,\n .dropup > .btn {\n > .caret {\n border-top-color: #000 !important;\n }\n }\n .label {\n border: 1px solid #000;\n }\n\n .table {\n border-collapse: collapse !important;\n\n td,\n th {\n background-color: #fff !important;\n }\n }\n .table-bordered {\n th,\n td {\n border: 1px solid #ddd !important;\n }\n }\n\n // Bootstrap specific changes end\n}\n","//\n// Glyphicons for Bootstrap\n//\n// Since icons are fonts, they can be placed anywhere text is placed and are\n// thus automatically sized to match the surrounding child. To use, create an\n// inline element with the appropriate classes, like so:\n//\n// Star\n\n// Import the fonts\n@font-face {\n font-family: 'Glyphicons Halflings';\n src: url('@{icon-font-path}@{icon-font-name}.eot');\n src: url('@{icon-font-path}@{icon-font-name}.eot?#iefix') format('embedded-opentype'),\n url('@{icon-font-path}@{icon-font-name}.woff2') format('woff2'),\n url('@{icon-font-path}@{icon-font-name}.woff') format('woff'),\n url('@{icon-font-path}@{icon-font-name}.ttf') format('truetype'),\n url('@{icon-font-path}@{icon-font-name}.svg#@{icon-font-svg-id}') format('svg');\n}\n\n// Catchall baseclass\n.glyphicon {\n position: relative;\n top: 1px;\n display: inline-block;\n font-family: 'Glyphicons Halflings';\n font-style: normal;\n font-weight: normal;\n line-height: 1;\n -webkit-font-smoothing: antialiased;\n -moz-osx-font-smoothing: grayscale;\n}\n\n// Individual icons\n.glyphicon-asterisk { &:before { content: \"\\2a\"; } }\n.glyphicon-plus { &:before { content: \"\\2b\"; } }\n.glyphicon-euro,\n.glyphicon-eur { &:before { content: \"\\20ac\"; } }\n.glyphicon-minus { &:before { content: \"\\2212\"; } }\n.glyphicon-cloud { &:before { content: \"\\2601\"; } }\n.glyphicon-envelope { &:before { content: \"\\2709\"; } }\n.glyphicon-pencil { &:before { content: \"\\270f\"; } }\n.glyphicon-glass { &:before { content: \"\\e001\"; } }\n.glyphicon-music { &:before { content: \"\\e002\"; } }\n.glyphicon-search { &:before { content: \"\\e003\"; } }\n.glyphicon-heart { &:before { content: \"\\e005\"; } }\n.glyphicon-star { &:before { content: \"\\e006\"; } }\n.glyphicon-star-empty { &:before { content: \"\\e007\"; } }\n.glyphicon-user { &:before { content: \"\\e008\"; } }\n.glyphicon-film { &:before { content: \"\\e009\"; } }\n.glyphicon-th-large { &:before { content: \"\\e010\"; } }\n.glyphicon-th { &:before { content: \"\\e011\"; } }\n.glyphicon-th-list { &:before { content: \"\\e012\"; } }\n.glyphicon-ok { &:before { content: \"\\e013\"; } }\n.glyphicon-remove { &:before { content: \"\\e014\"; } }\n.glyphicon-zoom-in { &:before { content: \"\\e015\"; } }\n.glyphicon-zoom-out { &:before { content: \"\\e016\"; } }\n.glyphicon-off { &:before { content: \"\\e017\"; } }\n.glyphicon-signal { &:before { content: \"\\e018\"; } }\n.glyphicon-cog { &:before { content: \"\\e019\"; } }\n.glyphicon-trash { &:before { content: \"\\e020\"; } }\n.glyphicon-home { &:before { content: \"\\e021\"; } }\n.glyphicon-file { &:before { content: \"\\e022\"; } }\n.glyphicon-time { &:before { content: \"\\e023\"; } }\n.glyphicon-road { &:before { content: \"\\e024\"; } }\n.glyphicon-download-alt { &:before { content: \"\\e025\"; } }\n.glyphicon-download { &:before { content: \"\\e026\"; } }\n.glyphicon-upload { &:before { content: \"\\e027\"; } }\n.glyphicon-inbox { &:before { content: \"\\e028\"; } }\n.glyphicon-play-circle { &:before { content: \"\\e029\"; } }\n.glyphicon-repeat { &:before { content: \"\\e030\"; } }\n.glyphicon-refresh { &:before { content: \"\\e031\"; } }\n.glyphicon-list-alt { &:before { content: \"\\e032\"; } }\n.glyphicon-lock { &:before { content: \"\\e033\"; } }\n.glyphicon-flag { &:before { content: \"\\e034\"; } }\n.glyphicon-headphones { &:before { content: \"\\e035\"; } }\n.glyphicon-volume-off { &:before { content: \"\\e036\"; } }\n.glyphicon-volume-down { &:before { content: \"\\e037\"; } }\n.glyphicon-volume-up { &:before { content: \"\\e038\"; } }\n.glyphicon-qrcode { &:before { content: \"\\e039\"; } }\n.glyphicon-barcode { &:before { content: \"\\e040\"; } }\n.glyphicon-tag { &:before { content: \"\\e041\"; } }\n.glyphicon-tags { &:before { content: \"\\e042\"; } }\n.glyphicon-book { &:before { content: \"\\e043\"; } }\n.glyphicon-bookmark { &:before { content: \"\\e044\"; } }\n.glyphicon-print { &:before { content: \"\\e045\"; } }\n.glyphicon-camera { &:before { content: \"\\e046\"; } }\n.glyphicon-font { &:before { content: \"\\e047\"; } }\n.glyphicon-bold { &:before { content: \"\\e048\"; } }\n.glyphicon-italic { &:before { content: \"\\e049\"; } }\n.glyphicon-text-height { &:before { content: \"\\e050\"; } }\n.glyphicon-text-width { &:before { content: \"\\e051\"; } }\n.glyphicon-align-left { &:before { content: \"\\e052\"; } }\n.glyphicon-align-center { &:before { content: \"\\e053\"; } }\n.glyphicon-align-right { &:before { content: \"\\e054\"; } }\n.glyphicon-align-justify { &:before { content: \"\\e055\"; } }\n.glyphicon-list { &:before { content: \"\\e056\"; } }\n.glyphicon-indent-left { &:before { content: \"\\e057\"; } }\n.glyphicon-indent-right { &:before { content: \"\\e058\"; } }\n.glyphicon-facetime-video { &:before { content: \"\\e059\"; } }\n.glyphicon-picture { &:before { content: \"\\e060\"; } }\n.glyphicon-map-marker { &:before { content: \"\\e062\"; } }\n.glyphicon-adjust { &:before { content: \"\\e063\"; } }\n.glyphicon-tint { &:before { content: \"\\e064\"; } }\n.glyphicon-edit { &:before { content: \"\\e065\"; } }\n.glyphicon-share { &:before { content: \"\\e066\"; } }\n.glyphicon-check { &:before { content: \"\\e067\"; } }\n.glyphicon-move { &:before { content: \"\\e068\"; } }\n.glyphicon-step-backward { &:before { content: \"\\e069\"; } }\n.glyphicon-fast-backward { &:before { content: \"\\e070\"; } }\n.glyphicon-backward { &:before { content: \"\\e071\"; } }\n.glyphicon-play { &:before { content: \"\\e072\"; } }\n.glyphicon-pause { &:before { content: \"\\e073\"; } }\n.glyphicon-stop { &:before { content: \"\\e074\"; } }\n.glyphicon-forward { &:before { content: \"\\e075\"; } }\n.glyphicon-fast-forward { &:before { content: \"\\e076\"; } }\n.glyphicon-step-forward { &:before { content: \"\\e077\"; } }\n.glyphicon-eject { &:before { content: \"\\e078\"; } }\n.glyphicon-chevron-left { &:before { content: \"\\e079\"; } }\n.glyphicon-chevron-right { &:before { content: \"\\e080\"; } }\n.glyphicon-plus-sign { &:before { content: \"\\e081\"; } }\n.glyphicon-minus-sign { &:before { content: \"\\e082\"; } }\n.glyphicon-remove-sign { &:before { content: \"\\e083\"; } }\n.glyphicon-ok-sign { &:before { content: \"\\e084\"; } }\n.glyphicon-question-sign { &:before { content: \"\\e085\"; } }\n.glyphicon-info-sign { &:before { content: \"\\e086\"; } }\n.glyphicon-screenshot { &:before { content: \"\\e087\"; } }\n.glyphicon-remove-circle { &:before { content: \"\\e088\"; } }\n.glyphicon-ok-circle { &:before { content: \"\\e089\"; } }\n.glyphicon-ban-circle { &:before { content: \"\\e090\"; } }\n.glyphicon-arrow-left { &:before { content: \"\\e091\"; } }\n.glyphicon-arrow-right { &:before { content: \"\\e092\"; } }\n.glyphicon-arrow-up { &:before { content: \"\\e093\"; } }\n.glyphicon-arrow-down { &:before { content: \"\\e094\"; } }\n.glyphicon-share-alt { &:before { content: \"\\e095\"; } }\n.glyphicon-resize-full { &:before { content: \"\\e096\"; } }\n.glyphicon-resize-small { &:before { content: \"\\e097\"; } }\n.glyphicon-exclamation-sign { &:before { content: \"\\e101\"; } }\n.glyphicon-gift { &:before { content: \"\\e102\"; } }\n.glyphicon-leaf { &:before { content: \"\\e103\"; } }\n.glyphicon-fire { &:before { content: \"\\e104\"; } }\n.glyphicon-eye-open { &:before { content: \"\\e105\"; } }\n.glyphicon-eye-close { &:before { content: \"\\e106\"; } }\n.glyphicon-warning-sign { &:before { content: \"\\e107\"; } }\n.glyphicon-plane { &:before { content: \"\\e108\"; } }\n.glyphicon-calendar { &:before { content: \"\\e109\"; } }\n.glyphicon-random { &:before { content: \"\\e110\"; } }\n.glyphicon-comment { &:before { content: \"\\e111\"; } }\n.glyphicon-magnet { &:before { content: \"\\e112\"; } }\n.glyphicon-chevron-up { &:before { content: \"\\e113\"; } }\n.glyphicon-chevron-down { &:before { content: \"\\e114\"; } }\n.glyphicon-retweet { &:before { content: \"\\e115\"; } }\n.glyphicon-shopping-cart { &:before { content: \"\\e116\"; } }\n.glyphicon-folder-close { &:before { content: \"\\e117\"; } }\n.glyphicon-folder-open { &:before { content: \"\\e118\"; } }\n.glyphicon-resize-vertical { &:before { content: \"\\e119\"; } }\n.glyphicon-resize-horizontal { &:before { content: \"\\e120\"; } }\n.glyphicon-hdd { &:before { content: \"\\e121\"; } }\n.glyphicon-bullhorn { &:before { content: \"\\e122\"; } }\n.glyphicon-bell { &:before { content: \"\\e123\"; } }\n.glyphicon-certificate { &:before { content: \"\\e124\"; } }\n.glyphicon-thumbs-up { &:before { content: \"\\e125\"; } }\n.glyphicon-thumbs-down { &:before { content: \"\\e126\"; } }\n.glyphicon-hand-right { &:before { content: \"\\e127\"; } }\n.glyphicon-hand-left { &:before { content: \"\\e128\"; } }\n.glyphicon-hand-up { &:before { content: \"\\e129\"; } }\n.glyphicon-hand-down { &:before { content: \"\\e130\"; } }\n.glyphicon-circle-arrow-right { &:before { content: \"\\e131\"; } }\n.glyphicon-circle-arrow-left { &:before { content: \"\\e132\"; } }\n.glyphicon-circle-arrow-up { &:before { content: \"\\e133\"; } }\n.glyphicon-circle-arrow-down { &:before { content: \"\\e134\"; } }\n.glyphicon-globe { &:before { content: \"\\e135\"; } }\n.glyphicon-wrench { &:before { content: \"\\e136\"; } }\n.glyphicon-tasks { &:before { content: \"\\e137\"; } }\n.glyphicon-filter { &:before { content: \"\\e138\"; } }\n.glyphicon-briefcase { &:before { content: \"\\e139\"; } }\n.glyphicon-fullscreen { &:before { content: \"\\e140\"; } }\n.glyphicon-dashboard { &:before { content: \"\\e141\"; } }\n.glyphicon-paperclip { &:before { content: \"\\e142\"; } }\n.glyphicon-heart-empty { &:before { content: \"\\e143\"; } }\n.glyphicon-link { &:before { content: \"\\e144\"; } }\n.glyphicon-phone { &:before { content: \"\\e145\"; } }\n.glyphicon-pushpin { &:before { content: \"\\e146\"; } }\n.glyphicon-usd { &:before { content: \"\\e148\"; } }\n.glyphicon-gbp { &:before { content: \"\\e149\"; } }\n.glyphicon-sort { &:before { content: \"\\e150\"; } }\n.glyphicon-sort-by-alphabet { &:before { content: \"\\e151\"; } }\n.glyphicon-sort-by-alphabet-alt { &:before { content: \"\\e152\"; } }\n.glyphicon-sort-by-order { &:before { content: \"\\e153\"; } }\n.glyphicon-sort-by-order-alt { &:before { content: \"\\e154\"; } }\n.glyphicon-sort-by-attributes { &:before { content: \"\\e155\"; } }\n.glyphicon-sort-by-attributes-alt { &:before { content: \"\\e156\"; } }\n.glyphicon-unchecked { &:before { content: \"\\e157\"; } }\n.glyphicon-expand { &:before { content: \"\\e158\"; } }\n.glyphicon-collapse-down { &:before { content: \"\\e159\"; } }\n.glyphicon-collapse-up { &:before { content: \"\\e160\"; } }\n.glyphicon-log-in { &:before { content: \"\\e161\"; } }\n.glyphicon-flash { &:before { content: \"\\e162\"; } }\n.glyphicon-log-out { &:before { content: \"\\e163\"; } }\n.glyphicon-new-window { &:before { content: \"\\e164\"; } }\n.glyphicon-record { &:before { content: \"\\e165\"; } }\n.glyphicon-save { &:before { content: \"\\e166\"; } }\n.glyphicon-open { &:before { content: \"\\e167\"; } }\n.glyphicon-saved { &:before { content: \"\\e168\"; } }\n.glyphicon-import { &:before { content: \"\\e169\"; } }\n.glyphicon-export { &:before { content: \"\\e170\"; } }\n.glyphicon-send { &:before { content: \"\\e171\"; } }\n.glyphicon-floppy-disk { &:before { content: \"\\e172\"; } }\n.glyphicon-floppy-saved { &:before { content: \"\\e173\"; } }\n.glyphicon-floppy-remove { &:before { content: \"\\e174\"; } }\n.glyphicon-floppy-save { &:before { content: \"\\e175\"; } }\n.glyphicon-floppy-open { &:before { content: \"\\e176\"; } }\n.glyphicon-credit-card { &:before { content: \"\\e177\"; } }\n.glyphicon-transfer { &:before { content: \"\\e178\"; } }\n.glyphicon-cutlery { &:before { content: \"\\e179\"; } }\n.glyphicon-header { &:before { content: \"\\e180\"; } }\n.glyphicon-compressed { &:before { content: \"\\e181\"; } }\n.glyphicon-earphone { &:before { content: \"\\e182\"; } }\n.glyphicon-phone-alt { &:before { content: \"\\e183\"; } }\n.glyphicon-tower { &:before { content: \"\\e184\"; } }\n.glyphicon-stats { &:before { content: \"\\e185\"; } }\n.glyphicon-sd-video { &:before { content: \"\\e186\"; } }\n.glyphicon-hd-video { &:before { content: \"\\e187\"; } }\n.glyphicon-subtitles { &:before { content: \"\\e188\"; } }\n.glyphicon-sound-stereo { &:before { content: \"\\e189\"; } }\n.glyphicon-sound-dolby { &:before { content: \"\\e190\"; } }\n.glyphicon-sound-5-1 { &:before { content: \"\\e191\"; } }\n.glyphicon-sound-6-1 { &:before { content: \"\\e192\"; } }\n.glyphicon-sound-7-1 { &:before { content: \"\\e193\"; } }\n.glyphicon-copyright-mark { &:before { content: \"\\e194\"; } }\n.glyphicon-registration-mark { &:before { content: \"\\e195\"; } }\n.glyphicon-cloud-download { &:before { content: \"\\e197\"; } }\n.glyphicon-cloud-upload { &:before { content: \"\\e198\"; } }\n.glyphicon-tree-conifer { &:before { content: \"\\e199\"; } }\n.glyphicon-tree-deciduous { &:before { content: \"\\e200\"; } }\n.glyphicon-cd { &:before { content: \"\\e201\"; } }\n.glyphicon-save-file { &:before { content: \"\\e202\"; } }\n.glyphicon-open-file { &:before { content: \"\\e203\"; } }\n.glyphicon-level-up { &:before { content: \"\\e204\"; } }\n.glyphicon-copy { &:before { content: \"\\e205\"; } }\n.glyphicon-paste { &:before { content: \"\\e206\"; } }\n// The following 2 Glyphicons are omitted for the time being because\n// they currently use Unicode codepoints that are outside the\n// Basic Multilingual Plane (BMP). Older buggy versions of WebKit can't handle\n// non-BMP codepoints in CSS string escapes, and thus can't display these two icons.\n// Notably, the bug affects some older versions of the Android Browser.\n// More info: https://github.com/twbs/bootstrap/issues/10106\n// .glyphicon-door { &:before { content: \"\\1f6aa\"; } }\n// .glyphicon-key { &:before { content: \"\\1f511\"; } }\n.glyphicon-alert { &:before { content: \"\\e209\"; } }\n.glyphicon-equalizer { &:before { content: \"\\e210\"; } }\n.glyphicon-king { &:before { content: \"\\e211\"; } }\n.glyphicon-queen { &:before { content: \"\\e212\"; } }\n.glyphicon-pawn { &:before { content: \"\\e213\"; } }\n.glyphicon-bishop { &:before { content: \"\\e214\"; } }\n.glyphicon-knight { &:before { content: \"\\e215\"; } }\n.glyphicon-baby-formula { &:before { content: \"\\e216\"; } }\n.glyphicon-tent { &:before { content: \"\\26fa\"; } }\n.glyphicon-blackboard { &:before { content: \"\\e218\"; } }\n.glyphicon-bed { &:before { content: \"\\e219\"; } }\n.glyphicon-apple { &:before { content: \"\\f8ff\"; } }\n.glyphicon-erase { &:before { content: \"\\e221\"; } }\n.glyphicon-hourglass { &:before { content: \"\\231b\"; } }\n.glyphicon-lamp { &:before { content: \"\\e223\"; } }\n.glyphicon-duplicate { &:before { content: \"\\e224\"; } }\n.glyphicon-piggy-bank { &:before { content: \"\\e225\"; } }\n.glyphicon-scissors { &:before { content: \"\\e226\"; } }\n.glyphicon-bitcoin { &:before { content: \"\\e227\"; } }\n.glyphicon-btc { &:before { content: \"\\e227\"; } }\n.glyphicon-xbt { &:before { content: \"\\e227\"; } }\n.glyphicon-yen { &:before { content: \"\\00a5\"; } }\n.glyphicon-jpy { &:before { content: \"\\00a5\"; } }\n.glyphicon-ruble { &:before { content: \"\\20bd\"; } }\n.glyphicon-rub { &:before { content: \"\\20bd\"; } }\n.glyphicon-scale { &:before { content: \"\\e230\"; } }\n.glyphicon-ice-lolly { &:before { content: \"\\e231\"; } }\n.glyphicon-ice-lolly-tasted { &:before { content: \"\\e232\"; } }\n.glyphicon-education { &:before { content: \"\\e233\"; } }\n.glyphicon-option-horizontal { &:before { content: \"\\e234\"; } }\n.glyphicon-option-vertical { &:before { content: \"\\e235\"; } }\n.glyphicon-menu-hamburger { &:before { content: \"\\e236\"; } }\n.glyphicon-modal-window { &:before { content: \"\\e237\"; } }\n.glyphicon-oil { &:before { content: \"\\e238\"; } }\n.glyphicon-grain { &:before { content: \"\\e239\"; } }\n.glyphicon-sunglasses { &:before { content: \"\\e240\"; } }\n.glyphicon-text-size { &:before { content: \"\\e241\"; } }\n.glyphicon-text-color { &:before { content: \"\\e242\"; } }\n.glyphicon-text-background { &:before { content: \"\\e243\"; } }\n.glyphicon-object-align-top { &:before { content: \"\\e244\"; } }\n.glyphicon-object-align-bottom { &:before { content: \"\\e245\"; } }\n.glyphicon-object-align-horizontal{ &:before { content: \"\\e246\"; } }\n.glyphicon-object-align-left { &:before { content: \"\\e247\"; } }\n.glyphicon-object-align-vertical { &:before { content: \"\\e248\"; } }\n.glyphicon-object-align-right { &:before { content: \"\\e249\"; } }\n.glyphicon-triangle-right { &:before { content: \"\\e250\"; } }\n.glyphicon-triangle-left { &:before { content: \"\\e251\"; } }\n.glyphicon-triangle-bottom { &:before { content: \"\\e252\"; } }\n.glyphicon-triangle-top { &:before { content: \"\\e253\"; } }\n.glyphicon-console { &:before { content: \"\\e254\"; } }\n.glyphicon-superscript { &:before { content: \"\\e255\"; } }\n.glyphicon-subscript { &:before { content: \"\\e256\"; } }\n.glyphicon-menu-left { &:before { content: \"\\e257\"; } }\n.glyphicon-menu-right { &:before { content: \"\\e258\"; } }\n.glyphicon-menu-down { &:before { content: \"\\e259\"; } }\n.glyphicon-menu-up { &:before { content: \"\\e260\"; } }\n","//\n// Scaffolding\n// --------------------------------------------------\n\n\n// Reset the box-sizing\n//\n// Heads up! This reset may cause conflicts with some third-party widgets.\n// For recommendations on resolving such conflicts, see\n// http://getbootstrap.com/getting-started/#third-box-sizing\n* {\n .box-sizing(border-box);\n}\n*:before,\n*:after {\n .box-sizing(border-box);\n}\n\n\n// Body reset\n\nhtml {\n font-size: 10px;\n -webkit-tap-highlight-color: rgba(0,0,0,0);\n}\n\nbody {\n font-family: @font-family-base;\n font-size: @font-size-base;\n line-height: @line-height-base;\n color: @text-color;\n background-color: @body-bg;\n}\n\n// Reset fonts for relevant elements\ninput,\nbutton,\nselect,\ntextarea {\n font-family: inherit;\n font-size: inherit;\n line-height: inherit;\n}\n\n\n// Links\n\na {\n color: @link-color;\n text-decoration: none;\n\n &:hover,\n &:focus {\n color: @link-hover-color;\n text-decoration: @link-hover-decoration;\n }\n\n &:focus {\n .tab-focus();\n }\n}\n\n\n// Figures\n//\n// We reset this here because previously Normalize had no `figure` margins. This\n// ensures we don't break anyone's use of the element.\n\nfigure {\n margin: 0;\n}\n\n\n// Images\n\nimg {\n vertical-align: middle;\n}\n\n// Responsive images (ensure images don't scale beyond their parents)\n.img-responsive {\n .img-responsive();\n}\n\n// Rounded corners\n.img-rounded {\n border-radius: @border-radius-large;\n}\n\n// Image thumbnails\n//\n// Heads up! This is mixin-ed into thumbnails.less for `.thumbnail`.\n.img-thumbnail {\n padding: @thumbnail-padding;\n line-height: @line-height-base;\n background-color: @thumbnail-bg;\n border: 1px solid @thumbnail-border;\n border-radius: @thumbnail-border-radius;\n .transition(all .2s ease-in-out);\n\n // Keep them at most 100% wide\n .img-responsive(inline-block);\n}\n\n// Perfect circle\n.img-circle {\n border-radius: 50%; // set radius in percents\n}\n\n\n// Horizontal rules\n\nhr {\n margin-top: @line-height-computed;\n margin-bottom: @line-height-computed;\n border: 0;\n border-top: 1px solid @hr-border;\n}\n\n\n// Only display content to screen readers\n//\n// See: http://a11yproject.com/posts/how-to-hide-content/\n\n.sr-only {\n position: absolute;\n width: 1px;\n height: 1px;\n margin: -1px;\n padding: 0;\n overflow: hidden;\n clip: rect(0,0,0,0);\n border: 0;\n}\n\n// Use in conjunction with .sr-only to only display content when it's focused.\n// Useful for \"Skip to main content\" links; see http://www.w3.org/TR/2013/NOTE-WCAG20-TECHS-20130905/G1\n// Credit: HTML5 Boilerplate\n\n.sr-only-focusable {\n &:active,\n &:focus {\n position: static;\n width: auto;\n height: auto;\n margin: 0;\n overflow: visible;\n clip: auto;\n }\n}\n\n\n// iOS \"clickable elements\" fix for role=\"button\"\n//\n// Fixes \"clickability\" issue (and more generally, the firing of events such as focus as well)\n// for traditionally non-focusable elements with role=\"button\"\n// see https://developer.mozilla.org/en-US/docs/Web/Events/click#Safari_Mobile\n\n[role=\"button\"] {\n cursor: pointer;\n}\n","// Vendor Prefixes\n//\n// All vendor mixins are deprecated as of v3.2.0 due to the introduction of\n// Autoprefixer in our Gruntfile. They will be removed in v4.\n\n// - Animations\n// - Backface visibility\n// - Box shadow\n// - Box sizing\n// - Content columns\n// - Hyphens\n// - Placeholder text\n// - Transformations\n// - Transitions\n// - User Select\n\n\n// Animations\n.animation(@animation) {\n -webkit-animation: @animation;\n -o-animation: @animation;\n animation: @animation;\n}\n.animation-name(@name) {\n -webkit-animation-name: @name;\n animation-name: @name;\n}\n.animation-duration(@duration) {\n -webkit-animation-duration: @duration;\n animation-duration: @duration;\n}\n.animation-timing-function(@timing-function) {\n -webkit-animation-timing-function: @timing-function;\n animation-timing-function: @timing-function;\n}\n.animation-delay(@delay) {\n -webkit-animation-delay: @delay;\n animation-delay: @delay;\n}\n.animation-iteration-count(@iteration-count) {\n -webkit-animation-iteration-count: @iteration-count;\n animation-iteration-count: @iteration-count;\n}\n.animation-direction(@direction) {\n -webkit-animation-direction: @direction;\n animation-direction: @direction;\n}\n.animation-fill-mode(@fill-mode) {\n -webkit-animation-fill-mode: @fill-mode;\n animation-fill-mode: @fill-mode;\n}\n\n// Backface visibility\n// Prevent browsers from flickering when using CSS 3D transforms.\n// Default value is `visible`, but can be changed to `hidden`\n\n.backface-visibility(@visibility){\n -webkit-backface-visibility: @visibility;\n -moz-backface-visibility: @visibility;\n backface-visibility: @visibility;\n}\n\n// Drop shadows\n//\n// Note: Deprecated `.box-shadow()` as of v3.1.0 since all of Bootstrap's\n// supported browsers that have box shadow capabilities now support it.\n\n.box-shadow(@shadow) {\n -webkit-box-shadow: @shadow; // iOS <4.3 & Android <4.1\n box-shadow: @shadow;\n}\n\n// Box sizing\n.box-sizing(@boxmodel) {\n -webkit-box-sizing: @boxmodel;\n -moz-box-sizing: @boxmodel;\n box-sizing: @boxmodel;\n}\n\n// CSS3 Content Columns\n.content-columns(@column-count; @column-gap: @grid-gutter-width) {\n -webkit-column-count: @column-count;\n -moz-column-count: @column-count;\n column-count: @column-count;\n -webkit-column-gap: @column-gap;\n -moz-column-gap: @column-gap;\n column-gap: @column-gap;\n}\n\n// Optional hyphenation\n.hyphens(@mode: auto) {\n word-wrap: break-word;\n -webkit-hyphens: @mode;\n -moz-hyphens: @mode;\n -ms-hyphens: @mode; // IE10+\n -o-hyphens: @mode;\n hyphens: @mode;\n}\n\n// Placeholder text\n.placeholder(@color: @input-color-placeholder) {\n // Firefox\n &::-moz-placeholder {\n color: @color;\n opacity: 1; // Override Firefox's unusual default opacity; see https://github.com/twbs/bootstrap/pull/11526\n }\n &:-ms-input-placeholder { color: @color; } // Internet Explorer 10+\n &::-webkit-input-placeholder { color: @color; } // Safari and Chrome\n}\n\n// Transformations\n.scale(@ratio) {\n -webkit-transform: scale(@ratio);\n -ms-transform: scale(@ratio); // IE9 only\n -o-transform: scale(@ratio);\n transform: scale(@ratio);\n}\n.scale(@ratioX; @ratioY) {\n -webkit-transform: scale(@ratioX, @ratioY);\n -ms-transform: scale(@ratioX, @ratioY); // IE9 only\n -o-transform: scale(@ratioX, @ratioY);\n transform: scale(@ratioX, @ratioY);\n}\n.scaleX(@ratio) {\n -webkit-transform: scaleX(@ratio);\n -ms-transform: scaleX(@ratio); // IE9 only\n -o-transform: scaleX(@ratio);\n transform: scaleX(@ratio);\n}\n.scaleY(@ratio) {\n -webkit-transform: scaleY(@ratio);\n -ms-transform: scaleY(@ratio); // IE9 only\n -o-transform: scaleY(@ratio);\n transform: scaleY(@ratio);\n}\n.skew(@x; @y) {\n -webkit-transform: skewX(@x) skewY(@y);\n -ms-transform: skewX(@x) skewY(@y); // See https://github.com/twbs/bootstrap/issues/4885; IE9+\n -o-transform: skewX(@x) skewY(@y);\n transform: skewX(@x) skewY(@y);\n}\n.translate(@x; @y) {\n -webkit-transform: translate(@x, @y);\n -ms-transform: translate(@x, @y); // IE9 only\n -o-transform: translate(@x, @y);\n transform: translate(@x, @y);\n}\n.translate3d(@x; @y; @z) {\n -webkit-transform: translate3d(@x, @y, @z);\n transform: translate3d(@x, @y, @z);\n}\n.rotate(@degrees) {\n -webkit-transform: rotate(@degrees);\n -ms-transform: rotate(@degrees); // IE9 only\n -o-transform: rotate(@degrees);\n transform: rotate(@degrees);\n}\n.rotateX(@degrees) {\n -webkit-transform: rotateX(@degrees);\n -ms-transform: rotateX(@degrees); // IE9 only\n -o-transform: rotateX(@degrees);\n transform: rotateX(@degrees);\n}\n.rotateY(@degrees) {\n -webkit-transform: rotateY(@degrees);\n -ms-transform: rotateY(@degrees); // IE9 only\n -o-transform: rotateY(@degrees);\n transform: rotateY(@degrees);\n}\n.perspective(@perspective) {\n -webkit-perspective: @perspective;\n -moz-perspective: @perspective;\n perspective: @perspective;\n}\n.perspective-origin(@perspective) {\n -webkit-perspective-origin: @perspective;\n -moz-perspective-origin: @perspective;\n perspective-origin: @perspective;\n}\n.transform-origin(@origin) {\n -webkit-transform-origin: @origin;\n -moz-transform-origin: @origin;\n -ms-transform-origin: @origin; // IE9 only\n transform-origin: @origin;\n}\n\n\n// Transitions\n\n.transition(@transition) {\n -webkit-transition: @transition;\n -o-transition: @transition;\n transition: @transition;\n}\n.transition-property(@transition-property) {\n -webkit-transition-property: @transition-property;\n transition-property: @transition-property;\n}\n.transition-delay(@transition-delay) {\n -webkit-transition-delay: @transition-delay;\n transition-delay: @transition-delay;\n}\n.transition-duration(@transition-duration) {\n -webkit-transition-duration: @transition-duration;\n transition-duration: @transition-duration;\n}\n.transition-timing-function(@timing-function) {\n -webkit-transition-timing-function: @timing-function;\n transition-timing-function: @timing-function;\n}\n.transition-transform(@transition) {\n -webkit-transition: -webkit-transform @transition;\n -moz-transition: -moz-transform @transition;\n -o-transition: -o-transform @transition;\n transition: transform @transition;\n}\n\n\n// User select\n// For selecting text on the page\n\n.user-select(@select) {\n -webkit-user-select: @select;\n -moz-user-select: @select;\n -ms-user-select: @select; // IE10+\n user-select: @select;\n}\n","// WebKit-style focus\n\n.tab-focus() {\n // Default\n outline: thin dotted;\n // WebKit\n outline: 5px auto -webkit-focus-ring-color;\n outline-offset: -2px;\n}\n","// Image Mixins\n// - Responsive image\n// - Retina image\n\n\n// Responsive image\n//\n// Keep images from scaling beyond the width of their parents.\n.img-responsive(@display: block) {\n display: @display;\n max-width: 100%; // Part 1: Set a maximum relative to the parent\n height: auto; // Part 2: Scale the height according to the width, otherwise you get stretching\n}\n\n\n// Retina image\n//\n// Short retina mixin for setting background-image and -size. Note that the\n// spelling of `min--moz-device-pixel-ratio` is intentional.\n.img-retina(@file-1x; @file-2x; @width-1x; @height-1x) {\n background-image: url(\"@{file-1x}\");\n\n @media\n only screen and (-webkit-min-device-pixel-ratio: 2),\n only screen and ( min--moz-device-pixel-ratio: 2),\n only screen and ( -o-min-device-pixel-ratio: 2/1),\n only screen and ( min-device-pixel-ratio: 2),\n only screen and ( min-resolution: 192dpi),\n only screen and ( min-resolution: 2dppx) {\n background-image: url(\"@{file-2x}\");\n background-size: @width-1x @height-1x;\n }\n}\n","//\n// Typography\n// --------------------------------------------------\n\n\n// Headings\n// -------------------------\n\nh1, h2, h3, h4, h5, h6,\n.h1, .h2, .h3, .h4, .h5, .h6 {\n font-family: @headings-font-family;\n font-weight: @headings-font-weight;\n line-height: @headings-line-height;\n color: @headings-color;\n\n small,\n .small {\n font-weight: normal;\n line-height: 1;\n color: @headings-small-color;\n }\n}\n\nh1, .h1,\nh2, .h2,\nh3, .h3 {\n margin-top: @line-height-computed;\n margin-bottom: (@line-height-computed / 2);\n\n small,\n .small {\n font-size: 65%;\n }\n}\nh4, .h4,\nh5, .h5,\nh6, .h6 {\n margin-top: (@line-height-computed / 2);\n margin-bottom: (@line-height-computed / 2);\n\n small,\n .small {\n font-size: 75%;\n }\n}\n\nh1, .h1 { font-size: @font-size-h1; }\nh2, .h2 { font-size: @font-size-h2; }\nh3, .h3 { font-size: @font-size-h3; }\nh4, .h4 { font-size: @font-size-h4; }\nh5, .h5 { font-size: @font-size-h5; }\nh6, .h6 { font-size: @font-size-h6; }\n\n\n// Body text\n// -------------------------\n\np {\n margin: 0 0 (@line-height-computed / 2);\n}\n\n.lead {\n margin-bottom: @line-height-computed;\n font-size: floor((@font-size-base * 1.15));\n font-weight: 300;\n line-height: 1.4;\n\n @media (min-width: @screen-sm-min) {\n font-size: (@font-size-base * 1.5);\n }\n}\n\n\n// Emphasis & misc\n// -------------------------\n\n// Ex: (12px small font / 14px base font) * 100% = about 85%\nsmall,\n.small {\n font-size: floor((100% * @font-size-small / @font-size-base));\n}\n\nmark,\n.mark {\n background-color: @state-warning-bg;\n padding: .2em;\n}\n\n// Alignment\n.text-left { text-align: left; }\n.text-right { text-align: right; }\n.text-center { text-align: center; }\n.text-justify { text-align: justify; }\n.text-nowrap { white-space: nowrap; }\n\n// Transformation\n.text-lowercase { text-transform: lowercase; }\n.text-uppercase { text-transform: uppercase; }\n.text-capitalize { text-transform: capitalize; }\n\n// Contextual colors\n.text-muted {\n color: @text-muted;\n}\n.text-primary {\n .text-emphasis-variant(@brand-primary);\n}\n.text-success {\n .text-emphasis-variant(@state-success-text);\n}\n.text-info {\n .text-emphasis-variant(@state-info-text);\n}\n.text-warning {\n .text-emphasis-variant(@state-warning-text);\n}\n.text-danger {\n .text-emphasis-variant(@state-danger-text);\n}\n\n// Contextual backgrounds\n// For now we'll leave these alongside the text classes until v4 when we can\n// safely shift things around (per SemVer rules).\n.bg-primary {\n // Given the contrast here, this is the only class to have its color inverted\n // automatically.\n color: #fff;\n .bg-variant(@brand-primary);\n}\n.bg-success {\n .bg-variant(@state-success-bg);\n}\n.bg-info {\n .bg-variant(@state-info-bg);\n}\n.bg-warning {\n .bg-variant(@state-warning-bg);\n}\n.bg-danger {\n .bg-variant(@state-danger-bg);\n}\n\n\n// Page header\n// -------------------------\n\n.page-header {\n padding-bottom: ((@line-height-computed / 2) - 1);\n margin: (@line-height-computed * 2) 0 @line-height-computed;\n border-bottom: 1px solid @page-header-border-color;\n}\n\n\n// Lists\n// -------------------------\n\n// Unordered and Ordered lists\nul,\nol {\n margin-top: 0;\n margin-bottom: (@line-height-computed / 2);\n ul,\n ol {\n margin-bottom: 0;\n }\n}\n\n// List options\n\n// Unstyled keeps list items block level, just removes default browser padding and list-style\n.list-unstyled {\n padding-left: 0;\n list-style: none;\n}\n\n// Inline turns list items into inline-block\n.list-inline {\n .list-unstyled();\n margin-left: -5px;\n\n > li {\n display: inline-block;\n padding-left: 5px;\n padding-right: 5px;\n }\n}\n\n// Description Lists\ndl {\n margin-top: 0; // Remove browser default\n margin-bottom: @line-height-computed;\n}\ndt,\ndd {\n line-height: @line-height-base;\n}\ndt {\n font-weight: bold;\n}\ndd {\n margin-left: 0; // Undo browser default\n}\n\n// Horizontal description lists\n//\n// Defaults to being stacked without any of the below styles applied, until the\n// grid breakpoint is reached (default of ~768px).\n\n.dl-horizontal {\n dd {\n &:extend(.clearfix all); // Clear the floated `dt` if an empty `dd` is present\n }\n\n @media (min-width: @grid-float-breakpoint) {\n dt {\n float: left;\n width: (@dl-horizontal-offset - 20);\n clear: left;\n text-align: right;\n .text-overflow();\n }\n dd {\n margin-left: @dl-horizontal-offset;\n }\n }\n}\n\n\n// Misc\n// -------------------------\n\n// Abbreviations and acronyms\nabbr[title],\n// Add data-* attribute to help out our tooltip plugin, per https://github.com/twbs/bootstrap/issues/5257\nabbr[data-original-title] {\n cursor: help;\n border-bottom: 1px dotted @abbr-border-color;\n}\n.initialism {\n font-size: 90%;\n .text-uppercase();\n}\n\n// Blockquotes\nblockquote {\n padding: (@line-height-computed / 2) @line-height-computed;\n margin: 0 0 @line-height-computed;\n font-size: @blockquote-font-size;\n border-left: 5px solid @blockquote-border-color;\n\n p,\n ul,\n ol {\n &:last-child {\n margin-bottom: 0;\n }\n }\n\n // Note: Deprecated small and .small as of v3.1.0\n // Context: https://github.com/twbs/bootstrap/issues/11660\n footer,\n small,\n .small {\n display: block;\n font-size: 80%; // back to default font-size\n line-height: @line-height-base;\n color: @blockquote-small-color;\n\n &:before {\n content: '\\2014 \\00A0'; // em dash, nbsp\n }\n }\n}\n\n// Opposite alignment of blockquote\n//\n// Heads up: `blockquote.pull-right` has been deprecated as of v3.1.0.\n.blockquote-reverse,\nblockquote.pull-right {\n padding-right: 15px;\n padding-left: 0;\n border-right: 5px solid @blockquote-border-color;\n border-left: 0;\n text-align: right;\n\n // Account for citation\n footer,\n small,\n .small {\n &:before { content: ''; }\n &:after {\n content: '\\00A0 \\2014'; // nbsp, em dash\n }\n }\n}\n\n// Addresses\naddress {\n margin-bottom: @line-height-computed;\n font-style: normal;\n line-height: @line-height-base;\n}\n","// Typography\n\n.text-emphasis-variant(@color) {\n color: @color;\n a&:hover,\n a&:focus {\n color: darken(@color, 10%);\n }\n}\n","// Contextual backgrounds\n\n.bg-variant(@color) {\n background-color: @color;\n a&:hover,\n a&:focus {\n background-color: darken(@color, 10%);\n }\n}\n","// Text overflow\n// Requires inline-block or block for proper styling\n\n.text-overflow() {\n overflow: hidden;\n text-overflow: ellipsis;\n white-space: nowrap;\n}\n","//\n// Code (inline and block)\n// --------------------------------------------------\n\n\n// Inline and block code styles\ncode,\nkbd,\npre,\nsamp {\n font-family: @font-family-monospace;\n}\n\n// Inline code\ncode {\n padding: 2px 4px;\n font-size: 90%;\n color: @code-color;\n background-color: @code-bg;\n border-radius: @border-radius-base;\n}\n\n// User input typically entered via keyboard\nkbd {\n padding: 2px 4px;\n font-size: 90%;\n color: @kbd-color;\n background-color: @kbd-bg;\n border-radius: @border-radius-small;\n box-shadow: inset 0 -1px 0 rgba(0,0,0,.25);\n\n kbd {\n padding: 0;\n font-size: 100%;\n font-weight: bold;\n box-shadow: none;\n }\n}\n\n// Blocks of code\npre {\n display: block;\n padding: ((@line-height-computed - 1) / 2);\n margin: 0 0 (@line-height-computed / 2);\n font-size: (@font-size-base - 1); // 14px to 13px\n line-height: @line-height-base;\n word-break: break-all;\n word-wrap: break-word;\n color: @pre-color;\n background-color: @pre-bg;\n border: 1px solid @pre-border-color;\n border-radius: @border-radius-base;\n\n // Account for some code outputs that place code tags in pre tags\n code {\n padding: 0;\n font-size: inherit;\n color: inherit;\n white-space: pre-wrap;\n background-color: transparent;\n border-radius: 0;\n }\n}\n\n// Enable scrollable blocks of code\n.pre-scrollable {\n max-height: @pre-scrollable-max-height;\n overflow-y: scroll;\n}\n","//\n// Grid system\n// --------------------------------------------------\n\n\n// Container widths\n//\n// Set the container width, and override it for fixed navbars in media queries.\n\n.container {\n .container-fixed();\n\n @media (min-width: @screen-sm-min) {\n width: @container-sm;\n }\n @media (min-width: @screen-md-min) {\n width: @container-md;\n }\n @media (min-width: @screen-lg-min) {\n width: @container-lg;\n }\n}\n\n\n// Fluid container\n//\n// Utilizes the mixin meant for fixed width containers, but without any defined\n// width for fluid, full width layouts.\n\n.container-fluid {\n .container-fixed();\n}\n\n\n// Row\n//\n// Rows contain and clear the floats of your columns.\n\n.row {\n .make-row();\n}\n\n\n// Columns\n//\n// Common styles for small and large grid columns\n\n.make-grid-columns();\n\n\n// Extra small grid\n//\n// Columns, offsets, pushes, and pulls for extra small devices like\n// smartphones.\n\n.make-grid(xs);\n\n\n// Small grid\n//\n// Columns, offsets, pushes, and pulls for the small device range, from phones\n// to tablets.\n\n@media (min-width: @screen-sm-min) {\n .make-grid(sm);\n}\n\n\n// Medium grid\n//\n// Columns, offsets, pushes, and pulls for the desktop device range.\n\n@media (min-width: @screen-md-min) {\n .make-grid(md);\n}\n\n\n// Large grid\n//\n// Columns, offsets, pushes, and pulls for the large desktop device range.\n\n@media (min-width: @screen-lg-min) {\n .make-grid(lg);\n}\n","// Grid system\n//\n// Generate semantic grid columns with these mixins.\n\n// Centered container element\n.container-fixed(@gutter: @grid-gutter-width) {\n margin-right: auto;\n margin-left: auto;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n &:extend(.clearfix all);\n}\n\n// Creates a wrapper for a series of columns\n.make-row(@gutter: @grid-gutter-width) {\n margin-left: ceil((@gutter / -2));\n margin-right: floor((@gutter / -2));\n &:extend(.clearfix all);\n}\n\n// Generate the extra small columns\n.make-xs-column(@columns; @gutter: @grid-gutter-width) {\n position: relative;\n float: left;\n width: percentage((@columns / @grid-columns));\n min-height: 1px;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n}\n.make-xs-column-offset(@columns) {\n margin-left: percentage((@columns / @grid-columns));\n}\n.make-xs-column-push(@columns) {\n left: percentage((@columns / @grid-columns));\n}\n.make-xs-column-pull(@columns) {\n right: percentage((@columns / @grid-columns));\n}\n\n// Generate the small columns\n.make-sm-column(@columns; @gutter: @grid-gutter-width) {\n position: relative;\n min-height: 1px;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n\n @media (min-width: @screen-sm-min) {\n float: left;\n width: percentage((@columns / @grid-columns));\n }\n}\n.make-sm-column-offset(@columns) {\n @media (min-width: @screen-sm-min) {\n margin-left: percentage((@columns / @grid-columns));\n }\n}\n.make-sm-column-push(@columns) {\n @media (min-width: @screen-sm-min) {\n left: percentage((@columns / @grid-columns));\n }\n}\n.make-sm-column-pull(@columns) {\n @media (min-width: @screen-sm-min) {\n right: percentage((@columns / @grid-columns));\n }\n}\n\n// Generate the medium columns\n.make-md-column(@columns; @gutter: @grid-gutter-width) {\n position: relative;\n min-height: 1px;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n\n @media (min-width: @screen-md-min) {\n float: left;\n width: percentage((@columns / @grid-columns));\n }\n}\n.make-md-column-offset(@columns) {\n @media (min-width: @screen-md-min) {\n margin-left: percentage((@columns / @grid-columns));\n }\n}\n.make-md-column-push(@columns) {\n @media (min-width: @screen-md-min) {\n left: percentage((@columns / @grid-columns));\n }\n}\n.make-md-column-pull(@columns) {\n @media (min-width: @screen-md-min) {\n right: percentage((@columns / @grid-columns));\n }\n}\n\n// Generate the large columns\n.make-lg-column(@columns; @gutter: @grid-gutter-width) {\n position: relative;\n min-height: 1px;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n\n @media (min-width: @screen-lg-min) {\n float: left;\n width: percentage((@columns / @grid-columns));\n }\n}\n.make-lg-column-offset(@columns) {\n @media (min-width: @screen-lg-min) {\n margin-left: percentage((@columns / @grid-columns));\n }\n}\n.make-lg-column-push(@columns) {\n @media (min-width: @screen-lg-min) {\n left: percentage((@columns / @grid-columns));\n }\n}\n.make-lg-column-pull(@columns) {\n @media (min-width: @screen-lg-min) {\n right: percentage((@columns / @grid-columns));\n }\n}\n","// Framework grid generation\n//\n// Used only by Bootstrap to generate the correct number of grid classes given\n// any value of `@grid-columns`.\n\n.make-grid-columns() {\n // Common styles for all sizes of grid columns, widths 1-12\n .col(@index) { // initial\n @item: ~\".col-xs-@{index}, .col-sm-@{index}, .col-md-@{index}, .col-lg-@{index}\";\n .col((@index + 1), @item);\n }\n .col(@index, @list) when (@index =< @grid-columns) { // general; \"=<\" isn't a typo\n @item: ~\".col-xs-@{index}, .col-sm-@{index}, .col-md-@{index}, .col-lg-@{index}\";\n .col((@index + 1), ~\"@{list}, @{item}\");\n }\n .col(@index, @list) when (@index > @grid-columns) { // terminal\n @{list} {\n position: relative;\n // Prevent columns from collapsing when empty\n min-height: 1px;\n // Inner gutter via padding\n padding-left: ceil((@grid-gutter-width / 2));\n padding-right: floor((@grid-gutter-width / 2));\n }\n }\n .col(1); // kickstart it\n}\n\n.float-grid-columns(@class) {\n .col(@index) { // initial\n @item: ~\".col-@{class}-@{index}\";\n .col((@index + 1), @item);\n }\n .col(@index, @list) when (@index =< @grid-columns) { // general\n @item: ~\".col-@{class}-@{index}\";\n .col((@index + 1), ~\"@{list}, @{item}\");\n }\n .col(@index, @list) when (@index > @grid-columns) { // terminal\n @{list} {\n float: left;\n }\n }\n .col(1); // kickstart it\n}\n\n.calc-grid-column(@index, @class, @type) when (@type = width) and (@index > 0) {\n .col-@{class}-@{index} {\n width: percentage((@index / @grid-columns));\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = push) and (@index > 0) {\n .col-@{class}-push-@{index} {\n left: percentage((@index / @grid-columns));\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = push) and (@index = 0) {\n .col-@{class}-push-0 {\n left: auto;\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = pull) and (@index > 0) {\n .col-@{class}-pull-@{index} {\n right: percentage((@index / @grid-columns));\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = pull) and (@index = 0) {\n .col-@{class}-pull-0 {\n right: auto;\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = offset) {\n .col-@{class}-offset-@{index} {\n margin-left: percentage((@index / @grid-columns));\n }\n}\n\n// Basic looping in LESS\n.loop-grid-columns(@index, @class, @type) when (@index >= 0) {\n .calc-grid-column(@index, @class, @type);\n // next iteration\n .loop-grid-columns((@index - 1), @class, @type);\n}\n\n// Create grid for specific class\n.make-grid(@class) {\n .float-grid-columns(@class);\n .loop-grid-columns(@grid-columns, @class, width);\n .loop-grid-columns(@grid-columns, @class, pull);\n .loop-grid-columns(@grid-columns, @class, push);\n .loop-grid-columns(@grid-columns, @class, offset);\n}\n","//\n// Tables\n// --------------------------------------------------\n\n\ntable {\n background-color: @table-bg;\n}\ncaption {\n padding-top: @table-cell-padding;\n padding-bottom: @table-cell-padding;\n color: @text-muted;\n text-align: left;\n}\nth {\n text-align: left;\n}\n\n\n// Baseline styles\n\n.table {\n width: 100%;\n max-width: 100%;\n margin-bottom: @line-height-computed;\n // Cells\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th,\n > td {\n padding: @table-cell-padding;\n line-height: @line-height-base;\n vertical-align: top;\n border-top: 1px solid @table-border-color;\n }\n }\n }\n // Bottom align for column headings\n > thead > tr > th {\n vertical-align: bottom;\n border-bottom: 2px solid @table-border-color;\n }\n // Remove top border from thead by default\n > caption + thead,\n > colgroup + thead,\n > thead:first-child {\n > tr:first-child {\n > th,\n > td {\n border-top: 0;\n }\n }\n }\n // Account for multiple tbody instances\n > tbody + tbody {\n border-top: 2px solid @table-border-color;\n }\n\n // Nesting\n .table {\n background-color: @body-bg;\n }\n}\n\n\n// Condensed table w/ half padding\n\n.table-condensed {\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th,\n > td {\n padding: @table-condensed-cell-padding;\n }\n }\n }\n}\n\n\n// Bordered version\n//\n// Add borders all around the table and between all the columns.\n\n.table-bordered {\n border: 1px solid @table-border-color;\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th,\n > td {\n border: 1px solid @table-border-color;\n }\n }\n }\n > thead > tr {\n > th,\n > td {\n border-bottom-width: 2px;\n }\n }\n}\n\n\n// Zebra-striping\n//\n// Default zebra-stripe styles (alternating gray and transparent backgrounds)\n\n.table-striped {\n > tbody > tr:nth-of-type(odd) {\n background-color: @table-bg-accent;\n }\n}\n\n\n// Hover effect\n//\n// Placed here since it has to come after the potential zebra striping\n\n.table-hover {\n > tbody > tr:hover {\n background-color: @table-bg-hover;\n }\n}\n\n\n// Table cell sizing\n//\n// Reset default table behavior\n\ntable col[class*=\"col-\"] {\n position: static; // Prevent border hiding in Firefox and IE9-11 (see https://github.com/twbs/bootstrap/issues/11623)\n float: none;\n display: table-column;\n}\ntable {\n td,\n th {\n &[class*=\"col-\"] {\n position: static; // Prevent border hiding in Firefox and IE9-11 (see https://github.com/twbs/bootstrap/issues/11623)\n float: none;\n display: table-cell;\n }\n }\n}\n\n\n// Table backgrounds\n//\n// Exact selectors below required to override `.table-striped` and prevent\n// inheritance to nested tables.\n\n// Generate the contextual variants\n.table-row-variant(active; @table-bg-active);\n.table-row-variant(success; @state-success-bg);\n.table-row-variant(info; @state-info-bg);\n.table-row-variant(warning; @state-warning-bg);\n.table-row-variant(danger; @state-danger-bg);\n\n\n// Responsive tables\n//\n// Wrap your tables in `.table-responsive` and we'll make them mobile friendly\n// by enabling horizontal scrolling. Only applies <768px. Everything above that\n// will display normally.\n\n.table-responsive {\n overflow-x: auto;\n min-height: 0.01%; // Workaround for IE9 bug (see https://github.com/twbs/bootstrap/issues/14837)\n\n @media screen and (max-width: @screen-xs-max) {\n width: 100%;\n margin-bottom: (@line-height-computed * 0.75);\n overflow-y: hidden;\n -ms-overflow-style: -ms-autohiding-scrollbar;\n border: 1px solid @table-border-color;\n\n // Tighten up spacing\n > .table {\n margin-bottom: 0;\n\n // Ensure the content doesn't wrap\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th,\n > td {\n white-space: nowrap;\n }\n }\n }\n }\n\n // Special overrides for the bordered tables\n > .table-bordered {\n border: 0;\n\n // Nuke the appropriate borders so that the parent can handle them\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th:first-child,\n > td:first-child {\n border-left: 0;\n }\n > th:last-child,\n > td:last-child {\n border-right: 0;\n }\n }\n }\n\n // Only nuke the last row's bottom-border in `tbody` and `tfoot` since\n // chances are there will be only one `tr` in a `thead` and that would\n // remove the border altogether.\n > tbody,\n > tfoot {\n > tr:last-child {\n > th,\n > td {\n border-bottom: 0;\n }\n }\n }\n\n }\n }\n}\n","// Tables\n\n.table-row-variant(@state; @background) {\n // Exact selectors below required to override `.table-striped` and prevent\n // inheritance to nested tables.\n .table > thead > tr,\n .table > tbody > tr,\n .table > tfoot > tr {\n > td.@{state},\n > th.@{state},\n &.@{state} > td,\n &.@{state} > th {\n background-color: @background;\n }\n }\n\n // Hover states for `.table-hover`\n // Note: this is not available for cells or rows within `thead` or `tfoot`.\n .table-hover > tbody > tr {\n > td.@{state}:hover,\n > th.@{state}:hover,\n &.@{state}:hover > td,\n &:hover > .@{state},\n &.@{state}:hover > th {\n background-color: darken(@background, 5%);\n }\n }\n}\n","//\n// Forms\n// --------------------------------------------------\n\n\n// Normalize non-controls\n//\n// Restyle and baseline non-control form elements.\n\nfieldset {\n padding: 0;\n margin: 0;\n border: 0;\n // Chrome and Firefox set a `min-width: min-content;` on fieldsets,\n // so we reset that to ensure it behaves more like a standard block element.\n // See https://github.com/twbs/bootstrap/issues/12359.\n min-width: 0;\n}\n\nlegend {\n display: block;\n width: 100%;\n padding: 0;\n margin-bottom: @line-height-computed;\n font-size: (@font-size-base * 1.5);\n line-height: inherit;\n color: @legend-color;\n border: 0;\n border-bottom: 1px solid @legend-border-color;\n}\n\nlabel {\n display: inline-block;\n max-width: 100%; // Force IE8 to wrap long content (see https://github.com/twbs/bootstrap/issues/13141)\n margin-bottom: 5px;\n font-weight: bold;\n}\n\n\n// Normalize form controls\n//\n// While most of our form styles require extra classes, some basic normalization\n// is required to ensure optimum display with or without those classes to better\n// address browser inconsistencies.\n\n// Override content-box in Normalize (* isn't specific enough)\ninput[type=\"search\"] {\n .box-sizing(border-box);\n}\n\n// Position radios and checkboxes better\ninput[type=\"radio\"],\ninput[type=\"checkbox\"] {\n margin: 4px 0 0;\n margin-top: 1px \\9; // IE8-9\n line-height: normal;\n}\n\ninput[type=\"file\"] {\n display: block;\n}\n\n// Make range inputs behave like textual form controls\ninput[type=\"range\"] {\n display: block;\n width: 100%;\n}\n\n// Make multiple select elements height not fixed\nselect[multiple],\nselect[size] {\n height: auto;\n}\n\n// Focus for file, radio, and checkbox\ninput[type=\"file\"]:focus,\ninput[type=\"radio\"]:focus,\ninput[type=\"checkbox\"]:focus {\n .tab-focus();\n}\n\n// Adjust output element\noutput {\n display: block;\n padding-top: (@padding-base-vertical + 1);\n font-size: @font-size-base;\n line-height: @line-height-base;\n color: @input-color;\n}\n\n\n// Common form controls\n//\n// Shared size and type resets for form controls. Apply `.form-control` to any\n// of the following form controls:\n//\n// select\n// textarea\n// input[type=\"text\"]\n// input[type=\"password\"]\n// input[type=\"datetime\"]\n// input[type=\"datetime-local\"]\n// input[type=\"date\"]\n// input[type=\"month\"]\n// input[type=\"time\"]\n// input[type=\"week\"]\n// input[type=\"number\"]\n// input[type=\"email\"]\n// input[type=\"url\"]\n// input[type=\"search\"]\n// input[type=\"tel\"]\n// input[type=\"color\"]\n\n.form-control {\n display: block;\n width: 100%;\n height: @input-height-base; // Make inputs at least the height of their button counterpart (base line-height + padding + border)\n padding: @padding-base-vertical @padding-base-horizontal;\n font-size: @font-size-base;\n line-height: @line-height-base;\n color: @input-color;\n background-color: @input-bg;\n background-image: none; // Reset unusual Firefox-on-Android default style; see https://github.com/necolas/normalize.css/issues/214\n border: 1px solid @input-border;\n border-radius: @input-border-radius; // Note: This has no effect on s in CSS.\n .box-shadow(inset 0 1px 1px rgba(0,0,0,.075));\n .transition(~\"border-color ease-in-out .15s, box-shadow ease-in-out .15s\");\n\n // Customize the `:focus` state to imitate native WebKit styles.\n .form-control-focus();\n\n // Placeholder\n .placeholder();\n\n // Disabled and read-only inputs\n //\n // HTML5 says that controls under a fieldset > legend:first-child won't be\n // disabled if the fieldset is disabled. Due to implementation difficulty, we\n // don't honor that edge case; we style them as disabled anyway.\n &[disabled],\n &[readonly],\n fieldset[disabled] & {\n background-color: @input-bg-disabled;\n opacity: 1; // iOS fix for unreadable disabled content; see https://github.com/twbs/bootstrap/issues/11655\n }\n\n &[disabled],\n fieldset[disabled] & {\n cursor: @cursor-disabled;\n }\n\n // Reset height for `textarea`s\n textarea& {\n height: auto;\n }\n}\n\n\n// Search inputs in iOS\n//\n// This overrides the extra rounded corners on search inputs in iOS so that our\n// `.form-control` class can properly style them. Note that this cannot simply\n// be added to `.form-control` as it's not specific enough. For details, see\n// https://github.com/twbs/bootstrap/issues/11586.\n\ninput[type=\"search\"] {\n -webkit-appearance: none;\n}\n\n\n// Special styles for iOS temporal inputs\n//\n// In Mobile Safari, setting `display: block` on temporal inputs causes the\n// text within the input to become vertically misaligned. As a workaround, we\n// set a pixel line-height that matches the given height of the input, but only\n// for Safari. See https://bugs.webkit.org/show_bug.cgi?id=139848\n//\n// Note that as of 8.3, iOS doesn't support `datetime` or `week`.\n\n@media screen and (-webkit-min-device-pixel-ratio: 0) {\n input[type=\"date\"],\n input[type=\"time\"],\n input[type=\"datetime-local\"],\n input[type=\"month\"] {\n &.form-control {\n line-height: @input-height-base;\n }\n\n &.input-sm,\n .input-group-sm & {\n line-height: @input-height-small;\n }\n\n &.input-lg,\n .input-group-lg & {\n line-height: @input-height-large;\n }\n }\n}\n\n\n// Form groups\n//\n// Designed to help with the organization and spacing of vertical forms. For\n// horizontal forms, use the predefined grid classes.\n\n.form-group {\n margin-bottom: @form-group-margin-bottom;\n}\n\n\n// Checkboxes and radios\n//\n// Indent the labels to position radios/checkboxes as hanging controls.\n\n.radio,\n.checkbox {\n position: relative;\n display: block;\n margin-top: 10px;\n margin-bottom: 10px;\n\n label {\n min-height: @line-height-computed; // Ensure the input doesn't jump when there is no text\n padding-left: 20px;\n margin-bottom: 0;\n font-weight: normal;\n cursor: pointer;\n }\n}\n.radio input[type=\"radio\"],\n.radio-inline input[type=\"radio\"],\n.checkbox input[type=\"checkbox\"],\n.checkbox-inline input[type=\"checkbox\"] {\n position: absolute;\n margin-left: -20px;\n margin-top: 4px \\9;\n}\n\n.radio + .radio,\n.checkbox + .checkbox {\n margin-top: -5px; // Move up sibling radios or checkboxes for tighter spacing\n}\n\n// Radios and checkboxes on same line\n.radio-inline,\n.checkbox-inline {\n position: relative;\n display: inline-block;\n padding-left: 20px;\n margin-bottom: 0;\n vertical-align: middle;\n font-weight: normal;\n cursor: pointer;\n}\n.radio-inline + .radio-inline,\n.checkbox-inline + .checkbox-inline {\n margin-top: 0;\n margin-left: 10px; // space out consecutive inline controls\n}\n\n// Apply same disabled cursor tweak as for inputs\n// Some special care is needed because